From 531ac136bf4ed333cb906ac229d986605a8207a6 Mon Sep 17 00:00:00 2001 From: Denny Date: Mon, 29 Oct 2012 14:53:47 -0700 Subject: [PATCH 001/696] BlockManager UI. --- core/src/main/scala/spark/RDD.scala | 8 ++ core/src/main/scala/spark/SparkContext.scala | 10 ++ .../spark/storage/BlockManagerMaster.scala | 33 +++++- .../scala/spark/storage/BlockManagerUI.scala | 102 ++++++++++++++++++ .../src/main/scala/spark/util/AkkaUtils.scala | 5 +- .../{deploy => }/common/layout.scala.html | 0 .../spark/deploy/master/index.scala.html | 2 +- .../deploy/master/job_details.scala.html | 2 +- .../spark/deploy/worker/index.scala.html | 2 +- .../main/twirl/spark/storage/index.scala.html | 28 +++++ .../main/twirl/spark/storage/rdd.scala.html | 65 +++++++++++ .../twirl/spark/storage/rdd_row.scala.html | 18 ++++ .../twirl/spark/storage/rdd_table.scala.html | 18 ++++ 13 files changed, 283 insertions(+), 10 deletions(-) create mode 100644 core/src/main/scala/spark/storage/BlockManagerUI.scala rename core/src/main/twirl/spark/{deploy => }/common/layout.scala.html (100%) create mode 100644 core/src/main/twirl/spark/storage/index.scala.html create mode 100644 core/src/main/twirl/spark/storage/rdd.scala.html create mode 100644 core/src/main/twirl/spark/storage/rdd_row.scala.html create mode 100644 core/src/main/twirl/spark/storage/rdd_table.scala.html diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 338dff40618cc..dc757dc6aa94b 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -107,6 +107,12 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial // Variables relating to persistence private var storageLevel: StorageLevel = StorageLevel.NONE + /* Assign a name to this RDD */ + def name(name: String) = { + sc.rddNames(this.id) = name + this + } + /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. Can only be called once on each RDD. @@ -118,6 +124,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial "Cannot change storage level of an RDD after it was already assigned a level") } storageLevel = newLevel + // Register the RDD with the SparkContext + sc.persistentRdds(id) = this this } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index d26cccbfe196d..71c9dcd017fc2 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,6 +1,7 @@ package spark import java.io._ +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger import java.net.{URI, URLClassLoader} @@ -102,10 +103,19 @@ class SparkContext( isLocal) SparkEnv.set(env) + // Start the BlockManager UI + spark.storage.BlockManagerUI.start(SparkEnv.get.actorSystem, + SparkEnv.get.blockManager.master.masterActor, this) + // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() private[spark] val addedJars = HashMap[String, Long]() + // Keeps track of all persisted RDDs + private[spark] val persistentRdds = new ConcurrentHashMap[Int, RDD[_]]() + // A HashMap for friendly RDD Names + private[spark] val rddNames = new ConcurrentHashMap[Int, String]() + // Add each JAR given through the constructor jars.foreach { addJar(_) } diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index ace27e758c4bf..d12a16869aa5c 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -3,7 +3,8 @@ package spark.storage import java.io._ import java.util.{HashMap => JHashMap} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.util.Random import akka.actor._ @@ -90,6 +91,15 @@ case object StopBlockManagerMaster extends ToBlockManagerMaster private[spark] case object GetMemoryStatus extends ToBlockManagerMaster +private[spark] +case class GetStorageStatus extends ToBlockManagerMaster + +private[spark] +case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) + +private[spark] +case class StorageStatus(maxMem: Long, remainingMem: Long, blocks: Map[String, BlockStatus]) + private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { @@ -99,7 +109,8 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor val maxMem: Long) { private var _lastSeenMs = timeMs private var _remainingMem = maxMem - private val _blocks = new JHashMap[String, StorageLevel] + + private val _blocks = new JHashMap[String, BlockStatus] logInfo("Registering block manager %s:%d with %s RAM".format( blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(maxMem))) @@ -115,7 +126,7 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor if (_blocks.containsKey(blockId)) { // The block exists on the slave already. - val originalLevel: StorageLevel = _blocks.get(blockId) + val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel if (originalLevel.useMemory) { _remainingMem += memSize @@ -124,7 +135,7 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor if (storageLevel.isValid) { // isValid means it is either stored in-memory or on-disk. - _blocks.put(blockId, storageLevel) + _blocks.put(blockId, BlockStatus(storageLevel, memSize, diskSize)) if (storageLevel.useMemory) { _remainingMem -= memSize logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format( @@ -137,7 +148,7 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. - val originalLevel: StorageLevel = _blocks.get(blockId) + val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel _blocks.remove(blockId) if (originalLevel.useMemory) { _remainingMem += memSize @@ -152,6 +163,8 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor } } + def blocks: JHashMap[String, BlockStatus] = _blocks + def remainingMem: Long = _remainingMem def lastSeenMs: Long = _lastSeenMs @@ -198,6 +211,9 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor case GetMemoryStatus => getMemoryStatus + case GetStorageStatus => + getStorageStatus + case RemoveHost(host) => removeHost(host) sender ! true @@ -219,6 +235,13 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor sender ! res } + private def getStorageStatus() { + val res = blockManagerInfo.map { case(blockManagerId, info) => + StorageStatus(info.maxMem, info.remainingMem, info.blocks.asScala) + } + sender ! res + } + private def register(blockManagerId: BlockManagerId, maxMemSize: Long) { val startTimeMs = System.currentTimeMillis() val tmp = " " + blockManagerId + " " diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala new file mode 100644 index 0000000000000..c168f60c35cab --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -0,0 +1,102 @@ +package spark.storage + +import akka.actor.{ActorRef, ActorSystem} +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.Timeout +import akka.util.duration._ +import cc.spray.Directives +import cc.spray.directives._ +import cc.spray.typeconversion.TwirlSupport._ +import scala.collection.mutable.ArrayBuffer +import spark.{Logging, SparkContext, SparkEnv} +import spark.util.AkkaUtils + +private[spark] +object BlockManagerUI extends Logging { + + /* Starts the Web interface for the BlockManager */ + def start(actorSystem : ActorSystem, masterActor: ActorRef, sc: SparkContext) { + val webUIDirectives = new BlockManagerUIDirectives(actorSystem, masterActor, sc) + try { + logInfo("Starting BlockManager WebUI.") + val port = Option(System.getenv("BLOCKMANAGER_UI_PORT")).getOrElse("9080").toInt + AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, webUIDirectives.handler, "BlockManagerHTTPServer") + } catch { + case e: Exception => + logError("Failed to create BlockManager WebUI", e) + System.exit(1) + } + } + +} + +private[spark] +case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, numPartitions: Int, memSize: Long, diskSize: Long) + +private[spark] +class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, sc: SparkContext) extends Directives { + + val STATIC_RESOURCE_DIR = "spark/deploy/static" + implicit val timeout = Timeout(1 seconds) + + val handler = { + + get { path("") { completeWith { + // Request the current storage status from the Master + val future = master ? GetStorageStatus + future.map { status => + val storageStati = status.asInstanceOf[ArrayBuffer[StorageStatus]] + + // Calculate macro-level statistics + val maxMem = storageStati.map(_.maxMem).reduce(_+_) + val remainingMem = storageStati.map(_.remainingMem).reduce(_+_) + val diskSpaceUsed = storageStati.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + + // Filter out everything that's not and rdd. + val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => k.startsWith("rdd") }.toMap + val rdds = rddInfoFromBlockStati(rddBlocks) + + spark.storage.html.index.render(maxMem, remainingMem, diskSpaceUsed, rdds.toList) + } + }}} ~ + get { path("rdd") { parameter("id") { id => { completeWith { + val future = master ? GetStorageStatus + future.map { status => + val prefix = "rdd_" + id.toString + + val storageStati = status.asInstanceOf[ArrayBuffer[StorageStatus]] + val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => k.startsWith(prefix) }.toMap + val rddInfo = rddInfoFromBlockStati(rddBlocks).first + + spark.storage.html.rdd.render(rddInfo, rddBlocks) + + } + }}}}} ~ + pathPrefix("static") { + getFromResourceDirectory(STATIC_RESOURCE_DIR) + } + + } + + private def rddInfoFromBlockStati(infos: Map[String, BlockStatus]) : Array[RDDInfo] = { + infos.groupBy { case(k,v) => + // Group by rdd name, ignore the partition name + k.substring(0,k.lastIndexOf('_')) + }.map { case(k,v) => + val blockStati = v.map(_._2).toArray + // Add up memory and disk sizes + val tmp = blockStati.map { x => (x.memSize, x.diskSize)}.reduce { (x,y) => + (x._1 + y._1, x._2 + y._2) + } + // Get the friendly name for the rdd, if available. + // This is pretty hacky, is there a better way? + val rddId = k.split("_").last.toInt + val rddName : String = Option(sc.rddNames.get(rddId)).getOrElse(k) + val rddStorageLevel = sc.persistentRdds.get(rddId).getStorageLevel + RDDInfo(rddId, rddName, rddStorageLevel, blockStati.length, tmp._1, tmp._2) + }.toArray + } + +} diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index b466b5239cf27..13bc0f8ccc0c9 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -50,12 +50,13 @@ private[spark] object AkkaUtils { * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to * handle requests. Throws a SparkException if this fails. */ - def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) { + def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, + name: String = "HttpServer") { val ioWorker = new IoWorker(actorSystem).start() val httpService = actorSystem.actorOf(Props(new HttpService(route))) val rootService = actorSystem.actorOf(Props(new SprayCanRootService(httpService))) val server = actorSystem.actorOf( - Props(new HttpServer(ioWorker, SingletonHandler(rootService))), name = "HttpServer") + Props(new HttpServer(ioWorker, SingletonHandler(rootService))), name = name) actorSystem.registerOnTermination { ioWorker.stop() } val timeout = 3.seconds val future = server.ask(HttpServer.Bind(ip, port))(timeout) diff --git a/core/src/main/twirl/spark/deploy/common/layout.scala.html b/core/src/main/twirl/spark/common/layout.scala.html similarity index 100% rename from core/src/main/twirl/spark/deploy/common/layout.scala.html rename to core/src/main/twirl/spark/common/layout.scala.html diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html index 7562076b0087a..2e15fe22003c5 100644 --- a/core/src/main/twirl/spark/deploy/master/index.scala.html +++ b/core/src/main/twirl/spark/deploy/master/index.scala.html @@ -1,7 +1,7 @@ @(state: spark.deploy.MasterState) @import spark.deploy.master._ -@spark.deploy.common.html.layout(title = "Spark Master on " + state.uri) { +@spark.common.html.layout(title = "Spark Master on " + state.uri) {
diff --git a/core/src/main/twirl/spark/deploy/master/job_details.scala.html b/core/src/main/twirl/spark/deploy/master/job_details.scala.html index dcf41c28f26f5..d02a51b214180 100644 --- a/core/src/main/twirl/spark/deploy/master/job_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/job_details.scala.html @@ -1,6 +1,6 @@ @(job: spark.deploy.master.JobInfo) -@spark.deploy.common.html.layout(title = "Job Details") { +@spark.common.html.layout(title = "Job Details") {
diff --git a/core/src/main/twirl/spark/deploy/worker/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html index 69746ed02cfed..40c2d81d77721 100644 --- a/core/src/main/twirl/spark/deploy/worker/index.scala.html +++ b/core/src/main/twirl/spark/deploy/worker/index.scala.html @@ -1,6 +1,6 @@ @(worker: spark.deploy.WorkerState) -@spark.deploy.common.html.layout(title = "Spark Worker on " + worker.uri) { +@spark.common.html.layout(title = "Spark Worker on " + worker.uri) {
diff --git a/core/src/main/twirl/spark/storage/index.scala.html b/core/src/main/twirl/spark/storage/index.scala.html new file mode 100644 index 0000000000000..fa7dad51ee3f8 --- /dev/null +++ b/core/src/main/twirl/spark/storage/index.scala.html @@ -0,0 +1,28 @@ +@(maxMem: Long, remainingMem: Long, diskSpaceUsed: Long, rdds: List[spark.storage.RDDInfo]) + +@spark.common.html.layout(title = "Storage Dashboard") { + + +
+
+
    +
  • Memory: + @{spark.Utils.memoryBytesToString(maxMem - remainingMem)} Used + (@{spark.Utils.memoryBytesToString(remainingMem)} Available)
  • +
  • Disk: @{spark.Utils.memoryBytesToString(diskSpaceUsed)} Used
  • +
+
+
+ +
+ + +
+
+

RDD Summary

+
+ @rdd_table(rdds) +
+
+ +} \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd.scala.html b/core/src/main/twirl/spark/storage/rdd.scala.html new file mode 100644 index 0000000000000..3a70326efeeab --- /dev/null +++ b/core/src/main/twirl/spark/storage/rdd.scala.html @@ -0,0 +1,65 @@ +@(rddInfo: spark.storage.RDDInfo, blocks: Map[String, spark.storage.BlockStatus]) + +@spark.common.html.layout(title = "RDD Info ") { + + +
+
+
    +
  • + Storage Level: + @(if (rddInfo.storageLevel.useDisk) "Disk" else "") + @(if (rddInfo.storageLevel.useMemory) "Memory" else "") + @(if (rddInfo.storageLevel.deserialized) "Deserialized" else "") + @(rddInfo.storageLevel.replication)x Replicated +
  • + Partitions: + @(rddInfo.numPartitions) +
  • +
  • + Memory Size: + @{spark.Utils.memoryBytesToString(rddInfo.memSize)} +
  • +
  • + Disk Size: + @{spark.Utils.memoryBytesToString(rddInfo.diskSize)} +
  • +
+
+
+ +
+ + +
+
+

RDD Summary

+
+ + + + + + + + + + + + + @blocks.map { case (k,v) => + + + + + + + } + +
Block NameStorage LevelSize in MemorySize on Disk
@k@v.storageLevel@{spark.Utils.memoryBytesToString(v.memSize)}@{spark.Utils.memoryBytesToString(v.diskSize)}
+ + +
+
+ +} \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd_row.scala.html b/core/src/main/twirl/spark/storage/rdd_row.scala.html new file mode 100644 index 0000000000000..3dd9944e3b3f8 --- /dev/null +++ b/core/src/main/twirl/spark/storage/rdd_row.scala.html @@ -0,0 +1,18 @@ +@(rdd: spark.storage.RDDInfo) + + + + + @rdd.name + + + + @(if (rdd.storageLevel.useDisk) "Disk" else "") + @(if (rdd.storageLevel.useMemory) "Memory" else "") + @(if (rdd.storageLevel.deserialized) "Deserialized" else "") + @(rdd.storageLevel.replication)x Replicated + + @rdd.numPartitions + @{spark.Utils.memoryBytesToString(rdd.memSize)} + @{spark.Utils.memoryBytesToString(rdd.diskSize)} + \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd_table.scala.html b/core/src/main/twirl/spark/storage/rdd_table.scala.html new file mode 100644 index 0000000000000..24f55ccefb25a --- /dev/null +++ b/core/src/main/twirl/spark/storage/rdd_table.scala.html @@ -0,0 +1,18 @@ +@(rdds: List[spark.storage.RDDInfo]) + + + + + + + + + + + + + @for(rdd <- rdds) { + @rdd_row(rdd) + } + +
RDD NameStorage LevelPartitionsSize in MemorySize on Disk
\ No newline at end of file From eb95212f4d24dbcd734922f39d51e6fdeaeb4c8b Mon Sep 17 00:00:00 2001 From: Denny Date: Mon, 29 Oct 2012 14:57:32 -0700 Subject: [PATCH 002/696] code Formatting --- .../scala/spark/storage/BlockManagerUI.scala | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index c168f60c35cab..635c096c8713d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -21,7 +21,8 @@ object BlockManagerUI extends Logging { try { logInfo("Starting BlockManager WebUI.") val port = Option(System.getenv("BLOCKMANAGER_UI_PORT")).getOrElse("9080").toInt - AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, webUIDirectives.handler, "BlockManagerHTTPServer") + AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, + webUIDirectives.handler, "BlockManagerHTTPServer") } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) @@ -32,10 +33,12 @@ object BlockManagerUI extends Logging { } private[spark] -case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, numPartitions: Int, memSize: Long, diskSize: Long) +case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, + numPartitions: Int, memSize: Long, diskSize: Long) private[spark] -class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, sc: SparkContext) extends Directives { +class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, + sc: SparkContext) extends Directives { val STATIC_RESOURCE_DIR = "spark/deploy/static" implicit val timeout = Timeout(1 seconds) @@ -55,7 +58,9 @@ class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, s .reduceOption(_+_).getOrElse(0L) // Filter out everything that's not and rdd. - val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => k.startsWith("rdd") }.toMap + val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => + k.startsWith("rdd") + }.toMap val rdds = rddInfoFromBlockStati(rddBlocks) spark.storage.html.index.render(maxMem, remainingMem, diskSpaceUsed, rdds.toList) @@ -67,7 +72,9 @@ class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, s val prefix = "rdd_" + id.toString val storageStati = status.asInstanceOf[ArrayBuffer[StorageStatus]] - val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => k.startsWith(prefix) }.toMap + val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => + k.startsWith(prefix) + }.toMap val rddInfo = rddInfoFromBlockStati(rddBlocks).first spark.storage.html.rdd.render(rddInfo, rddBlocks) From ceec1a1a6abb1fd03316e7fcc532d7e121d5bf65 Mon Sep 17 00:00:00 2001 From: Denny Date: Mon, 29 Oct 2012 15:03:01 -0700 Subject: [PATCH 003/696] Nicer storage level format on RDD page --- core/src/main/twirl/spark/storage/rdd.scala.html | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/twirl/spark/storage/rdd.scala.html b/core/src/main/twirl/spark/storage/rdd.scala.html index 3a70326efeeab..075289c826355 100644 --- a/core/src/main/twirl/spark/storage/rdd.scala.html +++ b/core/src/main/twirl/spark/storage/rdd.scala.html @@ -50,7 +50,12 @@

RDD Summary

@blocks.map { case (k,v) => @k - @v.storageLevel + + @(if (v.storageLevel.useDisk) "Disk" else "") + @(if (v.storageLevel.useMemory) "Memory" else "") + @(if (v.storageLevel.deserialized) "Deserialized" else "") + @(v.storageLevel.replication)x Replicated + @{spark.Utils.memoryBytesToString(v.memSize)} @{spark.Utils.memoryBytesToString(v.diskSize)} From 4a1be7e0dbf0031d85b91dc1132fe101d87ba097 Mon Sep 17 00:00:00 2001 From: Denny Date: Mon, 12 Nov 2012 10:56:35 -0800 Subject: [PATCH 004/696] Refactor BlockManager UI and adding worker details. --- core/src/main/scala/spark/RDD.scala | 7 +- core/src/main/scala/spark/SparkContext.scala | 2 - .../spark/storage/BlockManagerMaster.scala | 11 +-- .../scala/spark/storage/BlockManagerUI.scala | 51 ++++-------- .../scala/spark/storage/StorageLevel.scala | 9 +++ .../scala/spark/storage/StorageUtils.scala | 78 +++++++++++++++++++ .../main/twirl/spark/storage/index.scala.html | 22 ++++-- .../main/twirl/spark/storage/rdd.scala.html | 35 +++++---- .../twirl/spark/storage/rdd_row.scala.html | 18 ----- .../twirl/spark/storage/rdd_table.scala.html | 16 +++- .../spark/storage/worker_table.scala.html | 24 ++++++ 11 files changed, 186 insertions(+), 87 deletions(-) create mode 100644 core/src/main/scala/spark/storage/StorageUtils.scala delete mode 100644 core/src/main/twirl/spark/storage/rdd_row.scala.html create mode 100644 core/src/main/twirl/spark/storage/worker_table.scala.html diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index dc757dc6aa94b..3669bda2d2559 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -86,6 +86,9 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial @transient val dependencies: List[Dependency[_]] // Methods available on all RDDs: + + // A friendly name for this RDD + var name: String = null /** Record user function generating this RDD. */ private[spark] val origin = Utils.getSparkCallSite @@ -108,8 +111,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial private var storageLevel: StorageLevel = StorageLevel.NONE /* Assign a name to this RDD */ - def name(name: String) = { - sc.rddNames(this.id) = name + def setName(_name: String) = { + name = _name this } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 71c9dcd017fc2..7ea0f6f9e0d89 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -113,8 +113,6 @@ class SparkContext( // Keeps track of all persisted RDDs private[spark] val persistentRdds = new ConcurrentHashMap[Int, RDD[_]]() - // A HashMap for friendly RDD Names - private[spark] val rddNames = new ConcurrentHashMap[Int, String]() // Add each JAR given through the constructor jars.foreach { addJar(_) } diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 3fc9b629c1d58..beafdda9d1132 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -4,7 +4,7 @@ import java.io._ import java.util.{HashMap => JHashMap} import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Random import akka.actor._ @@ -95,10 +95,7 @@ private[spark] case class GetStorageStatus extends ToBlockManagerMaster private[spark] -case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) - -private[spark] -case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, remainingMem: Long, blocks: Map[String, BlockStatus]) +case class BlockStatus(blockManagerId: BlockManagerId, storageLevel: StorageLevel, memSize: Long, diskSize: Long) private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { @@ -135,7 +132,7 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor if (storageLevel.isValid) { // isValid means it is either stored in-memory or on-disk. - _blocks.put(blockId, BlockStatus(storageLevel, memSize, diskSize)) + _blocks.put(blockId, BlockStatus(blockManagerId, storageLevel, memSize, diskSize)) if (storageLevel.useMemory) { _remainingMem -= memSize logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format( @@ -237,7 +234,7 @@ private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor private def getStorageStatus() { val res = blockManagerInfo.map { case(blockManagerId, info) => - StorageStatus(blockManagerId, info.maxMem, info.remainingMem, info.blocks.asScala) + StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap) } sender ! res } diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 635c096c8713d..35cbd59280dce 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -12,6 +12,7 @@ import scala.collection.mutable.ArrayBuffer import spark.{Logging, SparkContext, SparkEnv} import spark.util.AkkaUtils + private[spark] object BlockManagerUI extends Logging { @@ -32,9 +33,6 @@ object BlockManagerUI extends Logging { } -private[spark] -case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numPartitions: Int, memSize: Long, diskSize: Long) private[spark] class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, @@ -49,21 +47,17 @@ class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, // Request the current storage status from the Master val future = master ? GetStorageStatus future.map { status => - val storageStati = status.asInstanceOf[ArrayBuffer[StorageStatus]] + val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray // Calculate macro-level statistics - val maxMem = storageStati.map(_.maxMem).reduce(_+_) - val remainingMem = storageStati.map(_.remainingMem).reduce(_+_) - val diskSpaceUsed = storageStati.flatMap(_.blocks.values.map(_.diskSize)) + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - // Filter out everything that's not and rdd. - val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => - k.startsWith("rdd") - }.toMap - val rdds = rddInfoFromBlockStati(rddBlocks) + val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - spark.storage.html.index.render(maxMem, remainingMem, diskSpaceUsed, rdds.toList) + spark.storage.html.index.render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) } }}} ~ get { path("rdd") { parameter("id") { id => { completeWith { @@ -71,13 +65,13 @@ class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, future.map { status => val prefix = "rdd_" + id.toString - val storageStati = status.asInstanceOf[ArrayBuffer[StorageStatus]] - val rddBlocks = storageStati.flatMap(_.blocks).filter { case(k,v) => - k.startsWith(prefix) - }.toMap - val rddInfo = rddInfoFromBlockStati(rddBlocks).first - spark.storage.html.rdd.render(rddInfo, rddBlocks) + val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray + val filteredStorageStatusList = StorageUtils.filterStorageStatusByPrefix(storageStatusList, prefix) + + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).first + + spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) } }}}}} ~ @@ -87,23 +81,6 @@ class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, } - private def rddInfoFromBlockStati(infos: Map[String, BlockStatus]) : Array[RDDInfo] = { - infos.groupBy { case(k,v) => - // Group by rdd name, ignore the partition name - k.substring(0,k.lastIndexOf('_')) - }.map { case(k,v) => - val blockStati = v.map(_._2).toArray - // Add up memory and disk sizes - val tmp = blockStati.map { x => (x.memSize, x.diskSize)}.reduce { (x,y) => - (x._1 + y._1, x._2 + y._2) - } - // Get the friendly name for the rdd, if available. - // This is pretty hacky, is there a better way? - val rddId = k.split("_").last.toInt - val rddName : String = Option(sc.rddNames.get(rddId)).getOrElse(k) - val rddStorageLevel = sc.persistentRdds.get(rddId).getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, blockStati.length, tmp._1, tmp._2) - }.toArray - } + } diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index c497f03e0c3e7..97d8c7566deea 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -68,6 +68,15 @@ class StorageLevel( override def toString: String = "StorageLevel(%b, %b, %b, %d)".format(useDisk, useMemory, deserialized, replication) + + def description : String = { + var result = "" + result += (if (useDisk) "Disk " else "") + result += (if (useMemory) "Memory " else "") + result += (if (deserialized) "Deserialized " else "Serialized") + result += "%sx Replicated".format(replication) + result + } } object StorageLevel { diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala new file mode 100644 index 0000000000000..ebc7390ee598c --- /dev/null +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -0,0 +1,78 @@ +package spark.storage + +import spark.SparkContext + +private[spark] +case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, + blocks: Map[String, BlockStatus]) { + + def memUsed(blockPrefix: String = "") = { + blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.memSize). + reduceOption(_+_).getOrElse(0l) + } + + def diskUsed(blockPrefix: String = "") = { + blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.diskSize). + reduceOption(_+_).getOrElse(0l) + } + + def memRemaining : Long = maxMem - memUsed() + +} + +case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, + numPartitions: Int, memSize: Long, diskSize: Long, locations: Array[BlockManagerId]) + + +/* Helper methods for storage-related objects */ +private[spark] +object StorageUtils { + + /* Given the current storage status of the BlockManager, returns information for each RDD */ + def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], + sc: SparkContext) : Array[RDDInfo] = { + rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) + } + + /* Given a list of BlockStatus objets, returns information for each RDD */ + def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], + sc: SparkContext) : Array[RDDInfo] = { + // Find all RDD Blocks (ignore broadcast variables) + val rddBlocks = infos.filterKeys(_.startsWith("rdd")) + + // Group by rddId, ignore the partition name + val groupedRddBlocks = infos.groupBy { case(k, v) => + k.substring(0,k.lastIndexOf('_')) + }.mapValues(_.values.toArray) + + // For each RDD, generate an RDDInfo object + groupedRddBlocks.map { case(rddKey, rddBlocks) => + + // Add up memory and disk sizes + val memSize = rddBlocks.map(_.memSize).reduce(_ + _) + val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _) + + // Find the id of the RDD, e.g. rdd_1 => 1 + val rddId = rddKey.split("_").last.toInt + // Get the friendly name for the rdd, if available. + val rddName = Option(sc.persistentRdds.get(rddId).name).getOrElse(rddKey) + val rddStorageLevel = sc.persistentRdds.get(rddId).getStorageLevel + + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, memSize, diskSize, + rddBlocks.map(_.blockManagerId)) + }.toArray + } + + /* Removes all BlockStatus object that are not part of a block prefix */ + def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus], + prefix: String) : Array[StorageStatus] = { + + storageStatusList.map { status => + val newBlocks = status.blocks.filterKeys(_.startsWith(prefix)) + //val newRemainingMem = status.maxMem - newBlocks.values.map(_.memSize).reduce(_ + _) + StorageStatus(status.blockManagerId, status.maxMem, newBlocks) + } + + } + +} \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/index.scala.html b/core/src/main/twirl/spark/storage/index.scala.html index fa7dad51ee3f8..2b337f61339b2 100644 --- a/core/src/main/twirl/spark/storage/index.scala.html +++ b/core/src/main/twirl/spark/storage/index.scala.html @@ -1,4 +1,5 @@ -@(maxMem: Long, remainingMem: Long, diskSpaceUsed: Long, rdds: List[spark.storage.RDDInfo]) +@(maxMem: Long, remainingMem: Long, diskSpaceUsed: Long, rdds: Array[spark.storage.RDDInfo], storageStatusList: Array[spark.storage.StorageStatus]) +@import spark.Utils @spark.common.html.layout(title = "Storage Dashboard") { @@ -7,16 +8,16 @@
  • Memory: - @{spark.Utils.memoryBytesToString(maxMem - remainingMem)} Used - (@{spark.Utils.memoryBytesToString(remainingMem)} Available)
  • -
  • Disk: @{spark.Utils.memoryBytesToString(diskSpaceUsed)} Used
  • + @{Utils.memoryBytesToString(maxMem - remainingMem)} Used + (@{Utils.memoryBytesToString(remainingMem)} Available) +
  • Disk: @{Utils.memoryBytesToString(diskSpaceUsed)} Used

- +

RDD Summary

@@ -25,4 +26,15 @@

RDD Summary

+
+ + +
+
+

Worker Summary

+
+ @worker_table(storageStatusList) +
+
+ } \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd.scala.html b/core/src/main/twirl/spark/storage/rdd.scala.html index 075289c826355..ac7f8c981fa83 100644 --- a/core/src/main/twirl/spark/storage/rdd.scala.html +++ b/core/src/main/twirl/spark/storage/rdd.scala.html @@ -1,4 +1,5 @@ -@(rddInfo: spark.storage.RDDInfo, blocks: Map[String, spark.storage.BlockStatus]) +@(rddInfo: spark.storage.RDDInfo, storageStatusList: Array[spark.storage.StorageStatus]) +@import spark.Utils @spark.common.html.layout(title = "RDD Info ") { @@ -8,21 +9,18 @@
  • Storage Level: - @(if (rddInfo.storageLevel.useDisk) "Disk" else "") - @(if (rddInfo.storageLevel.useMemory) "Memory" else "") - @(if (rddInfo.storageLevel.deserialized) "Deserialized" else "") - @(rddInfo.storageLevel.replication)x Replicated + @(rddInfo.storageLevel.description)
  • Partitions: @(rddInfo.numPartitions)
  • Memory Size: - @{spark.Utils.memoryBytesToString(rddInfo.memSize)} + @{Utils.memoryBytesToString(rddInfo.memSize)}
  • Disk Size: - @{spark.Utils.memoryBytesToString(rddInfo.diskSize)} + @{Utils.memoryBytesToString(rddInfo.diskSize)}
@@ -36,6 +34,7 @@

RDD Summary


+ @@ -47,17 +46,14 @@

RDD Summary

- @blocks.map { case (k,v) => + @storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => - - + + } @@ -67,4 +63,15 @@

RDD Summary

+
+ + +
+
+

Worker Summary

+
+ @worker_table(storageStatusList, "rdd_" + rddInfo.id ) +
+
+ } \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd_row.scala.html b/core/src/main/twirl/spark/storage/rdd_row.scala.html deleted file mode 100644 index 3dd9944e3b3f8..0000000000000 --- a/core/src/main/twirl/spark/storage/rdd_row.scala.html +++ /dev/null @@ -1,18 +0,0 @@ -@(rdd: spark.storage.RDDInfo) - - - - - - - - \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd_table.scala.html b/core/src/main/twirl/spark/storage/rdd_table.scala.html index 24f55ccefb25a..af801cf229569 100644 --- a/core/src/main/twirl/spark/storage/rdd_table.scala.html +++ b/core/src/main/twirl/spark/storage/rdd_table.scala.html @@ -1,4 +1,5 @@ -@(rdds: List[spark.storage.RDDInfo]) +@(rdds: Array[spark.storage.RDDInfo]) +@import spark.Utils
@k - @(if (v.storageLevel.useDisk) "Disk" else "") - @(if (v.storageLevel.useMemory) "Memory" else "") - @(if (v.storageLevel.deserialized) "Deserialized" else "") - @(v.storageLevel.replication)x Replicated + @(v.storageLevel.description) @{spark.Utils.memoryBytesToString(v.memSize)}@{spark.Utils.memoryBytesToString(v.diskSize)}@{Utils.memoryBytesToString(v.memSize)}@{Utils.memoryBytesToString(v.diskSize)}
- - @rdd.name - - - @(if (rdd.storageLevel.useDisk) "Disk" else "") - @(if (rdd.storageLevel.useMemory) "Memory" else "") - @(if (rdd.storageLevel.deserialized) "Deserialized" else "") - @(rdd.storageLevel.replication)x Replicated - @rdd.numPartitions@{spark.Utils.memoryBytesToString(rdd.memSize)}@{spark.Utils.memoryBytesToString(rdd.diskSize)}
@@ -12,7 +13,18 @@ @for(rdd <- rdds) { - @rdd_row(rdd) + + + + + + + }
+ + @rdd.name + + @(rdd.storageLevel.description) + @rdd.numPartitions@{Utils.memoryBytesToString(rdd.memSize)}@{Utils.memoryBytesToString(rdd.diskSize)}
\ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/worker_table.scala.html b/core/src/main/twirl/spark/storage/worker_table.scala.html new file mode 100644 index 0000000000000..d54b8de4cc813 --- /dev/null +++ b/core/src/main/twirl/spark/storage/worker_table.scala.html @@ -0,0 +1,24 @@ +@(workersStatusList: Array[spark.storage.StorageStatus], prefix: String = "") +@import spark.Utils + + + + + + + + + + + @for(status <- workersStatusList) { + + + + + + } + +
HostMemory UsageDisk Usage
@(status.blockManagerId.ip + ":" + status.blockManagerId.port) + @(Utils.memoryBytesToString(status.memUsed(prefix))) + (@(Utils.memoryBytesToString(status.memRemaining)) Total Available) + @(Utils.memoryBytesToString(status.diskUsed(prefix)))
\ No newline at end of file From b61a4ec77300d6e7fb40f771a9054ae8bc4488de Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 14 Jan 2013 17:13:10 -0700 Subject: [PATCH 005/696] Removing offset management code that is non-existent in kafka 0.7.0+ --- .../scala/spark/streaming/dstream/KafkaInputDStream.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 2b4740bdf7c7d..9605072382c5f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -173,13 +173,6 @@ class KafkaReceiver(host: String, port: Int, groupId: String, stream.takeWhile { msgAndMetadata => blockGenerator += msgAndMetadata.message - // Updating the offet. The key is (broker, topic, group, partition). - val key = KafkaPartitionKey(msgAndMetadata.topicInfo.brokerId, msgAndMetadata.topic, - groupId, msgAndMetadata.topicInfo.partition.partId) - val offset = msgAndMetadata.topicInfo.getConsumeOffset - offsets.put(key, offset) - // logInfo("Handled message: " + (key, offset).toString) - // Keep on handling messages true } From c203a292963a018bd9b84f02bb522fd191a110af Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 14 Jan 2013 17:22:03 -0700 Subject: [PATCH 006/696] StateDStream changes to give updateStateByKey consistent behavior --- .../scala/spark/streaming/dstream/StateDStream.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala index a1ec2f5454f1c..4e57968eed017 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala @@ -48,8 +48,16 @@ class StateDStream[K: ClassManifest, V: ClassManifest, S <: AnyRef : ClassManife //logDebug("Generating state RDD for time " + validTime) return Some(stateRDD) } - case None => { // If parent RDD does not exist, then return old state RDD - return Some(prevStateRDD) + case None => { // If parent RDD does not exist + + // Re-apply the update function to the old state RDD + val updateFuncLocal = updateFunc + val finalFunc = (iterator: Iterator[(K, S)]) => { + val i = iterator.map(t => (t._1, Seq[V](), Option(t._2))) + updateFuncLocal(i) + } + val stateRDD = prevStateRDD.mapPartitions(finalFunc, preservePartitioning) + return Some(stateRDD) } } } From 74d3b23929758328c2a7879381669d81bf899396 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 15 Jan 2013 14:03:28 -0600 Subject: [PATCH 007/696] Add spark.executor.memory to differentiate executor memory from spark-shell memory. --- core/src/main/scala/spark/SparkContext.scala | 4 ++-- .../scala/spark/deploy/worker/ExecutorRunner.scala | 3 +-- .../cluster/SparkDeploySchedulerBackend.scala | 11 +++++------ 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index bbf8272eb39e1..a5a1b759449fb 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -111,8 +111,8 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() - for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", - "SPARK_TESTING")) { + // Note: SPARK_MEM isn't included because it's set directly in ExecutorRunner + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index beceb55ecdf54..2f2ea617ff1fc 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -118,8 +118,7 @@ private[spark] class ExecutorRunner( for ((key, value) <- jobDesc.command.environment) { env.put(key, value) } - env.put("SPARK_CORES", cores.toString) - env.put("SPARK_MEMORY", memory.toString) + env.put("SPARK_MEM", memory.toString) // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index e2301347e5103..f2fb244b2451d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -23,12 +23,11 @@ private[spark] class SparkDeploySchedulerBackend( // Memory used by each executor (in megabytes) val executorMemory = { - if (System.getenv("SPARK_MEM") != null) { - Utils.memoryStringToMb(System.getenv("SPARK_MEM")) - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - } else { - 512 - } + // TODO: Might need to add some extra memory for the non-heap parts of the JVM + Option(System.getProperty("spark.executor.memory")) + .orElse(Option(System.getenv("SPARK_MEM"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) } override def start() { From 1db119a08f07b8707b901e92b03138b27e887844 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 18 Jan 2013 20:22:23 -0700 Subject: [PATCH 008/696] kafka jar wasn't being included by run script --- run | 3 +++ 1 file changed, 3 insertions(+) diff --git a/run b/run index 2f61cb2a8748e..494f04c3ac48f 100755 --- a/run +++ b/run @@ -76,6 +76,9 @@ CLASSPATH+=":$CORE_DIR/src/main/resources" CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +for jar in `find "$STREAMING_DIR/lib" -name '*jar'`; do + CLASSPATH+=":$jar" +done if [ -e "$FWDIR/lib_managed" ]; then for jar in `find "$FWDIR/lib_managed/jars" -name '*jar'`; do CLASSPATH+=":$jar" From 56b7fbafa2b7717896c613e39ecc134f2405b4c6 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 18 Jan 2013 21:15:54 -0700 Subject: [PATCH 009/696] further KafkaInputDStream cleanup (removing unused and commented out code relating to offset management) --- .../streaming/dstream/KafkaInputDStream.scala | 72 +------------------ 1 file changed, 3 insertions(+), 69 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 9605072382c5f..533c91ee95591 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -19,15 +19,6 @@ import scala.collection.JavaConversions._ // Key for a specific Kafka Partition: (broker, topic, group, part) case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, partId: Int) -// NOT USED - Originally intended for fault-tolerance -// Metadata for a Kafka Stream that it sent to the Master -private[streaming] -case class KafkaInputDStreamMetadata(timestamp: Long, data: Map[KafkaPartitionKey, Long]) -// NOT USED - Originally intended for fault-tolerance -// Checkpoint data specific to a KafkaInputDstream -private[streaming] -case class KafkaDStreamCheckpointData(kafkaRdds: HashMap[Time, Any], - savedOffsets: Map[KafkaPartitionKey, Long]) extends DStreamCheckpointData(kafkaRdds) /** * Input stream that pulls messages from a Kafka Broker. @@ -52,49 +43,6 @@ class KafkaInputDStream[T: ClassManifest]( storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_ ) with Logging { - // Metadata that keeps track of which messages have already been consumed. - var savedOffsets = HashMap[Long, Map[KafkaPartitionKey, Long]]() - - /* NOT USED - Originally intended for fault-tolerance - - // In case of a failure, the offets for a particular timestamp will be restored. - @transient var restoredOffsets : Map[KafkaPartitionKey, Long] = null - - - override protected[streaming] def addMetadata(metadata: Any) { - metadata match { - case x : KafkaInputDStreamMetadata => - savedOffsets(x.timestamp) = x.data - // TOOD: Remove logging - logInfo("New saved Offsets: " + savedOffsets) - case _ => logInfo("Received unknown metadata: " + metadata.toString) - } - } - - override protected[streaming] def updateCheckpointData(currentTime: Time) { - super.updateCheckpointData(currentTime) - if(savedOffsets.size > 0) { - // Find the offets that were stored before the checkpoint was initiated - val key = savedOffsets.keys.toList.sortWith(_ < _).filter(_ < currentTime.millis).last - val latestOffsets = savedOffsets(key) - logInfo("Updating KafkaDStream checkpoint data: " + latestOffsets.toString) - checkpointData = KafkaDStreamCheckpointData(checkpointData.rdds, latestOffsets) - // TODO: This may throw out offsets that are created after the checkpoint, - // but it's unlikely we'll need them. - savedOffsets.clear() - } - } - - override protected[streaming] def restoreCheckpointData() { - super.restoreCheckpointData() - logInfo("Restoring KafkaDStream checkpoint data.") - checkpointData match { - case x : KafkaDStreamCheckpointData => - restoredOffsets = x.savedOffsets - logInfo("Restored KafkaDStream offsets: " + savedOffsets) - } - } */ - def createReceiver(): NetworkReceiver[T] = { new KafkaReceiver(host, port, groupId, topics, initialOffsets, storageLevel) .asInstanceOf[NetworkReceiver[T]] @@ -111,8 +59,6 @@ class KafkaReceiver(host: String, port: Int, groupId: String, // Handles pushing data into the BlockManager lazy protected val blockGenerator = new BlockGenerator(storageLevel) - // Keeps track of the current offsets. Maps from (broker, topic, group, part) -> Offset - lazy val offsets = HashMap[KafkaPartitionKey, Long]() // Connection to Kafka var consumerConnector : ZookeeperConsumerConnector = null @@ -143,8 +89,8 @@ class KafkaReceiver(host: String, port: Int, groupId: String, consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector] logInfo("Connected to " + zooKeeperEndPoint) - // Reset the Kafka offsets in case we are recovering from a failure - resetOffsets(initialOffsets) + // If specified, set the topic offset + setOffsets(initialOffsets) // Create Threads for each Topic/Message Stream we are listening val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder()) @@ -157,7 +103,7 @@ class KafkaReceiver(host: String, port: Int, groupId: String, } // Overwrites the offets in Zookeper. - private def resetOffsets(offsets: Map[KafkaPartitionKey, Long]) { + private def setOffsets(offsets: Map[KafkaPartitionKey, Long]) { offsets.foreach { case(key, offset) => val topicDirs = new ZKGroupTopicDirs(key.groupId, key.topic) val partitionName = key.brokerId + "-" + key.partId @@ -178,16 +124,4 @@ class KafkaReceiver(host: String, port: Int, groupId: String, } } } - - // NOT USED - Originally intended for fault-tolerance - // class KafkaDataHandler(receiver: KafkaReceiver, storageLevel: StorageLevel) - // extends BufferingBlockCreator[Any](receiver, storageLevel) { - - // override def createBlock(blockId: String, iterator: Iterator[Any]) : Block = { - // // Creates a new Block with Kafka-specific Metadata - // new Block(blockId, iterator, KafkaInputDStreamMetadata(System.currentTimeMillis, offsets.toMap)) - // } - - // } - } From d3064fe70762cbfcb7dbd5e1fbd708539c3de5e9 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 18 Jan 2013 21:34:29 -0700 Subject: [PATCH 010/696] kafkaStream API cleanup. A quorum of zookeepers can now be specified --- .../streaming/examples/KafkaWordCount.scala | 16 ++++++++-------- .../spark/streaming/StreamingContext.scala | 8 +++----- .../streaming/dstream/KafkaInputDStream.scala | 17 +++++++---------- 3 files changed, 18 insertions(+), 23 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index fe55db6e2c631..65d5da82fcccc 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -13,19 +13,19 @@ import spark.streaming.util.RawTextHelper._ object KafkaWordCount { def main(args: Array[String]) { - if (args.length < 6) { - System.err.println("Usage: KafkaWordCount ") + if (args.length < 5) { + System.err.println("Usage: KafkaWordCount ") System.exit(1) } - val Array(master, hostname, port, group, topics, numThreads) = args + val Array(master, zkQuorum, group, topics, numThreads) = args val sc = new SparkContext(master, "KafkaWordCount") val ssc = new StreamingContext(sc, Seconds(2)) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap - val lines = ssc.kafkaStream[String](hostname, port.toInt, group, topicpMap) + val lines = ssc.kafkaStream[String](zkQuorum, group, topicpMap) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) wordCounts.print() @@ -38,16 +38,16 @@ object KafkaWordCount { object KafkaWordCountProducer { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: KafkaWordCountProducer ") + if (args.length < 2) { + System.err.println("Usage: KafkaWordCountProducer ") System.exit(1) } - val Array(hostname, port, topic, messagesPerSec, wordsPerMessage) = args + val Array(zkQuorum, topic, messagesPerSec, wordsPerMessage) = args // Zookeper connection properties val props = new Properties() - props.put("zk.connect", hostname + ":" + port) + props.put("zk.connect", zkQuorum) props.put("serializer.class", "kafka.serializer.StringEncoder") val config = new ProducerConfig(props) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 14500bdcb17a0..06cf7a06eddfa 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -136,8 +136,7 @@ class StreamingContext private ( /** * Create an input stream that pulls messages form a Kafka Broker. - * @param hostname Zookeper hostname. - * @param port Zookeper port. + * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. @@ -146,14 +145,13 @@ class StreamingContext private ( * @param storageLevel RDD storage level. Defaults to memory-only. */ def kafkaStream[T: ClassManifest]( - hostname: String, - port: Int, + zkQuorum: String, groupId: String, topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](), storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 ): DStream[T] = { - val inputStream = new KafkaInputDStream[T](this, hostname, port, groupId, topics, initialOffsets, storageLevel) + val inputStream = new KafkaInputDStream[T](this, zkQuorum, groupId, topics, initialOffsets, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 533c91ee95591..4f8c8b9d104bd 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -23,8 +23,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part /** * Input stream that pulls messages from a Kafka Broker. * - * @param host Zookeper hostname. - * @param port Zookeper port. + * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. @@ -35,8 +34,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part private[streaming] class KafkaInputDStream[T: ClassManifest]( @transient ssc_ : StreamingContext, - host: String, - port: Int, + zkQuorum: String, groupId: String, topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], @@ -44,13 +42,13 @@ class KafkaInputDStream[T: ClassManifest]( ) extends NetworkInputDStream[T](ssc_ ) with Logging { def createReceiver(): NetworkReceiver[T] = { - new KafkaReceiver(host, port, groupId, topics, initialOffsets, storageLevel) + new KafkaReceiver(zkQuorum, groupId, topics, initialOffsets, storageLevel) .asInstanceOf[NetworkReceiver[T]] } } private[streaming] -class KafkaReceiver(host: String, port: Int, groupId: String, +class KafkaReceiver(zkQuorum: String, groupId: String, topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel) extends NetworkReceiver[Any] { @@ -73,21 +71,20 @@ class KafkaReceiver(host: String, port: Int, groupId: String, // In case we are using multiple Threads to handle Kafka Messages val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) - val zooKeeperEndPoint = host + ":" + port logInfo("Starting Kafka Consumer Stream with group: " + groupId) logInfo("Initial offsets: " + initialOffsets.toString) // Zookeper connection properties val props = new Properties() - props.put("zk.connect", zooKeeperEndPoint) + props.put("zk.connect", zkQuorum) props.put("zk.connectiontimeout.ms", ZK_TIMEOUT.toString) props.put("groupid", groupId) // Create the connection to the cluster - logInfo("Connecting to Zookeper: " + zooKeeperEndPoint) + logInfo("Connecting to Zookeper: " + zkQuorum) val consumerConfig = new ConsumerConfig(props) consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector] - logInfo("Connected to " + zooKeeperEndPoint) + logInfo("Connected to " + zkQuorum) // If specified, set the topic offset setOffsets(initialOffsets) From bb6ab92e31b7aad464cf8262bc3567fdeb4c14c4 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 16 Jan 2013 14:34:58 +0530 Subject: [PATCH 011/696] Changed method name of createReceiver to getReceiver as it is not intended to be a factory. --- .../twitter/TwitterInputDStream.scala | 2 +- .../spark/streaming/NetworkInputTracker.scala | 14 +++++----- .../streaming/dstream/FlumeInputDStream.scala | 4 +-- .../streaming/dstream/KafkaInputDStream.scala | 26 +++++++++---------- .../dstream/NetworkInputDStream.scala | 8 +++--- .../streaming/dstream/RawInputDStream.scala | 2 +- .../dstream/SocketInputDStream.scala | 2 +- 7 files changed, 29 insertions(+), 29 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala index 99ed4cdc1c12d..2532f27d1ac38 100644 --- a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala @@ -22,7 +22,7 @@ class TwitterInputDStream( storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - override def createReceiver(): NetworkReceiver[Status] = { + override def getReceiver(): NetworkReceiver[Status] = { new TwitterReceiver(username, password, filters, storageLevel) } } diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index e4152f3a61bb5..4ddd0f8680370 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -23,7 +23,7 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) ext */ private[streaming] class NetworkInputTracker( - @transient ssc: StreamingContext, + @transient ssc: StreamingContext, @transient networkInputStreams: Array[NetworkInputDStream[_]]) extends Logging { @@ -65,12 +65,12 @@ class NetworkInputTracker( def receive = { case RegisterReceiver(streamId, receiverActor) => { if (!networkInputStreamMap.contains(streamId)) { - throw new Exception("Register received for unexpected id " + streamId) + throw new Exception("Register received for unexpected id " + streamId) } receiverInfo += ((streamId, receiverActor)) logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) sender ! true - } + } case AddBlocks(streamId, blockIds, metadata) => { val tmp = receivedBlockIds.synchronized { if (!receivedBlockIds.contains(streamId)) { @@ -95,8 +95,8 @@ class NetworkInputTracker( /** This thread class runs all the receivers on the cluster. */ class ReceiverExecutor extends Thread { val env = ssc.env - - override def run() { + + override def run() { try { SparkEnv.set(env) startReceivers() @@ -113,7 +113,7 @@ class NetworkInputTracker( */ def startReceivers() { val receivers = networkInputStreams.map(nis => { - val rcvr = nis.createReceiver() + val rcvr = nis.getReceiver() rcvr.setStreamId(nis.id) rcvr }) @@ -141,7 +141,7 @@ class NetworkInputTracker( // Distribute the receivers and start them ssc.sc.runJob(tempRDD, startReceiver) } - + /** Stops the receivers. */ def stopReceivers() { // Signal the receivers to stop diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala index efc7058480569..c9644b3a83aa0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala @@ -25,7 +25,7 @@ class FlumeInputDStream[T: ClassManifest]( storageLevel: StorageLevel ) extends NetworkInputDStream[SparkFlumeEvent](ssc_) { - override def createReceiver(): NetworkReceiver[SparkFlumeEvent] = { + override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel) } } @@ -134,4 +134,4 @@ class FlumeReceiver( } override def getLocationPreference = Some(host) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 2b4740bdf7c7d..682cb82709368 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -31,7 +31,7 @@ case class KafkaDStreamCheckpointData(kafkaRdds: HashMap[Time, Any], /** * Input stream that pulls messages from a Kafka Broker. - * + * * @param host Zookeper hostname. * @param port Zookeper port. * @param groupId The group id for this consumer. @@ -54,13 +54,13 @@ class KafkaInputDStream[T: ClassManifest]( // Metadata that keeps track of which messages have already been consumed. var savedOffsets = HashMap[Long, Map[KafkaPartitionKey, Long]]() - + /* NOT USED - Originally intended for fault-tolerance - + // In case of a failure, the offets for a particular timestamp will be restored. @transient var restoredOffsets : Map[KafkaPartitionKey, Long] = null - + override protected[streaming] def addMetadata(metadata: Any) { metadata match { case x : KafkaInputDStreamMetadata => @@ -88,14 +88,14 @@ class KafkaInputDStream[T: ClassManifest]( override protected[streaming] def restoreCheckpointData() { super.restoreCheckpointData() logInfo("Restoring KafkaDStream checkpoint data.") - checkpointData match { - case x : KafkaDStreamCheckpointData => + checkpointData match { + case x : KafkaDStreamCheckpointData => restoredOffsets = x.savedOffsets logInfo("Restored KafkaDStream offsets: " + savedOffsets) } } */ - def createReceiver(): NetworkReceiver[T] = { + def getReceiver(): NetworkReceiver[T] = { new KafkaReceiver(host, port, groupId, topics, initialOffsets, storageLevel) .asInstanceOf[NetworkReceiver[T]] } @@ -103,7 +103,7 @@ class KafkaInputDStream[T: ClassManifest]( private[streaming] class KafkaReceiver(host: String, port: Int, groupId: String, - topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], + topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel) extends NetworkReceiver[Any] { // Timeout for establishing a connection to Zookeper in ms. @@ -130,7 +130,7 @@ class KafkaReceiver(host: String, port: Int, groupId: String, val zooKeeperEndPoint = host + ":" + port logInfo("Starting Kafka Consumer Stream with group: " + groupId) logInfo("Initial offsets: " + initialOffsets.toString) - + // Zookeper connection properties val props = new Properties() props.put("zk.connect", zooKeeperEndPoint) @@ -161,7 +161,7 @@ class KafkaReceiver(host: String, port: Int, groupId: String, offsets.foreach { case(key, offset) => val topicDirs = new ZKGroupTopicDirs(key.groupId, key.topic) val partitionName = key.brokerId + "-" + key.partId - updatePersistentPath(consumerConnector.zkClient, + updatePersistentPath(consumerConnector.zkClient, topicDirs.consumerOffsetDir + "/" + partitionName, offset.toString) } } @@ -174,7 +174,7 @@ class KafkaReceiver(host: String, port: Int, groupId: String, blockGenerator += msgAndMetadata.message // Updating the offet. The key is (broker, topic, group, partition). - val key = KafkaPartitionKey(msgAndMetadata.topicInfo.brokerId, msgAndMetadata.topic, + val key = KafkaPartitionKey(msgAndMetadata.topicInfo.brokerId, msgAndMetadata.topic, groupId, msgAndMetadata.topicInfo.partition.partId) val offset = msgAndMetadata.topicInfo.getConsumeOffset offsets.put(key, offset) @@ -182,12 +182,12 @@ class KafkaReceiver(host: String, port: Int, groupId: String, // Keep on handling messages true - } + } } } // NOT USED - Originally intended for fault-tolerance - // class KafkaDataHandler(receiver: KafkaReceiver, storageLevel: StorageLevel) + // class KafkaDataHandler(receiver: KafkaReceiver, storageLevel: StorageLevel) // extends BufferingBlockCreator[Any](receiver, storageLevel) { // override def createBlock(blockId: String, iterator: Iterator[Any]) : Block = { diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index aa6be95f30f7a..9142deb9ed35e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -20,7 +20,7 @@ import java.util.concurrent.ArrayBlockingQueue /** * Abstract class for defining any InputDStream that has to start a receiver on worker * nodes to receive external data. Specific implementations of NetworkInputDStream must - * define the createReceiver() function that creates the receiver object of type + * define the getReceiver() function that gets the receiver object of type * [[spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive * data. * @param ssc_ Streaming context that will execute this input stream @@ -34,11 +34,11 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming val id = ssc.getNewNetworkStreamId() /** - * Creates the receiver object that will be sent to the worker nodes + * Gets the receiver object that will be sent to the worker nodes * to receive data. This method needs to defined by any specific implementation * of a NetworkInputDStream. */ - def createReceiver(): NetworkReceiver[T] + def getReceiver(): NetworkReceiver[T] // Nothing to start or stop as both taken care of by the NetworkInputTracker. def start() {} @@ -46,7 +46,7 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming def stop() {} override def compute(validTime: Time): Option[RDD[T]] = { - val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime) + val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime) Some(new BlockRDD[T](ssc.sc, blockIds)) } } diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 290fab1ce0f45..74ffa1c2a2000 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -25,7 +25,7 @@ class RawInputDStream[T: ClassManifest]( storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_ ) with Logging { - def createReceiver(): NetworkReceiver[T] = { + def getReceiver(): NetworkReceiver[T] = { new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]] } } diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index d42027092b620..4af839ad7f03d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -15,7 +15,7 @@ class SocketInputDStream[T: ClassManifest]( storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_) { - def createReceiver(): NetworkReceiver[T] = { + def getReceiver(): NetworkReceiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) } } From 56b9bd197c522f33e354c2e9ad7e76440cf817e9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 19 Jan 2013 21:20:49 +0530 Subject: [PATCH 012/696] Plug in actor as stream receiver API --- docs/plugin-custom-receiver.md | 101 ++++++++++++++++ .../examples/AkkaActorWordCount.scala | 80 +++++++++++++ .../examples/FileTextStreamFeeder.scala | 63 ++++++++++ .../spark/streaming/StreamingContext.scala | 29 +++++ .../dstream/PluggableInputDStream.scala | 12 ++ .../streaming/receivers/ActorReceiver.scala | 111 ++++++++++++++++++ 6 files changed, 396 insertions(+) create mode 100644 docs/plugin-custom-receiver.md create mode 100644 examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala create mode 100644 examples/src/main/scala/spark/streaming/examples/FileTextStreamFeeder.scala create mode 100644 streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala create mode 100644 streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala diff --git a/docs/plugin-custom-receiver.md b/docs/plugin-custom-receiver.md new file mode 100644 index 0000000000000..41e6a17e2c31e --- /dev/null +++ b/docs/plugin-custom-receiver.md @@ -0,0 +1,101 @@ +--- +layout: global +title: Tutorial - Spark streaming, Plugging in a custom receiver. +--- + +A "Spark streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. + +This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. + + +## A quick and naive walk-through + +### Write a simple receiver + +This starts with implementing [Actor](#References) + +Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api. + +{% highlight scala %} + + class SocketTextStreamReceiver (host:String, + port:Int, + bytesToString: ByteString => String) extends Actor { + + override def preStart = IOManager(context.system).connect(host, port) + + def receive = { + case IO.Read(socket, bytes) => context.parent ! Data(bytesToString(bytes)) + } + } + + +{% endhighlight %} + + +_Please see implementations of NetworkReceiver for more generic NetworkReceivers._ + +### A sample spark application + +* First create a Spark streaming context with master url and batchduration. + +{% highlight scala %} + + val ssc = new StreamingContext(master, "WordCountCustomStreamSource", + Seconds(batchDuration)) + +{% endhighlight %} + +* Plug-in the actor configuration into the spark streaming context and create a DStream. + +{% highlight scala %} + + val lines = ssc.pluggableActorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8445, z => z.utf8String)),"SocketReceiver") + +{% endhighlight %} + +* Process it. + +{% highlight scala %} + + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + + wordCounts.print() + ssc.start() + + +{% endhighlight %} + +* After processing it, stream can be tested using the netcat utility. + + $ nc -l localhost 8445 + hello world + hello hello + + +## Multiple homogeneous/heterogeneous receivers. + +A DStream union operation is provided for taking union on multiple input streams. + +{% highlight scala %} + + val lines = ssc.pluggableActorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8445, z => z.utf8String)),"SocketReceiver") + + // Another socket stream receiver + val lines2 = ssc.pluggableActorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8446, z => z.utf8String)),"SocketReceiver") + + val union = lines.union(lines2) + +{% endhighlight %} + +Above stream can be easily process as described earlier. + +_A more comprehensive example is provided in the spark streaming examples_ + +## References + +1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) diff --git a/examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala new file mode 100644 index 0000000000000..ff05842c71f0d --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala @@ -0,0 +1,80 @@ +package spark.streaming.examples + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.actor.actorRef2Scala + +import spark.streaming.Seconds +import spark.streaming.StreamingContext +import spark.streaming.StreamingContext.toPairDStreamFunctions +import spark.streaming.receivers.Data + +case class SubscribeReceiver(receiverActor: ActorRef) +case class UnsubscribeReceiver(receiverActor: ActorRef) + +/** + * A sample actor as receiver is also simplest. This receiver actor + * goes and subscribe to a typical publisher/feeder actor and receives + * data, thus it is important to have feeder running before this example + * can be run. Please see FileTextStreamFeeder(sample) for feeder of this + * receiver. + */ +class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) + extends Actor { + + lazy private val remotePublisher = context.actorFor(urlOfPublisher) + + override def preStart = remotePublisher ! SubscribeReceiver(context.self) + + def receive = { + case msg => context.parent ! Data(msg.asInstanceOf[T]) + } + + override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self) + +} + +/** + * A sample word count program demonstrating the use of plugging in + * AkkaActor as Receiver + */ +object AkkaActorWordCount { + def main(args: Array[String]) { + if (args.length < 4) { + System.err.println( + "Usage: AkkaActorWordCount " + + " " + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + val Seq(master, batchDuration, remoteAkkaHost, remoteAkkaPort) = args.toSeq + + // Create the context and set the batch size + val ssc = new StreamingContext(master, "AkkaActorWordCount", + Seconds(batchDuration.toLong)) + + /* + * Following is the use of pluggableActorStream to plug in custom actor as receiver + * + * An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e type of data received and PluggableInputDstream + * should be same. + * + * For example: Both pluggableActorStream and SampleActorReceiver are parameterized + * to same type to ensure type safety. + */ + + val lines = ssc.pluggableActorStream[String]( + Props(new SampleActorReceiver[String]("akka://spark@%s:%s/user/FeederActor".format( + remoteAkkaHost, remoteAkkaPort.toInt))), "SampleReceiver") + + //compute wordcount + lines.flatMap(_.split(" ")).map(x => (x, 1)).reduceByKey(_ + _).print() + + ssc.start() + + } +} diff --git a/examples/src/main/scala/spark/streaming/examples/FileTextStreamFeeder.scala b/examples/src/main/scala/spark/streaming/examples/FileTextStreamFeeder.scala new file mode 100644 index 0000000000000..f4c1b87f0e6a3 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/FileTextStreamFeeder.scala @@ -0,0 +1,63 @@ +package spark.streaming.examples + +import java.util.concurrent.CountDownLatch + +import scala.collection.mutable.LinkedList +import scala.io.Source + +import akka.actor.{ Actor, ActorRef, actorRef2Scala } +import akka.actor.Props + +import spark.util.AkkaUtils + +/** + * A feeder to which multiple message receiver (specified by "noOfReceivers")actors + * subscribe and receive file(s)'s text as stream of messages. This is provided + * as a demonstration application for trying out Actor as receiver feature. Please see + * SampleActorReceiver or AkkaActorWordCount example for details about the + * receiver of this feeder. + */ + +object FileTextStreamFeeder { + + var receivers: LinkedList[ActorRef] = new LinkedList[ActorRef]() + var countdownLatch: CountDownLatch = _ + def main(args: Array[String]) = args.toList match { + + case host :: port :: noOfReceivers :: fileNames => + val acs = AkkaUtils.createActorSystem("spark", host, port.toInt)._1 + countdownLatch = new CountDownLatch(noOfReceivers.toInt) + val actor = acs.actorOf(Props(new FeederActor), "FeederActor") + countdownLatch.await() //wait for all the receivers to subscribe + for (fileName <- fileNames;line <- Source.fromFile(fileName).getLines) { + actor ! line + } + acs.awaitTermination(); + + case _ => + System.err.println("Usage: FileTextStreamFeeder ") + System.exit(1) + } + + /** + * Sends the content to every receiver subscribed + */ + class FeederActor extends Actor { + + def receive: Receive = { + + case SubscribeReceiver(receiverActor: ActorRef) => + println("received subscribe from %s".format(receiverActor.toString)) + receivers = LinkedList(receiverActor) ++ receivers + countdownLatch.countDown() + + case UnsubscribeReceiver(receiverActor: ActorRef) => + println("received unsubscribe from %s".format(receiverActor.toString)) + receivers = receivers.dropWhile(x => x eq receiverActor) + + case textMessage: String => + receivers.foreach(_ ! textMessage) + + } + } +} \ No newline at end of file diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 14500bdcb17a0..cd7379da14c35 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -1,10 +1,15 @@ package spark.streaming +import akka.actor.Props + import spark.streaming.dstream._ import spark.{RDD, Logging, SparkEnv, SparkContext} import spark.storage.StorageLevel import spark.util.MetadataCleaner +import spark.streaming.receivers.ActorReceiver +import spark.streaming.receivers.Settings + import scala.collection.mutable.Queue @@ -134,6 +139,30 @@ class StreamingContext private ( protected[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement() + /** + * Create an input stream with any arbitrary user implemented network receiver. + * @param receiver Custom implementation of NetworkReceiver + */ + def pluggableNetworkStream[T: ClassManifest]( + receiver: NetworkReceiver[T]): DStream[T] = { + val inputStream = new PluggableInputDStream[T](this, + receiver) + graph.addInputStream(inputStream) + inputStream + } + + /** + * Create an input stream with any arbitrary user implemented akka actor receiver. + * @param props Props object defining creation of the actor + * @param name Name of the actor + * @param storageLevel RDD storage level. Defaults to memory-only. + */ + def pluggableActorStream[T: ClassManifest]( + props: Props, name: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2): DStream[T] = { + pluggableNetworkStream(new ActorReceiver(Settings(props, name, storageLevel))) + } + /** * Create an input stream that pulls messages form a Kafka Broker. * @param hostname Zookeper hostname. diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala new file mode 100644 index 0000000000000..674f1059fe0b7 --- /dev/null +++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala @@ -0,0 +1,12 @@ +package spark.streaming.dstream + +import spark.streaming.StreamingContext + +class PluggableInputDStream[T: ClassManifest]( + @transient ssc_ : StreamingContext, + receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { + + def getReceiver(): NetworkReceiver[T] = { + receiver + } +} diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala new file mode 100644 index 0000000000000..f24c99ad705d6 --- /dev/null +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -0,0 +1,111 @@ +package spark.streaming.receivers + +import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } +import akka.actor.{ actorRef2Scala, ActorRef } +import akka.actor.{ PossiblyHarmful, OneForOneStrategy } + +import spark.storage.StorageLevel +import spark.streaming.dstream.NetworkReceiver + +import java.util.concurrent.atomic.AtomicInteger + +/** A helper with set of defaults for supervisor strategy **/ +object ReceiverSupervisorStrategy { + + import akka.util.duration._ + import akka.actor.SupervisorStrategy._ + + val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = + 15 millis) { + case _: RuntimeException ⇒ Restart + case _: Exception ⇒ Escalate + } +} + +/** + * Settings for configuring the actor creation or defining supervisor strategy + */ +case class Settings(props: Props, + name: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy) + +/** + * Statistcs for querying the supervisor about state of workers + */ +case class Statistcs(numberOfMsgs: Int, + numberOfWorkers: Int, + numberOfHiccups: Int, + otherInfo: String) + +/** Case class to receive data sent by child actors **/ +case class Data[T: ClassManifest](data: T) + +/** + * Provides Actors as receivers for receiving stream. + * + * As Actors can also be used to receive data from almost any stream source. + * A nice set of abstraction(s) for actors as receivers is already provided for + * a few general cases. It is thus exposed as an API where user may come with + * his own Actor to run as receiver for Spark Streaming input source. + */ +class ActorReceiver[T: ClassManifest](settings: Settings) + extends NetworkReceiver[T] { + + protected lazy val blocksGenerator: BlockGenerator = + new BlockGenerator(settings.storageLevel) + + protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor), + "Supervisor" + streamId) + + private class Supervisor extends Actor { + + override val supervisorStrategy = settings.supervisorStrategy + val worker = context.actorOf(settings.props, settings.name) + logInfo("Started receiver worker at:" + worker.path) + + val n: AtomicInteger = new AtomicInteger(0) + val hiccups: AtomicInteger = new AtomicInteger(0) + + def receive = { + + case props: Props => + val worker = context.actorOf(props) + logInfo("Started receiver worker at:" + worker.path) + sender ! worker + + case (props: Props, name: String) => + val worker = context.actorOf(props, name) + logInfo("Started receiver worker at:" + worker.path) + sender ! worker + + case _: PossiblyHarmful => hiccups.incrementAndGet() + + case _: Statistcs => + val workers = context.children + sender ! Statistcs(n.get, workers.size, hiccups.get, workers.mkString("\n")) + + case Data(iter: Iterator[_]) => push(iter.asInstanceOf[Iterator[T]]) + + case Data(msg) => + blocksGenerator += msg.asInstanceOf[T] + n.incrementAndGet + } + } + + protected def push(iter: Iterator[T]) { + pushBlock("block-" + streamId + "-" + System.nanoTime(), + iter, null, settings.storageLevel) + } + + protected def onStart() = { + blocksGenerator.start() + supervisor + logInfo("Supervision tree for receivers initialized at:" + supervisor.path) + } + + protected def onStop() = { + supervisor ! PoisonPill + } + +} From ea739251eb763b756a282534268e765b8d4b70f0 Mon Sep 17 00:00:00 2001 From: seanm Date: Sun, 20 Jan 2013 11:29:21 -0700 Subject: [PATCH 013/696] adding updateStateByKey object lifecycle test --- .../streaming/BasicOperationsSuite.scala | 45 +++++++++++++++++++ .../scala/spark/streaming/TestSuiteBase.scala | 5 +++ 2 files changed, 50 insertions(+) diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index f73f9b1823213..2bc94463b1331 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -160,6 +160,51 @@ class BasicOperationsSuite extends TestSuiteBase { testOperation(inputData, updateStateOperation, outputData, true) } + test("updateStateByKey - object lifecycle") { + val inputData = + Seq( + Seq("a","b"), + null, + Seq("a","c","a"), + Seq("c"), + null, + null + ) + + val outputData = + Seq( + Seq(("a", 1), ("b", 1)), + Seq(("a", 1), ("b", 1)), + Seq(("a", 3), ("c", 1)), + Seq(("a", 3), ("c", 2)), + Seq(("c", 2)), + Seq() + ) + + val updateStateOperation = (s: DStream[String]) => { + class StateObject(var counter: Int = 0, var expireCounter: Int = 0) extends Serializable + + // updateFunc clears a state when a StateObject is seen without new values twice in a row + val updateFunc = (values: Seq[Int], state: Option[StateObject]) => { + val stateObj = state.getOrElse(new StateObject) + values.foldLeft(0)(_ + _) match { + case 0 => stateObj.expireCounter += 1 // no new values + case n => { // has new values, increment and reset expireCounter + stateObj.counter += n + stateObj.expireCounter = 0 + } + } + stateObj.expireCounter match { + case 2 => None // seen twice with no new values, give it the boot + case _ => Option(stateObj) + } + } + s.map(_ -> 1).updateStateByKey[StateObject](updateFunc).mapValues(_.counter) + } + + testOperation(inputData, updateStateOperation, outputData, true) + } + test("forgetting of RDDs - map and window operations") { assert(batchDuration === Seconds(1), "Batch duration has changed from 1 second") diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index a76f61d4adb06..11cfcba827a78 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -28,6 +28,11 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[ logInfo("Computing RDD for time " + validTime) val index = ((validTime - zeroTime) / slideDuration - 1).toInt val selectedInput = if (index < input.size) input(index) else Seq[T]() + + // lets us test cases where RDDs are not created + if (selectedInput == null) + return None + val rdd = ssc.sc.makeRDD(selectedInput, numPartitions) logInfo("Created RDD " + rdd.id + " with " + selectedInput) Some(rdd) From c0694291c81ad775918421941a80a00ca9593a38 Mon Sep 17 00:00:00 2001 From: seanm Date: Sun, 20 Jan 2013 12:09:45 -0700 Subject: [PATCH 014/696] Splitting StreamingContext.queueStream into two methods --- .../spark/streaming/StreamingContext.scala | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 14500bdcb17a0..3cec35cb377ff 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -283,17 +283,31 @@ class StreamingContext private ( } /** - * Creates a input stream from an queue of RDDs. In each batch, + * Creates an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * @param queue Queue of RDDs * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval - * @param defaultRDD Default RDD is returned by the DStream when the queue is empty * @tparam T Type of objects in the RDD */ def queueStream[T: ClassManifest]( queue: Queue[RDD[T]], - oneAtATime: Boolean = true, - defaultRDD: RDD[T] = null + oneAtATime: Boolean = true + ): DStream[T] = { + queueStream(queue, oneAtATime, sc.makeRDD(Seq[T](), 1)) + } + + /** + * Creates an input stream from a queue of RDDs. In each batch, + * it will process either one or all of the RDDs returned by the queue. + * @param queue Queue of RDDs + * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval + * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. Set as null if no RDD should be returned when empty + * @tparam T Type of objects in the RDD + */ + def queueStream[T: ClassManifest]( + queue: Queue[RDD[T]], + oneAtATime: Boolean, + defaultRDD: RDD[T] ): DStream[T] = { val inputStream = new QueueInputDStream(this, queue, oneAtATime, defaultRDD) registerInputStream(inputStream) From 43bfd7bb21e6f8a9d083686a83bcd309a84f937e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 21 Jan 2013 11:39:30 +0530 Subject: [PATCH 015/696] Changed method name of createReceiver to getReceiver as it is not intended to be a factory. --- .../streaming/examples/twitter/TwitterInputDStream.scala | 2 +- .../main/scala/spark/streaming/NetworkInputTracker.scala | 2 +- .../scala/spark/streaming/dstream/FlumeInputDStream.scala | 2 +- .../scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- .../scala/spark/streaming/dstream/NetworkInputDStream.scala | 6 +++--- .../scala/spark/streaming/dstream/RawInputDStream.scala | 2 +- .../scala/spark/streaming/dstream/SocketInputDStream.scala | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala index 99ed4cdc1c12d..2532f27d1ac38 100644 --- a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala @@ -22,7 +22,7 @@ class TwitterInputDStream( storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - override def createReceiver(): NetworkReceiver[Status] = { + override def getReceiver(): NetworkReceiver[Status] = { new TwitterReceiver(username, password, filters, storageLevel) } } diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index e4152f3a61bb5..665842a683b12 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -113,7 +113,7 @@ class NetworkInputTracker( */ def startReceivers() { val receivers = networkInputStreams.map(nis => { - val rcvr = nis.createReceiver() + val rcvr = nis.getReceiver() rcvr.setStreamId(nis.id) rcvr }) diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala index efc7058480569..83e9e59cadfdd 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala @@ -25,7 +25,7 @@ class FlumeInputDStream[T: ClassManifest]( storageLevel: StorageLevel ) extends NetworkInputDStream[SparkFlumeEvent](ssc_) { - override def createReceiver(): NetworkReceiver[SparkFlumeEvent] = { + override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel) } } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 2b4740bdf7c7d..0eca4000336df 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -95,7 +95,7 @@ class KafkaInputDStream[T: ClassManifest]( } } */ - def createReceiver(): NetworkReceiver[T] = { + def getReceiver(): NetworkReceiver[T] = { new KafkaReceiver(host, port, groupId, topics, initialOffsets, storageLevel) .asInstanceOf[NetworkReceiver[T]] } diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index aa6be95f30f7a..e74c2aa44839e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -20,7 +20,7 @@ import java.util.concurrent.ArrayBlockingQueue /** * Abstract class for defining any InputDStream that has to start a receiver on worker * nodes to receive external data. Specific implementations of NetworkInputDStream must - * define the createReceiver() function that creates the receiver object of type + * define the getReceiver() function that gets the receiver object of type * [[spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive * data. * @param ssc_ Streaming context that will execute this input stream @@ -34,11 +34,11 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming val id = ssc.getNewNetworkStreamId() /** - * Creates the receiver object that will be sent to the worker nodes + * Gets the receiver object that will be sent to the worker nodes * to receive data. This method needs to defined by any specific implementation * of a NetworkInputDStream. */ - def createReceiver(): NetworkReceiver[T] + def getReceiver(): NetworkReceiver[T] // Nothing to start or stop as both taken care of by the NetworkInputTracker. def start() {} diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 290fab1ce0f45..74ffa1c2a2000 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -25,7 +25,7 @@ class RawInputDStream[T: ClassManifest]( storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_ ) with Logging { - def createReceiver(): NetworkReceiver[T] = { + def getReceiver(): NetworkReceiver[T] = { new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]] } } diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index d42027092b620..4af839ad7f03d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -15,7 +15,7 @@ class SocketInputDStream[T: ClassManifest]( storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_) { - def createReceiver(): NetworkReceiver[T] = { + def getReceiver(): NetworkReceiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) } } From ffd1623595cdce4080ad1e4e676e65898ebdd6dd Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Mon, 21 Jan 2013 15:55:46 -0600 Subject: [PATCH 016/696] Minor cleanup. --- core/src/main/scala/spark/Accumulators.scala | 3 +-- core/src/main/scala/spark/Logging.scala | 3 +-- .../main/scala/spark/ParallelCollection.scala | 15 +++++---------- core/src/main/scala/spark/TaskContext.scala | 3 +-- core/src/main/scala/spark/rdd/BlockRDD.scala | 6 ++---- core/src/main/scala/spark/rdd/CartesianRDD.scala | 3 +-- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 6 ++---- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 6 ++---- core/src/main/scala/spark/rdd/SampledRDD.scala | 5 ++--- core/src/main/scala/spark/rdd/ShuffledRDD.scala | 3 +-- core/src/main/scala/spark/rdd/UnionRDD.scala | 3 +-- core/src/main/scala/spark/rdd/ZippedRDD.scala | 3 +-- .../spark/scheduler/local/LocalScheduler.scala | 4 ++-- .../mesos/CoarseMesosSchedulerBackend.scala | 16 ++++++---------- .../scheduler/mesos/MesosSchedulerBackend.scala | 10 +++------- core/src/test/scala/spark/FileServerSuite.scala | 4 ++-- 16 files changed, 33 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala index b644aba5f869d..57c6df35bed9f 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/spark/Accumulators.scala @@ -25,8 +25,7 @@ class Accumulable[R, T] ( extends Serializable { val id = Accumulators.newId - @transient - private var value_ = initialValue // Current value on master + @transient private var value_ = initialValue // Current value on master val zero = param.zero(initialValue) // Zero value to be passed to workers var deserialized = false diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala index 90bae26202bd9..7c1c1bb1440bc 100644 --- a/core/src/main/scala/spark/Logging.scala +++ b/core/src/main/scala/spark/Logging.scala @@ -11,8 +11,7 @@ import org.slf4j.LoggerFactory trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine - @transient - private var log_ : Logger = null + @transient private var log_ : Logger = null // Method to get or create the logger for this object protected def log: Logger = { diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/ParallelCollection.scala index ede933c9e9a20..ad23e5bec8681 100644 --- a/core/src/main/scala/spark/ParallelCollection.scala +++ b/core/src/main/scala/spark/ParallelCollection.scala @@ -23,32 +23,28 @@ private[spark] class ParallelCollectionSplit[T: ClassManifest]( } private[spark] class ParallelCollection[T: ClassManifest]( - @transient sc : SparkContext, + @transient sc: SparkContext, @transient data: Seq[T], numSlices: Int, - locationPrefs : Map[Int,Seq[String]]) + locationPrefs: Map[Int,Seq[String]]) extends RDD[T](sc, Nil) { // TODO: Right now, each split sends along its full data, even if later down the RDD chain it gets // cached. It might be worthwhile to write the data to a file in the DFS and read it in the split // instead. // UPDATE: A parallel collection can be checkpointed to HDFS, which achieves this goal. - @transient - var splits_ : Array[Split] = { + @transient var splits_ : Array[Split] = { val slices = ParallelCollection.slice(data, numSlices).toArray slices.indices.map(i => new ParallelCollectionSplit(id, i, slices(i))).toArray } - override def getSplits = splits_.asInstanceOf[Array[Split]] + override def getSplits = splits_ override def compute(s: Split, context: TaskContext) = s.asInstanceOf[ParallelCollectionSplit[T]].iterator override def getPreferredLocations(s: Split): Seq[String] = { - locationPrefs.get(s.index) match { - case Some(s) => s - case _ => Nil - } + locationPrefs.get(s.index) getOrElse Nil } override def clearDependencies() { @@ -56,7 +52,6 @@ private[spark] class ParallelCollection[T: ClassManifest]( } } - private object ParallelCollection { /** * Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index d2746b26b3ec1..eab85f85a262b 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -5,8 +5,7 @@ import scala.collection.mutable.ArrayBuffer class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long) extends Serializable { - @transient - val onCompleteCallbacks = new ArrayBuffer[() => Unit] + @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit] // Add a callback function to be executed on task completion. An example use // is for HadoopRDD to register a callback to close the input stream. diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index b1095a52b4b92..2c022f88e0def 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -11,13 +11,11 @@ private[spark] class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { - @transient - var splits_ : Array[Split] = (0 until blockIds.size).map(i => { + @transient var splits_ : Array[Split] = (0 until blockIds.size).map(i => { new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split] }).toArray - @transient - lazy val locations_ = { + @transient lazy val locations_ = { val blockManager = SparkEnv.get.blockManager /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ val locations = blockManager.getLocations(blockIds) diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 79e7c24e7c749..453d410ad446b 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -35,8 +35,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( val numSplitsInRdd2 = rdd2.splits.size - @transient - var splits_ = { + @transient var splits_ = { // create the cross product split val array = new Array[Split](rdd1.splits.size * rdd2.splits.size) for (s1 <- rdd1.splits; s2 <- rdd2.splits) { diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 1d528be2aa5ab..8fafd27bb6985 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -45,8 +45,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) val aggr = new CoGroupAggregator - @transient - var deps_ = { + @transient var deps_ = { val deps = new ArrayBuffer[Dependency[_]] for ((rdd, index) <- rdds.zipWithIndex) { if (rdd.partitioner == Some(part)) { @@ -63,8 +62,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) override def getDependencies = deps_ - @transient - var splits_ : Array[Split] = { + @transient var splits_ : Array[Split] = { val array = new Array[Split](part.numPartitions) for (i <- 0 until array.size) { array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (r, j) => diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index bb22db073c50f..c3b155fcbddd6 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -37,11 +37,9 @@ class NewHadoopRDD[K, V]( formatter.format(new Date()) } - @transient - private val jobId = new JobID(jobtrackerId, id) + @transient private val jobId = new JobID(jobtrackerId, id) - @transient - private val splits_ : Array[Split] = { + @transient private val splits_ : Array[Split] = { val inputFormat = inputFormatClass.newInstance val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index 1bc9c96112dfa..e24ad23b21423 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -19,13 +19,12 @@ class SampledRDD[T: ClassManifest]( seed: Int) extends RDD[T](prev) { - @transient - var splits_ : Array[Split] = { + @transient var splits_ : Array[Split] = { val rg = new Random(seed) firstParent[T].splits.map(x => new SampledRDDSplit(x, rg.nextInt)) } - override def getSplits = splits_.asInstanceOf[Array[Split]] + override def getSplits = splits_ override def getPreferredLocations(split: Split) = firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev) diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 1b219473e0dfc..28ff19876dacb 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -22,8 +22,7 @@ class ShuffledRDD[K, V]( override val partitioner = Some(part) - @transient - var splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) + @transient var splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) override def getSplits = splits_ diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 24a085df02aae..82f0a44ecd900 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -28,8 +28,7 @@ class UnionRDD[T: ClassManifest]( @transient var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs - @transient - var splits_ : Array[Split] = { + @transient var splits_ : Array[Split] = { val array = new Array[Split](rdds.map(_.splits.size).sum) var pos = 0 for (rdd <- rdds; split <- rdd.splits) { diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 16e6cc0f1ba93..d950b06c85042 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -34,8 +34,7 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( // TODO: FIX THIS. - @transient - var splits_ : Array[Split] = { + @transient var splits_ : Array[Split] = { if (rdd1.splits.size != rdd2.splits.size) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index dff550036d50b..21d255debdf5a 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -19,8 +19,8 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon extends TaskScheduler with Logging { - var attemptId = new AtomicInteger(0) - var threadPool = Executors.newFixedThreadPool(threads, DaemonThreadFactory) + val attemptId = new AtomicInteger(0) + val threadPool = Executors.newFixedThreadPool(threads, DaemonThreadFactory) val env = SparkEnv.get var listener: TaskSchedulerListener = null diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index c45c7df69c63d..014906b02850d 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -64,13 +64,9 @@ private[spark] class CoarseMesosSchedulerBackend( val taskIdToSlaveId = new HashMap[Int, String] val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed - val sparkHome = sc.getSparkHome() match { - case Some(path) => - path - case None => - throw new SparkException("Spark home is not set; set it through the spark.home system " + - "property, the SPARK_HOME environment variable or the SparkContext constructor") - } + val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException( + "Spark home is not set; set it through the spark.home system " + + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt @@ -184,7 +180,7 @@ private[spark] class CoarseMesosSchedulerBackend( } /** Helper function to pull out a resource from a Mesos Resources protobuf */ - def getResource(res: JList[Resource], name: String): Double = { + private def getResource(res: JList[Resource], name: String): Double = { for (r <- res if r.getName == name) { return r.getScalar.getValue } @@ -193,7 +189,7 @@ private[spark] class CoarseMesosSchedulerBackend( } /** Build a Mesos resource protobuf object */ - def createResource(resourceName: String, quantity: Double): Protos.Resource = { + private def createResource(resourceName: String, quantity: Double): Protos.Resource = { Resource.newBuilder() .setName(resourceName) .setType(Value.Type.SCALAR) @@ -202,7 +198,7 @@ private[spark] class CoarseMesosSchedulerBackend( } /** Check whether a Mesos task state represents a finished task */ - def isFinished(state: MesosTaskState) = { + private def isFinished(state: MesosTaskState) = { state == MesosTaskState.TASK_FINISHED || state == MesosTaskState.TASK_FAILED || state == MesosTaskState.TASK_KILLED || diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 8c7a1dfbc0570..2989e31f5e9d8 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -76,13 +76,9 @@ private[spark] class MesosSchedulerBackend( } def createExecutorInfo(): ExecutorInfo = { - val sparkHome = sc.getSparkHome() match { - case Some(path) => - path - case None => - throw new SparkException("Spark home is not set; set it through the spark.home system " + - "property, the SPARK_HOME environment variable or the SparkContext constructor") - } + val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException( + "Spark home is not set; set it through the spark.home system " + + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val execScript = new File(sparkHome, "spark-executor").getCanonicalPath val environment = Environment.newBuilder() sc.executorEnvs.foreach { case (key, value) => diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index b4283d960488b..fe964bd893f40 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -9,8 +9,8 @@ import SparkContext._ class FileServerSuite extends FunSuite with BeforeAndAfter { @transient var sc: SparkContext = _ - @transient var tmpFile : File = _ - @transient var testJarFile : File = _ + @transient var tmpFile: File = _ + @transient var testJarFile: File = _ before { // Create a sample text file From ef711902c1f42db14c8ddd524195f0a9efb56e65 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 21 Jan 2013 16:42:24 -0800 Subject: [PATCH 017/696] Don't download files to master's working directory. This should avoid exceptions caused by existing files with different contents. I also removed some unused code. --- .../src/main/scala/spark/HttpFileServer.scala | 8 ++-- core/src/main/scala/spark/SparkContext.scala | 7 ++-- core/src/main/scala/spark/SparkEnv.scala | 20 ++++++---- core/src/main/scala/spark/SparkFiles.java | 25 ++++++++++++ core/src/main/scala/spark/Utils.scala | 16 +------- .../spark/api/java/JavaSparkContext.scala | 5 ++- .../scala/spark/api/python/PythonRDD.scala | 2 + .../spark/deploy/worker/ExecutorRunner.scala | 5 --- .../main/scala/spark/executor/Executor.scala | 6 +-- .../scheduler/local/LocalScheduler.scala | 6 +-- .../test/scala/spark/FileServerSuite.scala | 9 +++-- python/pyspark/__init__.py | 5 ++- python/pyspark/context.py | 40 +++++++++++++++++-- python/pyspark/files.py | 24 +++++++++++ python/pyspark/worker.py | 3 ++ python/run-tests | 3 ++ 16 files changed, 133 insertions(+), 51 deletions(-) create mode 100644 core/src/main/scala/spark/SparkFiles.java create mode 100644 python/pyspark/files.py diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala index 659d17718fae4..00901d95e26c3 100644 --- a/core/src/main/scala/spark/HttpFileServer.scala +++ b/core/src/main/scala/spark/HttpFileServer.scala @@ -1,9 +1,7 @@ package spark -import java.io.{File, PrintWriter} -import java.net.URL -import scala.collection.mutable.HashMap -import org.apache.hadoop.fs.FileUtil +import java.io.{File} +import com.google.common.io.Files private[spark] class HttpFileServer extends Logging { @@ -40,7 +38,7 @@ private[spark] class HttpFileServer extends Logging { } def addFileToDir(file: File, dir: File) : String = { - Utils.copyFile(file, new File(dir, file.getName)) + Files.copy(file, new File(dir, file.getName)) return dir + "/" + file.getName } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 8b6f4b3b7d530..2eeca66ed6e70 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -439,9 +439,10 @@ class SparkContext( def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) /** - * Add a file to be downloaded into the working directory of this Spark job on every node. + * Add a file to be downloaded with this Spark job on every node. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported - * filesystems), or an HTTP, HTTPS or FTP URI. + * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, + * use `SparkFiles.get(path)` to find its download location. */ def addFile(path: String) { val uri = new URI(path) @@ -454,7 +455,7 @@ class SparkContext( // Fetch the file locally in case a job is executed locally. // Jobs that run through LocalScheduler will already fetch the required dependencies, // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here. - Utils.fetchFile(path, new File(".")) + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory)) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) } diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 41441720a7c8f..6b44e29f4c053 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -28,14 +28,10 @@ class SparkEnv ( val broadcastManager: BroadcastManager, val blockManager: BlockManager, val connectionManager: ConnectionManager, - val httpFileServer: HttpFileServer + val httpFileServer: HttpFileServer, + val sparkFilesDir: String ) { - /** No-parameter constructor for unit tests. */ - def this() = { - this(null, new JavaSerializer, new JavaSerializer, null, null, null, null, null, null, null) - } - def stop() { httpFileServer.stop() mapOutputTracker.stop() @@ -112,6 +108,15 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + // Set the sparkFiles directory, used when downloading dependencies. In local mode, + // this is a temporary directory; in distributed mode, this is the executor's current working + // directory. + val sparkFilesDir: String = if (isMaster) { + Utils.createTempDir().getAbsolutePath + } else { + "." + } + // Warn about deprecated spark.cache.class property if (System.getProperty("spark.cache.class") != null) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + @@ -128,6 +133,7 @@ object SparkEnv extends Logging { broadcastManager, blockManager, connectionManager, - httpFileServer) + httpFileServer, + sparkFilesDir) } } diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/spark/SparkFiles.java new file mode 100644 index 0000000000000..b59d8ce93f23d --- /dev/null +++ b/core/src/main/scala/spark/SparkFiles.java @@ -0,0 +1,25 @@ +package spark; + +import java.io.File; + +/** + * Resolves paths to files added through `addFile(). + */ +public class SparkFiles { + + private SparkFiles() {} + + /** + * Get the absolute path of a file added through `addFile()`. + */ + public static String get(String filename) { + return new File(getRootDirectory(), filename).getAbsolutePath(); + } + + /** + * Get the root directory that contains files added through `addFile()`. + */ + public static String getRootDirectory() { + return SparkEnv.get().sparkFilesDir(); + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 692a3f4050294..827c8bd81eff4 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -111,20 +111,6 @@ private object Utils extends Logging { } } - /** Copy a file on the local file system */ - def copyFile(source: File, dest: File) { - val in = new FileInputStream(source) - val out = new FileOutputStream(dest) - copyStream(in, out, true) - } - - /** Download a file from a given URL to the local filesystem */ - def downloadFile(url: URL, localPath: String) { - val in = url.openStream() - val out = new FileOutputStream(localPath) - Utils.copyStream(in, out, true) - } - /** * Download a file requested by the executor. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. @@ -201,7 +187,7 @@ private object Utils extends Logging { Utils.execute(Seq("tar", "-xf", filename), targetDir) } // Make the file executable - That's necessary for scripts - FileUtil.chmod(filename, "a+x") + FileUtil.chmod(targetFile.getAbsolutePath, "a+x") } /** diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index 16c122c58426e..50b8970cd8aad 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -323,9 +323,10 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def getSparkHome(): Option[String] = sc.getSparkHome() /** - * Add a file to be downloaded into the working directory of this Spark job on every node. + * Add a file to be downloaded with this Spark job on every node. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported - * filesystems), or an HTTP, HTTPS or FTP URI. + * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, + * use `SparkFiles.get(path)` to find its download location. */ def addFile(path: String) { sc.addFile(path) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 5526406a20902..f43a152ca7498 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -67,6 +67,8 @@ private[spark] class PythonRDD[T: ClassManifest]( val dOut = new DataOutputStream(proc.getOutputStream) // Split index dOut.writeInt(split.index) + // sparkFilesDir + PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dOut) // Broadcast variables dOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index beceb55ecdf54..0d1fe2a6b4970 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -106,11 +106,6 @@ private[spark] class ExecutorRunner( throw new IOException("Failed to create directory " + executorDir) } - // Download the files it depends on into it (disabled for now) - //for (url <- jobDesc.fileUrls) { - // fetchFile(url, executorDir) - //} - // Launch the process val command = buildCommandSeq() val builder = new ProcessBuilder(command: _*).directory(executorDir) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2552958d27e37..70629f6003b04 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -162,16 +162,16 @@ private[spark] class Executor extends Logging { // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(".")) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) currentFiles(name) = timestamp } for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(".")) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) currentJars(name) = timestamp // Add it to our class loader val localName = name.split("/").last - val url = new File(".", localName).toURI.toURL + val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL if (!urlClassLoader.getURLs.contains(url)) { logInfo("Adding " + url + " to class loader") urlClassLoader.addURL(url) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index dff550036d50b..4451d314e68ce 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -116,16 +116,16 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(".")) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) currentFiles(name) = timestamp } for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(".")) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) currentJars(name) = timestamp // Add it to our class loader val localName = name.split("/").last - val url = new File(".", localName).toURI.toURL + val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL if (!classLoader.getURLs.contains(url)) { logInfo("Adding " + url + " to class loader") classLoader.addURL(url) diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index b4283d960488b..528c6b8424310 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -40,7 +40,8 @@ class FileServerSuite extends FunSuite with BeforeAndAfter { sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { - val in = new BufferedReader(new FileReader("FileServerSuite.txt")) + val path = SparkFiles.get("FileServerSuite.txt") + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal @@ -54,7 +55,8 @@ class FileServerSuite extends FunSuite with BeforeAndAfter { sc.addFile((new File(tmpFile.toString)).toURL.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { - val in = new BufferedReader(new FileReader("FileServerSuite.txt")) + val path = SparkFiles.get("FileServerSuite.txt") + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal @@ -83,7 +85,8 @@ class FileServerSuite extends FunSuite with BeforeAndAfter { sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { - val in = new BufferedReader(new FileReader("FileServerSuite.txt")) + val path = SparkFiles.get("FileServerSuite.txt") + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 00666bc0a3a5b..3e8bca62f069f 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -11,6 +11,8 @@ A broadcast variable that gets reused across tasks. - L{Accumulator} An "add-only" shared variable that tasks can only add values to. + - L{SparkFiles} + Access files shipped with jobs. """ import sys import os @@ -19,6 +21,7 @@ from pyspark.context import SparkContext from pyspark.rdd import RDD +from pyspark.files import SparkFiles -__all__ = ["SparkContext", "RDD"] +__all__ = ["SparkContext", "RDD", "SparkFiles"] diff --git a/python/pyspark/context.py b/python/pyspark/context.py index dcbed37270fc3..ec0cc7c2f9671 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1,5 +1,7 @@ import os import atexit +import shutil +import tempfile from tempfile import NamedTemporaryFile from pyspark import accumulators @@ -173,10 +175,26 @@ def accumulator(self, value, accum_param=None): def addFile(self, path): """ - Add a file to be downloaded into the working directory of this Spark - job on every node. The C{path} passed can be either a local file, - a file in HDFS (or other Hadoop-supported filesystems), or an HTTP, - HTTPS or FTP URI. + Add a file to be downloaded with this Spark job on every node. + The C{path} passed can be either a local file, a file in HDFS + (or other Hadoop-supported filesystems), or an HTTP, HTTPS or + FTP URI. + + To access the file in Spark jobs, use + L{SparkFiles.get(path)} to find its + download location. + + >>> from pyspark import SparkFiles + >>> path = os.path.join(tempdir, "test.txt") + >>> with open(path, "w") as testFile: + ... testFile.write("100") + >>> sc.addFile(path) + >>> def func(iterator): + ... with open(SparkFiles.get("test.txt")) as testFile: + ... fileVal = int(testFile.readline()) + ... return [x * 100 for x in iterator] + >>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect() + [100, 200, 300, 400] """ self._jsc.sc().addFile(path) @@ -211,3 +229,17 @@ def setCheckpointDir(self, dirName, useExisting=False): accidental overriding of checkpoint files in the existing directory. """ self._jsc.sc().setCheckpointDir(dirName, useExisting) + + +def _test(): + import doctest + globs = globals().copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + globs['tempdir'] = tempfile.mkdtemp() + atexit.register(lambda: shutil.rmtree(globs['tempdir'])) + doctest.testmod(globs=globs) + globs['sc'].stop() + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/files.py b/python/pyspark/files.py new file mode 100644 index 0000000000000..de1334f046c6b --- /dev/null +++ b/python/pyspark/files.py @@ -0,0 +1,24 @@ +import os + + +class SparkFiles(object): + """ + Resolves paths to files added through + L{addFile()}. + + SparkFiles contains only classmethods; users should not create SparkFiles + instances. + """ + + _root_directory = None + + def __init__(self): + raise NotImplementedError("Do not construct SparkFiles objects") + + @classmethod + def get(cls, filename): + """ + Get the absolute path of a file added through C{addFile()}. + """ + path = os.path.join(SparkFiles._root_directory, filename) + return os.path.abspath(path) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index b2b9288089714..e7bdb7682b1ea 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -8,6 +8,7 @@ from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler +from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, read_with_length, write_int, \ read_long, read_int, dump_pickle, load_pickle, read_from_pickle_file @@ -23,6 +24,8 @@ def load_obj(): def main(): split_index = read_int(sys.stdin) + spark_files_dir = load_pickle(read_with_length(sys.stdin)) + SparkFiles._root_directory = spark_files_dir num_broadcast_variables = read_int(sys.stdin) for _ in range(num_broadcast_variables): bid = read_long(sys.stdin) diff --git a/python/run-tests b/python/run-tests index ce214e98a8f71..a3a9ff5dcb260 100755 --- a/python/run-tests +++ b/python/run-tests @@ -8,6 +8,9 @@ FAILED=0 $FWDIR/pyspark pyspark/rdd.py FAILED=$(($?||$FAILED)) +$FWDIR/pyspark pyspark/context.py +FAILED=$(($?||$FAILED)) + $FWDIR/pyspark -m doctest pyspark/broadcast.py FAILED=$(($?||$FAILED)) From 7b9e96c99206c0679d9925e0161fde738a5c7c3a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 21 Jan 2013 16:45:00 -0800 Subject: [PATCH 018/696] Add synchronization to Executor.updateDependencies() (SPARK-662) --- .../main/scala/spark/executor/Executor.scala | 34 ++++++++++--------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 70629f6003b04..28d9d40d43b92 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -159,22 +159,24 @@ private[spark] class Executor extends Logging { * SparkContext. Also adds any new JARs we fetched to the class loader. */ private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { - // Fetch missing dependencies - for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { - logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) - currentFiles(name) = timestamp - } - for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { - logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) - currentJars(name) = timestamp - // Add it to our class loader - val localName = name.split("/").last - val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL - if (!urlClassLoader.getURLs.contains(url)) { - logInfo("Adding " + url + " to class loader") - urlClassLoader.addURL(url) + synchronized { + // Fetch missing dependencies + for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { + logInfo("Fetching " + name + " with timestamp " + timestamp) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) + currentFiles(name) = timestamp + } + for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { + logInfo("Fetching " + name + " with timestamp " + timestamp) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) + currentJars(name) = timestamp + // Add it to our class loader + val localName = name.split("/").last + val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL + if (!urlClassLoader.getURLs.contains(url)) { + logInfo("Adding " + url + " to class loader") + urlClassLoader.addURL(url) + } } } } From df9ae8a74e2d42f627a82f9b0a8123a4d54b1d09 Mon Sep 17 00:00:00 2001 From: "haitao.yao" Date: Tue, 22 Jan 2013 10:12:37 +0800 Subject: [PATCH 019/696] add ensime --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index c207409e3cfe0..672c60af3d673 100644 --- a/.gitignore +++ b/.gitignore @@ -32,3 +32,5 @@ logs/ log/ spark-tests.log dependency-reduced-pom.xml +.ensime +.ensime_lucene From a8baeb93272b03a98e44c7bf5c541611aec4a64b Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Mon, 21 Jan 2013 21:30:24 -0600 Subject: [PATCH 020/696] Further simplify getOrElse call. --- core/src/main/scala/spark/ParallelCollection.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/ParallelCollection.scala index ad23e5bec8681..10adcd53ecafd 100644 --- a/core/src/main/scala/spark/ParallelCollection.scala +++ b/core/src/main/scala/spark/ParallelCollection.scala @@ -44,7 +44,7 @@ private[spark] class ParallelCollection[T: ClassManifest]( s.asInstanceOf[ParallelCollectionSplit[T]].iterator override def getPreferredLocations(s: Split): Seq[String] = { - locationPrefs.get(s.index) getOrElse Nil + locationPrefs.getOrElse(s.index, Nil) } override def clearDependencies() { From c75ae3622eeed068c44b1f823ef4d87d01a720fd Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 20 Jan 2013 15:12:54 -0800 Subject: [PATCH 021/696] Make AccumulatorParam an abstract base class. --- python/pyspark/accumulators.py | 29 ++++++++++++++++++++++++++--- python/pyspark/context.py | 15 +++++---------- 2 files changed, 31 insertions(+), 13 deletions(-) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 8011779ddc298..5a9269f9bb251 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -61,6 +61,7 @@ Exception:... """ +from abc import ABCMeta, abstractmethod import struct import SocketServer import threading @@ -90,8 +91,7 @@ class Accumulator(object): While C{SparkContext} supports accumulators for primitive data types like C{int} and C{float}, users can also define accumulators for custom types by providing a custom - C{AccumulatorParam} object with a C{zero} and C{addInPlace} method. Refer to the doctest - of this module for an example. + L{AccumulatorParam} object. Refer to the doctest of this module for an example. """ def __init__(self, aid, value, accum_param): @@ -134,7 +134,30 @@ def __repr__(self): return "Accumulator" % (self.aid, self._value) -class AddingAccumulatorParam(object): +class AccumulatorParam(object): + """ + Helper object that defines how to accumulate values of a given type. + """ + __metaclass__ = ABCMeta + + @abstractmethod + def zero(self, value): + """ + Provide a "zero value" for the type, compatible in dimensions with the + provided C{value} (e.g., a zero vector) + """ + return + + @abstractmethod + def addInPlace(self, value1, value2): + """ + Add two values of the accumulator's data type, returning a new value; + for efficiency, can also update C{value1} in place and return it. + """ + return + + +class AddingAccumulatorParam(AccumulatorParam): """ An AccumulatorParam that uses the + operators to add values. Designed for simple types such as integers, floats, and lists. Requires the zero value for the underlying type diff --git a/python/pyspark/context.py b/python/pyspark/context.py index dcbed37270fc3..a17e7a4ad15ff 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -148,16 +148,11 @@ def broadcast(self, value): def accumulator(self, value, accum_param=None): """ - Create an C{Accumulator} with the given initial value, using a given - AccumulatorParam helper object to define how to add values of the data - type if provided. Default AccumulatorParams are used for integers and - floating-point numbers if you do not provide one. For other types, the - AccumulatorParam must implement two methods: - - C{zero(value)}: provide a "zero value" for the type, compatible in - dimensions with the provided C{value} (e.g., a zero vector). - - C{addInPlace(val1, val2)}: add two values of the accumulator's data - type, returning a new value; for efficiency, can also update C{val1} - in place and return it. + Create an L{Accumulator} with the given initial value, using a given + L{AccumulatorParam} helper object to define how to add values of the + data type if provided. Default AccumulatorParams are used for integers + and floating-point numbers if you do not provide one. For other types, + a custom AccumulatorParam can be used. """ if accum_param == None: if isinstance(value, int): From 551a47a620c7dc207e3530e54d794a3c3aa8e45e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 21 Jan 2013 23:31:00 -0800 Subject: [PATCH 022/696] Refactor daemon thread pool creation. --- .../scala/spark/DaemonThreadFactory.scala | 18 ---------- core/src/main/scala/spark/Utils.scala | 33 ++++--------------- .../spark/network/ConnectionManager.scala | 5 ++- .../scheduler/local/LocalScheduler.scala | 2 +- .../streaming/dstream/RawInputDStream.scala | 5 +-- 5 files changed, 13 insertions(+), 50 deletions(-) delete mode 100644 core/src/main/scala/spark/DaemonThreadFactory.scala diff --git a/core/src/main/scala/spark/DaemonThreadFactory.scala b/core/src/main/scala/spark/DaemonThreadFactory.scala deleted file mode 100644 index 56e59adeb7152..0000000000000 --- a/core/src/main/scala/spark/DaemonThreadFactory.scala +++ /dev/null @@ -1,18 +0,0 @@ -package spark - -import java.util.concurrent.ThreadFactory - -/** - * A ThreadFactory that creates daemon threads - */ -private object DaemonThreadFactory extends ThreadFactory { - override def newThread(r: Runnable): Thread = new DaemonThread(r) -} - -private class DaemonThread(r: Runnable = null) extends Thread { - override def run() { - if (r != null) { - r.run() - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 692a3f4050294..9b8636f6c8baf 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -10,6 +10,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.io.Source import com.google.common.io.Files +import com.google.common.util.concurrent.ThreadFactoryBuilder /** * Various utility methods used by Spark. @@ -287,29 +288,14 @@ private object Utils extends Logging { customHostname.getOrElse(InetAddress.getLocalHost.getHostName) } - /** - * Returns a standard ThreadFactory except all threads are daemons. - */ - private def newDaemonThreadFactory: ThreadFactory = { - new ThreadFactory { - def newThread(r: Runnable): Thread = { - var t = Executors.defaultThreadFactory.newThread (r) - t.setDaemon (true) - return t - } - } - } + private[spark] val daemonThreadFactory: ThreadFactory = + new ThreadFactoryBuilder().setDaemon(true).build() /** * Wrapper over newCachedThreadPool. */ - def newDaemonCachedThreadPool(): ThreadPoolExecutor = { - var threadPool = Executors.newCachedThreadPool.asInstanceOf[ThreadPoolExecutor] - - threadPool.setThreadFactory (newDaemonThreadFactory) - - return threadPool - } + def newDaemonCachedThreadPool(): ThreadPoolExecutor = + Executors.newCachedThreadPool(daemonThreadFactory).asInstanceOf[ThreadPoolExecutor] /** * Return the string to tell how long has passed in seconds. The passing parameter should be in @@ -322,13 +308,8 @@ private object Utils extends Logging { /** * Wrapper over newFixedThreadPool. */ - def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor = { - var threadPool = Executors.newFixedThreadPool(nThreads).asInstanceOf[ThreadPoolExecutor] - - threadPool.setThreadFactory(newDaemonThreadFactory) - - return threadPool - } + def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor = + Executors.newFixedThreadPool(nThreads, daemonThreadFactory).asInstanceOf[ThreadPoolExecutor] /** * Delete a file or directory and its contents recursively. diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 36c01ad629bbb..2ecd14f536747 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -52,9 +52,8 @@ private[spark] class ConnectionManager(port: Int) extends Logging { val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)] val sendMessageRequests = new Queue[(Message, SendingConnection)] - implicit val futureExecContext = ExecutionContext.fromExecutor( - Executors.newCachedThreadPool(DaemonThreadFactory)) - + implicit val futureExecContext = ExecutionContext.fromExecutor(Utils.newDaemonCachedThreadPool()) + var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null serverChannel.configureBlocking(false) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index dff550036d50b..87f8474ea0bf3 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -20,7 +20,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon with Logging { var attemptId = new AtomicInteger(0) - var threadPool = Executors.newFixedThreadPool(threads, DaemonThreadFactory) + var threadPool = Utils.newDaemonFixedThreadPool(threads) val env = SparkEnv.get var listener: TaskSchedulerListener = null diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 290fab1ce0f45..04e6b69b7ba9d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -1,6 +1,6 @@ package spark.streaming.dstream -import spark.{DaemonThread, Logging} +import spark.Logging import spark.storage.StorageLevel import spark.streaming.StreamingContext @@ -48,7 +48,8 @@ class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) val queue = new ArrayBlockingQueue[ByteBuffer](2) - blockPushingThread = new DaemonThread { + blockPushingThread = new Thread { + setDaemon(true) override def run() { var nextBlockNumber = 0 while (true) { From d17065c4b565ec975a46c6d375998ef8ae7a32d5 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 22 Jan 2013 13:28:29 +0530 Subject: [PATCH 023/696] actor as receiver --- docs/plugin-custom-receiver.md | 101 ++++++++++++ .../streaming/examples/ActorWordCount.scala | 130 +++++++++++++++ .../spark/streaming/StreamingContext.scala | 35 ++++ .../dstream/PluggableInputDStream.scala | 13 ++ .../streaming/receivers/ActorReceiver.scala | 153 ++++++++++++++++++ .../spark/streaming/InputStreamsSuite.scala | 68 ++++++++ 6 files changed, 500 insertions(+) create mode 100644 docs/plugin-custom-receiver.md create mode 100644 examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala create mode 100644 streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala create mode 100644 streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala diff --git a/docs/plugin-custom-receiver.md b/docs/plugin-custom-receiver.md new file mode 100644 index 0000000000000..0eb4246158e53 --- /dev/null +++ b/docs/plugin-custom-receiver.md @@ -0,0 +1,101 @@ +--- +layout: global +title: Tutorial - Spark streaming, Plugging in a custom receiver. +--- + +A "Spark streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. + +This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. + + +## A quick and naive walk-through + +### Write a simple receiver + +This starts with implementing [Actor](#References) + +Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api. + +{% highlight scala %} + + class SocketTextStreamReceiver (host:String, + port:Int, + bytesToString: ByteString => String) extends Actor with Receiver { + + override def preStart = IOManager(context.system).connect(host, port) + + def receive = { + case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes)) + } + + } + + +{% endhighlight %} + +All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details. + +### A sample spark application + +* First create a Spark streaming context with master url and batchduration. + +{% highlight scala %} + + val ssc = new StreamingContext(master, "WordCountCustomStreamSource", + Seconds(batchDuration)) + +{% endhighlight %} + +* Plug-in the actor configuration into the spark streaming context and create a DStream. + +{% highlight scala %} + + val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8445, z => z.utf8String)),"SocketReceiver") + +{% endhighlight %} + +* Process it. + +{% highlight scala %} + + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + + wordCounts.print() + ssc.start() + + +{% endhighlight %} + +* After processing it, stream can be tested using the netcat utility. + + $ nc -l localhost 8445 + hello world + hello hello + + +## Multiple homogeneous/heterogeneous receivers. + +A DStream union operation is provided for taking union on multiple input streams. + +{% highlight scala %} + + val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8445, z => z.utf8String)),"SocketReceiver") + + // Another socket stream receiver + val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8446, z => z.utf8String)),"SocketReceiver") + + val union = lines.union(lines2) + +{% endhighlight %} + +Above stream can be easily process as described earlier. + +_A more comprehensive example is provided in the spark streaming examples_ + +## References + +1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala new file mode 100644 index 0000000000000..c3d3755953242 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -0,0 +1,130 @@ +package spark.streaming.examples + +import scala.collection.mutable.LinkedList +import scala.util.Random + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.actor.actorRef2Scala + +import spark.streaming.Seconds +import spark.streaming.StreamingContext +import spark.streaming.StreamingContext.toPairDStreamFunctions +import spark.streaming.receivers.Receiver +import spark.util.AkkaUtils + +case class SubscribeReceiver(receiverActor: ActorRef) +case class UnsubscribeReceiver(receiverActor: ActorRef) + +/** + * Sends the random content to every receiver subscribed with 1/2 + * second delay. + */ +class FeederActor extends Actor { + + val rand = new Random() + var receivers: LinkedList[ActorRef] = new LinkedList[ActorRef]() + + val strings: Array[String] = Array("words ", "may ", "count ") + + def makeMessage(): String = { + val x = rand.nextInt(3) + strings(x) + strings(2 - x) + } + + /* + * A thread to generate random messages + */ + new Thread() { + override def run() { + while (true) { + Thread.sleep(500) + receivers.foreach(_ ! makeMessage) + } + } + }.start() + + def receive: Receive = { + + case SubscribeReceiver(receiverActor: ActorRef) => + println("received subscribe from %s".format(receiverActor.toString)) + receivers = LinkedList(receiverActor) ++ receivers + + case UnsubscribeReceiver(receiverActor: ActorRef) => + println("received unsubscribe from %s".format(receiverActor.toString)) + receivers = receivers.dropWhile(x => x eq receiverActor) + + } +} + +/** + * A sample actor as receiver is also simplest. This receiver actor + * goes and subscribe to a typical publisher/feeder actor and receives + * data, thus it is important to have feeder running before this example + * can be run. + * + * @see [[spark.streaming.examples.FeederActor]] + */ +class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) + extends Actor with Receiver { + + lazy private val remotePublisher = context.actorFor(urlOfPublisher) + + override def preStart = remotePublisher ! SubscribeReceiver(context.self) + + def receive = { + case msg ⇒ context.parent ! pushBlock(msg.asInstanceOf[T]) + } + + override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self) + +} + +/** + * A sample word count program demonstrating the use of plugging in + * Actor as Receiver + */ +object ActorWordCount { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println( + "Usage: ActorWordCount " + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + val Seq(master, host, port) = args.toSeq + + // Create the context and set the batch size + val ssc = new StreamingContext(master, "ActorWordCount", + Seconds(10)) + + //Start feeder actor on this actor system. + val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1 + + val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") + + /* + * Following is the use of actorStream to plug in custom actor as receiver + * + * An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e type of data received and InputDstream + * should be same. + * + * For example: Both actorStream and SampleActorReceiver are parameterized + * to same type to ensure type safety. + */ + + val lines = ssc.actorStream[String]( + Props(new SampleActorReceiver[String]("akka://spark@%s:%s/user/FeederActor".format( + host, port.toInt))), "SampleReceiver") + + //compute wordcount + lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print() + + ssc.start() + + } +} diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 14500bdcb17a0..9d960e883fad7 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -1,8 +1,13 @@ package spark.streaming +import akka.actor.Props +import akka.actor.SupervisorStrategy + import spark.streaming.dstream._ import spark.{RDD, Logging, SparkEnv, SparkContext} +import spark.streaming.receivers.ActorReceiver +import spark.streaming.receivers.ReceiverSupervisorStrategy import spark.storage.StorageLevel import spark.util.MetadataCleaner @@ -134,6 +139,36 @@ class StreamingContext private ( protected[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement() + /** + * Create an input stream with any arbitrary user implemented network receiver. + * @param receiver Custom implementation of NetworkReceiver + */ + def pluggableNetworkStream[T: ClassManifest]( + receiver: NetworkReceiver[T]): DStream[T] = { + val inputStream = new PluggableInputDStream[T](this, + receiver) + graph.addInputStream(inputStream) + inputStream + } + + /** + * Create an input stream with any arbitrary user implemented actor receiver. + * @param props Props object defining creation of the actor + * @param name Name of the actor + * @param storageLevel RDD storage level. Defaults to memory-only. + * + * @note An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e parametrized type of data received and actorStream + * should be same. + */ + def actorStream[T: ClassManifest]( + props: Props, name: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = { + pluggableNetworkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) + } + /** * Create an input stream that pulls messages form a Kafka Broker. * @param hostname Zookeper hostname. diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala new file mode 100644 index 0000000000000..3c2a81947b96b --- /dev/null +++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala @@ -0,0 +1,13 @@ +package spark.streaming.dstream + +import spark.streaming.StreamingContext + +private[streaming] +class PluggableInputDStream[T: ClassManifest]( + @transient ssc_ : StreamingContext, + receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { + + def getReceiver(): NetworkReceiver[T] = { + receiver + } +} diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala new file mode 100644 index 0000000000000..b3201d0b28d79 --- /dev/null +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -0,0 +1,153 @@ +package spark.streaming.receivers + +import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } +import akka.actor.{ actorRef2Scala, ActorRef } +import akka.actor.{ PossiblyHarmful, OneForOneStrategy } + +import spark.storage.StorageLevel +import spark.streaming.dstream.NetworkReceiver + +import java.util.concurrent.atomic.AtomicInteger + +/** A helper with set of defaults for supervisor strategy **/ +object ReceiverSupervisorStrategy { + + import akka.util.duration._ + import akka.actor.SupervisorStrategy._ + + val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = + 15 millis) { + case _: RuntimeException ⇒ Restart + case _: Exception ⇒ Escalate + } +} + +/** + * A receiver trait to be mixed in with your Actor to gain access to + * pushBlock API. + * + * @example {{{ + * class MyActor extends Actor with Receiver{ + * def receive { + * case anything :String ⇒ pushBlock(anything) + * } + * } + * //Can be plugged in actorStream as follows + * ssc.actorStream[String](Props(new MyActor),"MyActorReceiver") + * + * }}} + * + * @note An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e parametrized type of push block and InputDStream + * should be same. + * + */ +trait Receiver { self: Actor ⇒ + def pushBlock[T: ClassManifest](iter: Iterator[T]) { + context.parent ! Data(iter) + } + + def pushBlock[T: ClassManifest](data: T) { + context.parent ! Data(data) + } + +} + +/** + * Statistics for querying the supervisor about state of workers + */ +case class Statistics(numberOfMsgs: Int, + numberOfWorkers: Int, + numberOfHiccups: Int, + otherInfo: String) + +/** Case class to receive data sent by child actors **/ +private[streaming] case class Data[T: ClassManifest](data: T) + +/** + * Provides Actors as receivers for receiving stream. + * + * As Actors can also be used to receive data from almost any stream source. + * A nice set of abstraction(s) for actors as receivers is already provided for + * a few general cases. It is thus exposed as an API where user may come with + * his own Actor to run as receiver for Spark Streaming input source. + * + * This starts a supervisor actor which starts workers and also provides + * [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance]. + * + * Here's a way to start more supervisor/workers as its children. + * + * @example {{{ + * context.parent ! Props(new Supervisor) + * }}} OR {{{ + * context.parent ! Props(new Worker,"Worker") + * }}} + * + * + */ +private[streaming] class ActorReceiver[T: ClassManifest]( + props: Props, + name: String, + storageLevel: StorageLevel, + receiverSupervisorStrategy: SupervisorStrategy) + extends NetworkReceiver[T] { + + protected lazy val blocksGenerator: BlockGenerator = + new BlockGenerator(storageLevel) + + protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor), + "Supervisor" + streamId) + + private class Supervisor extends Actor { + + override val supervisorStrategy = receiverSupervisorStrategy + val worker = context.actorOf(props, name) + logInfo("Started receiver worker at:" + worker.path) + + val n: AtomicInteger = new AtomicInteger(0) + val hiccups: AtomicInteger = new AtomicInteger(0) + + def receive = { + + case Data(iter: Iterator[_]) ⇒ pushBlock(iter.asInstanceOf[Iterator[T]]) + + case Data(msg) ⇒ + blocksGenerator += msg.asInstanceOf[T] + n.incrementAndGet + + case props: Props ⇒ + val worker = context.actorOf(props) + logInfo("Started receiver worker at:" + worker.path) + sender ! worker + + case (props: Props, name: String) ⇒ + val worker = context.actorOf(props, name) + logInfo("Started receiver worker at:" + worker.path) + sender ! worker + + case _: PossiblyHarmful => hiccups.incrementAndGet() + + case _: Statistics ⇒ + val workers = context.children + sender ! Statistics(n.get, workers.size, hiccups.get, workers.mkString("\n")) + + } + } + + protected def pushBlock(iter: Iterator[T]) { + pushBlock("block-" + streamId + "-" + System.nanoTime(), + iter, null, storageLevel) + } + + protected def onStart() = { + blocksGenerator.start() + supervisor + logInfo("Supervision tree for receivers initialized at:" + supervisor.path) + } + + protected def onStop() = { + supervisor ! PoisonPill + } + +} diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index d7ba7a5d17da8..d59750178173e 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -1,5 +1,11 @@ package spark.streaming +import akka.actor.Actor +import akka.actor.IO +import akka.actor.IOManager +import akka.actor.Props +import akka.util.ByteString + import dstream.SparkFlumeEvent import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket} import java.io.{File, BufferedWriter, OutputStreamWriter} @@ -7,6 +13,7 @@ import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import util.ManualClock import spark.storage.StorageLevel +import spark.streaming.receivers.Receiver import spark.Logging import scala.util.Random import org.apache.commons.io.FileUtils @@ -242,6 +249,55 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i).head.toString === expectedOutput(i)) } } + test("actor input stream") { + // Start the server + val port = testPort + testServer = new TestServer(port) + testServer.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(master, framework, batchDuration) + val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", + StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(networkStream, outputBuffer) + def output = outputBuffer.flatMap(x => x) + ssc.registerOutputStream(outputStream) + ssc.start() + + // Feed data to the server to send to the network receiver + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = 1 to 9 + val expectedOutput = input.map(x => x.toString) + Thread.sleep(1000) + for (i <- 0 until input.size) { + testServer.send(input(i).toString) + Thread.sleep(500) + clock.addToTime(batchDuration.milliseconds) + } + Thread.sleep(1000) + logInfo("Stopping server") + testServer.stop() + logInfo("Stopping context") + ssc.stop() + + // Verify whether data received was as expected + logInfo("--------------------------------") + logInfo("output.size = " + outputBuffer.size) + logInfo("output") + outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("expected output.size = " + expectedOutput.size) + logInfo("expected output") + expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("--------------------------------") + + // Verify whether all the elements received are as expected + // (whether the elements were received one in each interval is not verified) + assert(output.size === expectedOutput.size) + for (i <- 0 until output.size) { + assert(output(i) === expectedOutput(i)) + } + } test("file input stream with checkpoint") { // Create a temporary directory @@ -353,3 +409,15 @@ object TestServer { } } } + +class TestActor(port: Int) extends Actor with Receiver { + + def bytesToString(byteString: ByteString) = byteString.utf8String + + override def preStart = IOManager(context.system).connect(new InetSocketAddress(port)) + + def receive = { + case IO.Read(socket, bytes) => + pushBlock(bytesToString(bytes)) + } +} From e353886a8ca6179f25b4176d7a62b5d04ce79276 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 22 Jan 2013 00:23:31 -0800 Subject: [PATCH 024/696] Use generation numbers for fetch failure tracking --- .../scala/spark/scheduler/DAGScheduler.scala | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 59f2099e911a5..39a1e6d6c6f58 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -72,8 +72,12 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val cacheTracker = env.cacheTracker val mapOutputTracker = env.mapOutputTracker - val deadHosts = new HashSet[String] // TODO: The code currently assumes these can't come back; - // that's not going to be a realistic assumption in general + // For tracking failed nodes, we use the MapOutputTracker's generation number, which is + // sent with every task. When we detect a node failing, we note the current generation number + // and failed host, increment it for new tasks, and use this to ignore stray ShuffleMapTask + // results. + // TODO: Garbage collect information about failure generations when new stages start. + val failedGeneration = new HashMap[String, Long] val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done val running = new HashSet[Stage] // Stages we are running right now @@ -429,7 +433,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val status = event.result.asInstanceOf[MapStatus] val host = status.address.ip logInfo("ShuffleMapTask finished with host " + host) - if (!deadHosts.contains(host)) { // TODO: Make sure hostnames are consistent with Mesos + if (failedGeneration.contains(host) && smt.generation <= failedGeneration(host)) { + logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + host) + } else { stage.addOutputLoc(smt.partition, status) } if (running.contains(stage) && pendingTasks(stage).isEmpty) { @@ -495,7 +501,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with lastFetchFailureTime = System.currentTimeMillis() // TODO: Use pluggable clock // TODO: mark the host as failed only if there were lots of fetch failures on it if (bmAddress != null) { - handleHostLost(bmAddress.ip) + handleHostLost(bmAddress.ip, Some(task.generation)) } case other => @@ -507,11 +513,15 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with /** * Responds to a host being lost. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use hostLost() to post a host lost event from outside. + * + * Optionally the generation during which the failure was caught can be passed to avoid allowing + * stray fetch failures from possibly retriggering the detection of a node as lost. */ - def handleHostLost(host: String) { - if (!deadHosts.contains(host)) { + def handleHostLost(host: String, maybeGeneration: Option[Long] = None) { + val currentGeneration = maybeGeneration.getOrElse(mapOutputTracker.getGeneration) + if (!failedGeneration.contains(host) || failedGeneration(host) < currentGeneration) { + failedGeneration(host) = currentGeneration logInfo("Host lost: " + host) - deadHosts += host env.blockManager.master.notifyADeadHost(host) // TODO: This will be really slow if we keep accumulating shuffle map stages for ((shuffleId, stage) <- shuffleToMapStage) { @@ -519,6 +529,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray mapOutputTracker.registerMapOutputs(shuffleId, locs, true) } + if (shuffleToMapStage.isEmpty) { + mapOutputTracker.incrementGeneration() + } cacheTracker.cacheLost(host) updateCacheLocs() } From 364cdb679cf2b0d5e6ed7ab89628f15594d7947f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 22 Jan 2013 00:43:31 -0800 Subject: [PATCH 025/696] Refactored DStreamCheckpointData. --- .../main/scala/spark/streaming/DStream.scala | 58 ++----------- .../streaming/DStreamCheckpointData.scala | 84 +++++++++++++++++++ .../streaming/dstream/KafkaInputDStream.scala | 9 -- .../spark/streaming/CheckpointSuite.scala | 12 +-- 4 files changed, 99 insertions(+), 64 deletions(-) create mode 100644 streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index b11ef443dc5e2..3c1861a840313 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -12,7 +12,7 @@ import scala.collection.mutable.HashMap import java.io.{ObjectInputStream, IOException, ObjectOutputStream} -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.conf.Configuration /** @@ -75,7 +75,7 @@ abstract class DStream[T: ClassManifest] ( // Checkpoint details protected[streaming] val mustCheckpoint = false protected[streaming] var checkpointDuration: Duration = null - protected[streaming] var checkpointData = new DStreamCheckpointData(HashMap[Time, Any]()) + protected[streaming] val checkpointData = new DStreamCheckpointData(this) // Reference to whole DStream graph protected[streaming] var graph: DStreamGraph = null @@ -85,10 +85,10 @@ abstract class DStream[T: ClassManifest] ( // Duration for which the DStream requires its parent DStream to remember each RDD created protected[streaming] def parentRememberDuration = rememberDuration - /** Returns the StreamingContext associated with this DStream */ + /** Return the StreamingContext associated with this DStream */ def context() = ssc - /** Persists the RDDs of this DStream with the given storage level */ + /** Persist the RDDs of this DStream with the given storage level */ def persist(level: StorageLevel): DStream[T] = { if (this.isInitialized) { throw new UnsupportedOperationException( @@ -342,40 +342,10 @@ abstract class DStream[T: ClassManifest] ( */ protected[streaming] def updateCheckpointData(currentTime: Time) { logInfo("Updating checkpoint data for time " + currentTime) - - // Get the checkpointed RDDs from the generated RDDs - val newRdds = generatedRDDs.filter(_._2.getCheckpointFile.isDefined) - .map(x => (x._1, x._2.getCheckpointFile.get)) - - // Make a copy of the existing checkpoint data (checkpointed RDDs) - val oldRdds = checkpointData.rdds.clone() - - // If the new checkpoint data has checkpoints then replace existing with the new one - if (newRdds.size > 0) { - checkpointData.rdds.clear() - checkpointData.rdds ++= newRdds - } - - // Make parent DStreams update their checkpoint data + checkpointData.update() dependencies.foreach(_.updateCheckpointData(currentTime)) - - // TODO: remove this, this is just for debugging - newRdds.foreach { - case (time, data) => { logInfo("Added checkpointed RDD for time " + time + " to stream checkpoint") } - } - - if (newRdds.size > 0) { - (oldRdds -- newRdds.keySet).foreach { - case (time, data) => { - val path = new Path(data.toString) - val fs = path.getFileSystem(new Configuration()) - fs.delete(path, true) - logInfo("Deleted checkpoint file '" + path + "' for time " + time) - } - } - } - logInfo("Updated checkpoint data for time " + currentTime + ", " + checkpointData.rdds.size + " checkpoints, " - + "[" + checkpointData.rdds.mkString(",") + "]") + checkpointData.cleanup() + logDebug("Updated checkpoint data for time " + currentTime + ": " + checkpointData) } /** @@ -386,14 +356,8 @@ abstract class DStream[T: ClassManifest] ( */ protected[streaming] def restoreCheckpointData() { // Create RDDs from the checkpoint data - logInfo("Restoring checkpoint data from " + checkpointData.rdds.size + " checkpointed RDDs") - checkpointData.rdds.foreach { - case(time, data) => { - logInfo("Restoring checkpointed RDD for time " + time + " from file '" + data.toString + "'") - val rdd = ssc.sc.checkpointFile[T](data.toString) - generatedRDDs += ((time, rdd)) - } - } + logInfo("Restoring checkpoint data from " + checkpointData.checkpointFiles.size + " checkpointed RDDs") + checkpointData.restore() dependencies.foreach(_.restoreCheckpointData()) logInfo("Restored checkpoint data") } @@ -651,7 +615,3 @@ abstract class DStream[T: ClassManifest] ( ssc.registerOutputStream(this) } } - -private[streaming] -case class DStreamCheckpointData(rdds: HashMap[Time, Any]) - diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala new file mode 100644 index 0000000000000..abf903293f80d --- /dev/null +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -0,0 +1,84 @@ +package spark.streaming + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.conf.Configuration +import collection.mutable.HashMap +import spark.Logging + + + +private[streaming] +class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) + extends Serializable with Logging { + private[streaming] val checkpointFiles = new HashMap[Time, String]() + @transient private lazy val fileSystem = + new Path(dstream.context.checkpointDir).getFileSystem(new Configuration()) + @transient private var lastCheckpointFiles: HashMap[Time, String] = null + + /** + * Update the checkpoint data of the DStream. Default implementation records the checkpoint files to + * which the generate RDDs of the DStream has been saved. + */ + def update() { + + // Get the checkpointed RDDs from the generated RDDs + val newCheckpointFiles = dstream.generatedRDDs.filter(_._2.getCheckpointFile.isDefined) + .map(x => (x._1, x._2.getCheckpointFile.get)) + + // Make a copy of the existing checkpoint data (checkpointed RDDs) + lastCheckpointFiles = checkpointFiles.clone() + + // If the new checkpoint data has checkpoints then replace existing with the new one + if (newCheckpointFiles.size > 0) { + checkpointFiles.clear() + checkpointFiles ++= newCheckpointFiles + } + + // TODO: remove this, this is just for debugging + newCheckpointFiles.foreach { + case (time, data) => { logInfo("Added checkpointed RDD for time " + time + " to stream checkpoint") } + } + } + + /** + * Cleanup old checkpoint data. Default implementation, cleans up old checkpoint files. + */ + def cleanup() { + // If there is at least on checkpoint file in the current checkpoint files, + // then delete the old checkpoint files. + if (checkpointFiles.size > 0 && lastCheckpointFiles != null) { + (lastCheckpointFiles -- checkpointFiles.keySet).foreach { + case (time, file) => { + try { + val path = new Path(file) + fileSystem.delete(path, true) + logInfo("Deleted checkpoint file '" + file + "' for time " + time) + } catch { + case e: Exception => + logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e) + } + } + } + } + } + + /** + * Restore the checkpoint data. Default implementation restores the RDDs from their + * checkpoint files. + */ + def restore() { + // Create RDDs from the checkpoint data + checkpointFiles.foreach { + case(time, file) => { + logInfo("Restoring checkpointed RDD for time " + time + " from file '" + file + "'") + dstream.generatedRDDs += ((time, dstream.context.sc.checkpointFile[T](file))) + } + } + } + + override def toString() = { + "[\n" + checkpointFiles.size + "\n" + checkpointFiles.mkString("\n") + "\n]" + } +} + diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 2b4740bdf7c7d..760d9b5cf36e0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -19,15 +19,6 @@ import scala.collection.JavaConversions._ // Key for a specific Kafka Partition: (broker, topic, group, part) case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, partId: Int) -// NOT USED - Originally intended for fault-tolerance -// Metadata for a Kafka Stream that it sent to the Master -private[streaming] -case class KafkaInputDStreamMetadata(timestamp: Long, data: Map[KafkaPartitionKey, Long]) -// NOT USED - Originally intended for fault-tolerance -// Checkpoint data specific to a KafkaInputDstream -private[streaming] -case class KafkaDStreamCheckpointData(kafkaRdds: HashMap[Time, Any], - savedOffsets: Map[KafkaPartitionKey, Long]) extends DStreamCheckpointData(kafkaRdds) /** * Input stream that pulls messages from a Kafka Broker. diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index d2f32c189bc98..58da4ee539816 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -63,9 +63,9 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // then check whether some RDD has been checkpointed or not ssc.start() runStreamsWithRealDelay(ssc, firstNumBatches) - logInfo("Checkpoint data of state stream = \n[" + stateStream.checkpointData.rdds.mkString(",\n") + "]") - assert(!stateStream.checkpointData.rdds.isEmpty, "No checkpointed RDDs in state stream before first failure") - stateStream.checkpointData.rdds.foreach { + logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData) + assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure") + stateStream.checkpointData.checkpointFiles.foreach { case (time, data) => { val file = new File(data.toString) assert(file.exists(), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist") @@ -74,7 +74,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Run till a further time such that previous checkpoint files in the stream would be deleted // and check whether the earlier checkpoint files are deleted - val checkpointFiles = stateStream.checkpointData.rdds.map(x => new File(x._2.toString)) + val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2)) runStreamsWithRealDelay(ssc, secondNumBatches) checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted")) ssc.stop() @@ -91,8 +91,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // is present in the checkpoint data or not ssc.start() runStreamsWithRealDelay(ssc, 1) - assert(!stateStream.checkpointData.rdds.isEmpty, "No checkpointed RDDs in state stream before second failure") - stateStream.checkpointData.rdds.foreach { + assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure") + stateStream.checkpointData.checkpointFiles.foreach { case (time, data) => { val file = new File(data.toString) assert(file.exists(), From 50e2b23927956c14db40093d31bc80892764006a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 22 Jan 2013 09:27:33 -0800 Subject: [PATCH 026/696] Fix up some problems from the merge --- .../scala/spark/storage/BlockManagerMasterActor.scala | 11 +++++++++++ .../scala/spark/storage/BlockManagerMessages.scala | 3 +++ core/src/main/scala/spark/storage/StorageUtils.scala | 8 ++++---- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index f4d026da3329c..c945c34c711c3 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -68,6 +68,9 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { case GetMemoryStatus => getMemoryStatus + case GetStorageStatus => + getStorageStatus + case RemoveBlock(blockId) => removeBlock(blockId) @@ -177,6 +180,14 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { sender ! res } + private def getStorageStatus() { + val res = blockManagerInfo.map { case(blockManagerId, info) => + import collection.JavaConverters._ + StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap) + } + sender ! res + } + private def register(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { val startTimeMs = System.currentTimeMillis() val tmp = " " + blockManagerId + " " diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index d73a9b790f479..3a381fd385650 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -100,3 +100,6 @@ case object GetMemoryStatus extends ToBlockManagerMaster private[spark] case object ExpireDeadHosts extends ToBlockManagerMaster + +private[spark] +case object GetStorageStatus extends ToBlockManagerMaster \ No newline at end of file diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index ebc7390ee598c..63ad5c125bd33 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -1,6 +1,7 @@ package spark.storage import spark.SparkContext +import BlockManagerMasterActor.BlockStatus private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, @@ -20,8 +21,8 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } -case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numPartitions: Int, memSize: Long, diskSize: Long, locations: Array[BlockManagerId]) +case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, + numPartitions: Int, memSize: Long, diskSize: Long) /* Helper methods for storage-related objects */ @@ -58,8 +59,7 @@ object StorageUtils { val rddName = Option(sc.persistentRdds.get(rddId).name).getOrElse(rddKey) val rddStorageLevel = sc.persistentRdds.get(rddId).getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, memSize, diskSize, - rddBlocks.map(_.blockManagerId)) + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, memSize, diskSize) }.toArray } From 6f2194f7576eb188c23f18125f5101ae0b4e9e4d Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 22 Jan 2013 15:38:58 -0600 Subject: [PATCH 027/696] Call removeJob instead of killing the cluster. --- core/src/main/scala/spark/deploy/master/Master.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 2c2cd0231be70..d1a65204b8e7a 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -103,8 +103,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor val e = new SparkException("Job %s wth ID %s failed %d times.".format( jobInfo.desc.name, jobInfo.id, jobInfo.retryCount)) logError(e.getMessage, e) - throw e - //System.exit(1) + removeJob(jobInfo) } } } From 250fe89679bb59ef0d31f74985f72556dcfe2d06 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 22 Jan 2013 16:29:05 -0600 Subject: [PATCH 028/696] Handle Master telling the Worker to kill an already-dead executor. --- core/src/main/scala/spark/deploy/worker/Worker.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 19bf2be1181b2..d040b86908393 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -143,9 +143,13 @@ private[spark] class Worker( case KillExecutor(jobId, execId) => val fullId = jobId + "/" + execId - val executor = executors(fullId) - logInfo("Asked to kill executor " + fullId) - executor.kill() + executors.get(fullId) match { + case Some(executor) => + logInfo("Asked to kill executor " + fullId) + executor.kill() + case None => + logInfo("Asked to kill non-existent existent " + fullId) + } case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => masterDisconnected() From 2437f6741b9c5b0a778d55d324aabdc4642889e5 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 22 Jan 2013 18:01:03 -0600 Subject: [PATCH 029/696] Restore SPARK_MEM in executorEnvs. --- core/src/main/scala/spark/SparkContext.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index a5a1b759449fb..402355bd52463 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -111,8 +111,9 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() - // Note: SPARK_MEM isn't included because it's set directly in ExecutorRunner - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { + // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner + for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", + "SPARK_TESTING")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value From fdec42385a1a8f10f9dd803525cb3c132a25ba53 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 22 Jan 2013 18:01:12 -0600 Subject: [PATCH 030/696] Fix SPARK_MEM in ExecutorRunner. --- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 2f2ea617ff1fc..e910416235c9d 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -118,7 +118,7 @@ private[spark] class ExecutorRunner( for ((key, value) <- jobDesc.command.environment) { env.put(key, value) } - env.put("SPARK_MEM", memory.toString) + env.put("SPARK_MEM", memory.toString + "m") // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") From 8c51322cd05f2ae97a08c3af314c7608fcf71b57 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 22 Jan 2013 18:09:10 -0600 Subject: [PATCH 031/696] Don't bother creating an exception. --- core/src/main/scala/spark/deploy/master/Master.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index d1a65204b8e7a..361e5ac627654 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -100,9 +100,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor if (jobInfo.incrementRetryCount <= JobState.MAX_NUM_RETRY) { schedule() } else { - val e = new SparkException("Job %s wth ID %s failed %d times.".format( + logError("Job %s wth ID %s failed %d times, removing it".format( jobInfo.desc.name, jobInfo.id, jobInfo.retryCount)) - logError(e.getMessage, e) removeJob(jobInfo) } } From 98d0b7747d7539db009a9bbc261f899955871524 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 22 Jan 2013 18:11:51 -0600 Subject: [PATCH 032/696] Fix Worker logInfo about unknown executor. --- core/src/main/scala/spark/deploy/worker/Worker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index d040b86908393..5a83a42daff17 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -148,7 +148,7 @@ private[spark] class Worker( logInfo("Asked to kill executor " + fullId) executor.kill() case None => - logInfo("Asked to kill non-existent existent " + fullId) + logInfo("Asked to kill unknown executor " + fullId) } case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => From 35168d9c89904f0dc0bb470c1799f5ca3b04221f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 22 Jan 2013 17:54:11 -0800 Subject: [PATCH 033/696] Fix sys.path bug in PySpark SparkContext.addPyFile --- python/pyspark/context.py | 2 -- python/pyspark/tests.py | 38 ++++++++++++++++++++++++++---- python/pyspark/worker.py | 1 + python/test_support/userlibrary.py | 7 ++++++ 4 files changed, 41 insertions(+), 7 deletions(-) create mode 100755 python/test_support/userlibrary.py diff --git a/python/pyspark/context.py b/python/pyspark/context.py index ec0cc7c2f9671..b8d7dc05af43d 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -215,8 +215,6 @@ def addPyFile(self, path): """ self.addFile(path) filename = path.split("/")[-1] - os.environ["PYTHONPATH"] = \ - "%s:%s" % (filename, os.environ["PYTHONPATH"]) def setCheckpointDir(self, dirName, useExisting=False): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b0a403b58054a..4d70ee4f12500 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -9,21 +9,32 @@ import unittest from pyspark.context import SparkContext +from pyspark.java_gateway import SPARK_HOME -class TestCheckpoint(unittest.TestCase): +class PySparkTestCase(unittest.TestCase): def setUp(self): - self.sc = SparkContext('local[4]', 'TestPartitioning', batchSize=2) - self.checkpointDir = NamedTemporaryFile(delete=False) - os.unlink(self.checkpointDir.name) - self.sc.setCheckpointDir(self.checkpointDir.name) + class_name = self.__class__.__name__ + self.sc = SparkContext('local[4]', class_name , batchSize=2) def tearDown(self): self.sc.stop() # To avoid Akka rebinding to the same port, since it doesn't unbind # immediately on shutdown self.sc.jvm.System.clearProperty("spark.master.port") + + +class TestCheckpoint(PySparkTestCase): + + def setUp(self): + PySparkTestCase.setUp(self) + self.checkpointDir = NamedTemporaryFile(delete=False) + os.unlink(self.checkpointDir.name) + self.sc.setCheckpointDir(self.checkpointDir.name) + + def tearDown(self): + PySparkTestCase.tearDown(self) shutil.rmtree(self.checkpointDir.name) def test_basic_checkpointing(self): @@ -57,5 +68,22 @@ def test_checkpoint_and_restore(self): self.assertEquals([1, 2, 3, 4], recovered.collect()) +class TestAddFile(PySparkTestCase): + + def test_add_py_file(self): + # To ensure that we're actually testing addPyFile's effects, check that + # this job fails due to `userlibrary` not being on the Python path: + def func(x): + from userlibrary import UserClass + return UserClass().hello() + self.assertRaises(Exception, + self.sc.parallelize(range(2)).map(func).first) + # Add the file, so the job should now succeed: + path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") + self.sc.addPyFile(path) + res = self.sc.parallelize(range(2)).map(func).first() + self.assertEqual("Hello World!", res) + + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index e7bdb7682b1ea..4bf643da66d1d 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -26,6 +26,7 @@ def main(): split_index = read_int(sys.stdin) spark_files_dir = load_pickle(read_with_length(sys.stdin)) SparkFiles._root_directory = spark_files_dir + sys.path.append(spark_files_dir) num_broadcast_variables = read_int(sys.stdin) for _ in range(num_broadcast_variables): bid = read_long(sys.stdin) diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py new file mode 100755 index 0000000000000..5bb6f5009ffbb --- /dev/null +++ b/python/test_support/userlibrary.py @@ -0,0 +1,7 @@ +""" +Used to test shipping of code depenencies with SparkContext.addPyFile(). +""" + +class UserClass(object): + def hello(self): + return "Hello World!" From fad2b82fc8fb49f2171af10cf7e408d8b8dd7349 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 22 Jan 2013 18:10:00 -0800 Subject: [PATCH 034/696] Added support for saving input files of FileInputDStream to graph checkpoints. Modified 'file input stream with checkpoint' testcase to test recovery of pre-master-failure input files. --- .../main/scala/spark/streaming/DStream.scala | 29 +++--- .../streaming/DStreamCheckpointData.scala | 27 ++++-- .../scala/spark/streaming/DStreamGraph.scala | 2 +- .../spark/streaming/StreamingContext.scala | 7 +- .../streaming/dstream/FileInputDStream.scala | 96 +++++++++++++++---- .../spark/streaming/InputStreamsSuite.scala | 64 +++++++++---- 6 files changed, 159 insertions(+), 66 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 3c1861a840313..07ecb018eee73 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -86,7 +86,7 @@ abstract class DStream[T: ClassManifest] ( protected[streaming] def parentRememberDuration = rememberDuration /** Return the StreamingContext associated with this DStream */ - def context() = ssc + def context = ssc /** Persist the RDDs of this DStream with the given storage level */ def persist(level: StorageLevel): DStream[T] = { @@ -159,7 +159,7 @@ abstract class DStream[T: ClassManifest] ( ) assert( - checkpointDuration == null || ssc.sc.checkpointDir.isDefined, + checkpointDuration == null || context.sparkContext.checkpointDir.isDefined, "The checkpoint directory has not been set. Please use StreamingContext.checkpoint()" + " or SparkContext.checkpoint() to set the checkpoint directory." ) @@ -298,8 +298,8 @@ abstract class DStream[T: ClassManifest] ( getOrCompute(time) match { case Some(rdd) => { val jobFunc = () => { - val emptyFunc = { (iterator: Iterator[T]) => {} } - ssc.sc.runJob(rdd, emptyFunc) + val emptyFunc = { (iterator: Iterator[T]) => {} } + context.sparkContext.runJob(rdd, emptyFunc) } Some(new Job(time, jobFunc)) } @@ -310,10 +310,9 @@ abstract class DStream[T: ClassManifest] ( /** * Dereference RDDs that are older than rememberDuration. */ - protected[streaming] def forgetOldRDDs(time: Time) { - val keys = generatedRDDs.keys + protected[streaming] def forgetOldMetadata(time: Time) { var numForgotten = 0 - keys.foreach(t => { + generatedRDDs.keys.foreach(t => { if (t <= (time - rememberDuration)) { generatedRDDs.remove(t) numForgotten += 1 @@ -321,7 +320,7 @@ abstract class DStream[T: ClassManifest] ( } }) logInfo("Forgot " + numForgotten + " RDDs from " + this) - dependencies.foreach(_.forgetOldRDDs(time)) + dependencies.foreach(_.forgetOldMetadata(time)) } /* Adds metadata to the Stream while it is running. @@ -356,7 +355,7 @@ abstract class DStream[T: ClassManifest] ( */ protected[streaming] def restoreCheckpointData() { // Create RDDs from the checkpoint data - logInfo("Restoring checkpoint data from " + checkpointData.checkpointFiles.size + " checkpointed RDDs") + logInfo("Restoring checkpoint data") checkpointData.restore() dependencies.foreach(_.restoreCheckpointData()) logInfo("Restored checkpoint data") @@ -397,7 +396,7 @@ abstract class DStream[T: ClassManifest] ( /** Return a new DStream by applying a function to all elements of this DStream. */ def map[U: ClassManifest](mapFunc: T => U): DStream[U] = { - new MappedDStream(this, ssc.sc.clean(mapFunc)) + new MappedDStream(this, context.sparkContext.clean(mapFunc)) } /** @@ -405,7 +404,7 @@ abstract class DStream[T: ClassManifest] ( * and then flattening the results */ def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = { - new FlatMappedDStream(this, ssc.sc.clean(flatMapFunc)) + new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc)) } /** Return a new DStream containing only the elements that satisfy a predicate. */ @@ -427,7 +426,7 @@ abstract class DStream[T: ClassManifest] ( mapPartFunc: Iterator[T] => Iterator[U], preservePartitioning: Boolean = false ): DStream[U] = { - new MapPartitionedDStream(this, ssc.sc.clean(mapPartFunc), preservePartitioning) + new MapPartitionedDStream(this, context.sparkContext.clean(mapPartFunc), preservePartitioning) } /** @@ -456,7 +455,7 @@ abstract class DStream[T: ClassManifest] ( * this DStream will be registered as an output stream and therefore materialized. */ def foreach(foreachFunc: (RDD[T], Time) => Unit) { - val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc)) + val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc)) ssc.registerOutputStream(newStream) newStream } @@ -474,7 +473,7 @@ abstract class DStream[T: ClassManifest] ( * on each RDD of this DStream. */ def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { - new TransformedDStream(this, ssc.sc.clean(transformFunc)) + new TransformedDStream(this, context.sparkContext.clean(transformFunc)) } /** @@ -491,7 +490,7 @@ abstract class DStream[T: ClassManifest] ( if (first11.size > 10) println("...") println() } - val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc)) + val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc)) ssc.registerOutputStream(newStream) } diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala index abf903293f80d..a375980b84a5c 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -11,14 +11,17 @@ import spark.Logging private[streaming] class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) extends Serializable with Logging { - private[streaming] val checkpointFiles = new HashMap[Time, String]() - @transient private lazy val fileSystem = - new Path(dstream.context.checkpointDir).getFileSystem(new Configuration()) + protected val data = new HashMap[Time, AnyRef]() + + @transient private var fileSystem : FileSystem = null @transient private var lastCheckpointFiles: HashMap[Time, String] = null + protected[streaming] def checkpointFiles = data.asInstanceOf[HashMap[Time, String]] + /** - * Update the checkpoint data of the DStream. Default implementation records the checkpoint files to - * which the generate RDDs of the DStream has been saved. + * Updates the checkpoint data of the DStream. This gets called every time + * the graph checkpoint is initiated. Default implementation records the + * checkpoint files to which the generate RDDs of the DStream has been saved. */ def update() { @@ -42,7 +45,9 @@ class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) } /** - * Cleanup old checkpoint data. Default implementation, cleans up old checkpoint files. + * Cleanup old checkpoint data. This gets called every time the graph + * checkpoint is initiated, but after `update` is called. Default + * implementation, cleans up old checkpoint files. */ def cleanup() { // If there is at least on checkpoint file in the current checkpoint files, @@ -52,6 +57,9 @@ class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) case (time, file) => { try { val path = new Path(file) + if (fileSystem == null) { + fileSystem = path.getFileSystem(new Configuration()) + } fileSystem.delete(path, true) logInfo("Deleted checkpoint file '" + file + "' for time " + time) } catch { @@ -64,15 +72,16 @@ class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) } /** - * Restore the checkpoint data. Default implementation restores the RDDs from their - * checkpoint files. + * Restore the checkpoint data. This gets called once when the DStream graph + * (along with its DStreams) are being restored from a graph checkpoint file. + * Default implementation restores the RDDs from their checkpoint files. */ def restore() { // Create RDDs from the checkpoint data checkpointFiles.foreach { case(time, file) => { logInfo("Restoring checkpointed RDD for time " + time + " from file '" + file + "'") - dstream.generatedRDDs += ((time, dstream.context.sc.checkpointFile[T](file))) + dstream.generatedRDDs += ((time, dstream.context.sparkContext.checkpointFile[T](file))) } } } diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index bc4a40d7bccd4..d5a5496839407 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -87,7 +87,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { private[streaming] def forgetOldRDDs(time: Time) { this.synchronized { - outputStreams.foreach(_.forgetOldRDDs(time)) + outputStreams.foreach(_.forgetOldMetadata(time)) } } diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 14500bdcb17a0..2cf00e3baab88 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -61,7 +61,7 @@ class StreamingContext private ( protected[streaming] val isCheckpointPresent = (cp_ != null) - val sc: SparkContext = { + protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars) } else { @@ -100,6 +100,11 @@ class StreamingContext private ( protected[streaming] var receiverJobThread: Thread = null protected[streaming] var scheduler: Scheduler = null + /** + * Returns the associated Spark context + */ + def sparkContext = sc + /** * Sets each DStreams in this context to remember RDDs it generated in the last given duration. * DStreams remember RDDs only for a limited duration of time and releases them for garbage diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index 1e6ad84b44014..c6ffb252ce887 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -2,13 +2,14 @@ package spark.streaming.dstream import spark.RDD import spark.rdd.UnionRDD -import spark.streaming.{StreamingContext, Time} +import spark.streaming.{DStreamCheckpointData, StreamingContext, Time} import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import scala.collection.mutable.HashSet +import scala.collection.mutable.{HashSet, HashMap} +import java.io.{ObjectInputStream, IOException} private[streaming] class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest]( @@ -18,21 +19,14 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K newFilesOnly: Boolean = true) extends InputDStream[(K, V)](ssc_) { - @transient private var path_ : Path = null - @transient private var fs_ : FileSystem = null - - var lastModTime = 0L - val lastModTimeFiles = new HashSet[String]() + protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData - def path(): Path = { - if (path_ == null) path_ = new Path(directory) - path_ - } + private val lastModTimeFiles = new HashSet[String]() + private var lastModTime = 0L - def fs(): FileSystem = { - if (fs_ == null) fs_ = path.getFileSystem(new Configuration()) - fs_ - } + @transient private var path_ : Path = null + @transient private var fs_ : FileSystem = null + @transient private var files = new HashMap[Time, Array[String]] override def start() { if (newFilesOnly) { @@ -79,8 +73,8 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K } } - val newFiles = fs.listStatus(path, newFilter) - logInfo("New files: " + newFiles.map(_.getPath).mkString(", ")) + val newFiles = fs.listStatus(path, newFilter).map(_.getPath.toString) + logInfo("New files: " + newFiles.mkString(", ")) if (newFiles.length > 0) { // Update the modification time and the files processed for that modification time if (lastModTime != newFilter.latestModTime) { @@ -89,9 +83,70 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K } lastModTimeFiles ++= newFilter.latestModTimeFiles } - val newRDD = new UnionRDD(ssc.sc, newFiles.map( - file => ssc.sc.newAPIHadoopFile[K, V, F](file.getPath.toString))) - Some(newRDD) + files += ((validTime, newFiles)) + Some(filesToRDD(newFiles)) + } + + /** Forget the old time-to-files mappings along with old RDDs */ + protected[streaming] override def forgetOldMetadata(time: Time) { + super.forgetOldMetadata(time) + val filesToBeRemoved = files.filter(_._1 <= (time - rememberDuration)) + files --= filesToBeRemoved.keys + logInfo("Forgot " + filesToBeRemoved.size + " files from " + this) + } + + /** Generate one RDD from an array of files */ + protected[streaming] def filesToRDD(files: Seq[String]): RDD[(K, V)] = { + new UnionRDD( + context.sparkContext, + files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file)) + ) + } + + private def path: Path = { + if (path_ == null) path_ = new Path(directory) + path_ + } + + private def fs: FileSystem = { + if (fs_ == null) fs_ = path.getFileSystem(new Configuration()) + fs_ + } + + @throws(classOf[IOException]) + private def readObject(ois: ObjectInputStream) { + logDebug(this.getClass().getSimpleName + ".readObject used") + ois.defaultReadObject() + generatedRDDs = new HashMap[Time, RDD[(K,V)]] () + files = new HashMap[Time, Array[String]] + } + + /** + * A custom version of the DStreamCheckpointData that stores names of + * Hadoop files as checkpoint data. + */ + private[streaming] + class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { + + def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]] + + override def update() { + hadoopFiles.clear() + hadoopFiles ++= files + } + + override def cleanup() { } + + override def restore() { + hadoopFiles.foreach { + case (time, files) => { + logInfo("Restoring Hadoop RDD for time " + time + " from files " + + files.mkString("[", ",", "]") ) + files + generatedRDDs += ((time, filesToRDD(files))) + } + } + } } } @@ -100,3 +155,4 @@ object FileInputDStream { def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") } + diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index d7ba7a5d17da8..4f6204f2058b4 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -214,10 +214,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { //Thread.sleep(100) } val startTime = System.currentTimeMillis() - /*while (output.size < expectedOutput.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { - logInfo("output.size = " + output.size + ", expectedOutput.size = " + expectedOutput.size) - Thread.sleep(100) - }*/ Thread.sleep(1000) val timeTaken = System.currentTimeMillis() - startTime assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") @@ -226,11 +222,9 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Verify whether data received by Spark Streaming was as expected logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) - logInfo("output") + logInfo("output, size = " + outputBuffer.size) outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output.size = " + expectedOutput.size) - logInfo("expected output") + logInfo("expected output, size = " + expectedOutput.size) expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) logInfo("--------------------------------") @@ -256,8 +250,13 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams var ssc = new StreamingContext(master, framework, batchDuration) ssc.checkpoint(checkpointDir, checkpointInterval) - val filestream = ssc.textFileStream(testDir.toString) - var outputStream = new TestOutputStream(filestream, new ArrayBuffer[Seq[String]]) + val fileStream = ssc.textFileStream(testDir.toString) + val outputBuffer = new ArrayBuffer[Seq[Int]] + // Reduced over a large window to ensure that recovery from master failure + // requires reprocessing of all the files seen before the failure + val reducedStream = fileStream.map(_.toInt) + .reduceByWindow(_ + _, batchDuration * 30, batchDuration) + var outputStream = new TestOutputStream(reducedStream, outputBuffer) ssc.registerOutputStream(outputStream) ssc.start() @@ -266,31 +265,56 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { Thread.sleep(1000) for (i <- Seq(1, 2, 3)) { FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") - Thread.sleep(100) + // wait to make sure that the file is written such that it gets shown in the file listings + Thread.sleep(500) clock.addToTime(batchDuration.milliseconds) + // wait to make sure that FileInputDStream picks up this file only and not any other file + Thread.sleep(500) } - Thread.sleep(500) logInfo("Output = " + outputStream.output.mkString(",")) - assert(outputStream.output.size > 0) + assert(outputStream.output.size > 0, "No files processed before restart") ssc.stop() + for (i <- Seq(4, 5, 6)) { + FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") + Thread.sleep(1000) + } + // Restart stream computation from checkpoint and create more files to see whether // they are being processed logInfo("*********** RESTARTING ************") ssc = new StreamingContext(checkpointDir) ssc.start() clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - Thread.sleep(500) - for (i <- Seq(4, 5, 6)) { + for (i <- Seq(7, 8, 9)) { FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") - Thread.sleep(100) + Thread.sleep(500) clock.addToTime(batchDuration.milliseconds) + Thread.sleep(500) } - Thread.sleep(500) - outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[String]] - logInfo("Output = " + outputStream.output.mkString(",")) - assert(outputStream.output.size > 0) + Thread.sleep(1000) + assert(outputStream.output.size > 0, "No files processed after restart") ssc.stop() + + // Append the new output to the old buffer + outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] + outputBuffer ++= outputStream.output + + // Verify whether data received by Spark Streaming was as expected + val expectedOutput = Seq(1, 3, 6, 28, 36, 45) + logInfo("--------------------------------") + logInfo("output, size = " + outputBuffer.size) + outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("expected output, size = " + expectedOutput.size) + expectedOutput.foreach(x => logInfo("[" + x + "]")) + logInfo("--------------------------------") + + // Verify whether all the elements received are as expected + assert(outputBuffer.size === expectedOutput.size) + for (i <- 0 until outputBuffer.size) { + assert(outputBuffer(i).size === 1) + assert(outputBuffer(i).head === expectedOutput(i)) + } } } From 325297e5c31418f32deeb2a3cc52755094a11cea Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Tue, 22 Jan 2013 17:31:11 -0800 Subject: [PATCH 035/696] Add an Avro dependency to REPL to make it compile with Hadoop 2 --- pom.xml | 11 +++++++++++ repl/pom.xml | 10 ++++++++++ 2 files changed, 21 insertions(+) diff --git a/pom.xml b/pom.xml index 483b0f9595dd9..3ea989a082b2a 100644 --- a/pom.xml +++ b/pom.xml @@ -542,6 +542,17 @@ hadoop-client 2.0.0-mr1-cdh${cdh.version} + + + org.apache.avro + avro + 1.7.1.cloudera.2 + + + org.apache.avro + avro-ipc + 1.7.1.cloudera.2 + diff --git a/repl/pom.xml b/repl/pom.xml index 2fc96929698c6..2dc96beaf55fe 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -175,6 +175,16 @@ hadoop-client provided + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + From 284993100022cc4bd43bf84a0be4dd91cf7a4ac0 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 22 Jan 2013 22:19:30 -0800 Subject: [PATCH 036/696] Eliminate CacheTracker. Replaces DAGScheduler's queries of CacheTracker with BlockManagerMaster queries. Adds CacheManager to locally coordinate computation of cached RDDs. --- core/src/main/scala/spark/CacheTracker.scala | 240 ------------------ core/src/main/scala/spark/RDD.scala | 2 +- core/src/main/scala/spark/SparkEnv.scala | 8 +- .../scala/spark/scheduler/DAGScheduler.scala | 24 +- .../scala/spark/storage/BlockManager.scala | 24 +- .../test/scala/spark/CacheTrackerSuite.scala | 131 ---------- 6 files changed, 18 insertions(+), 411 deletions(-) delete mode 100644 core/src/main/scala/spark/CacheTracker.scala delete mode 100644 core/src/test/scala/spark/CacheTrackerSuite.scala diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala deleted file mode 100644 index 86ad737583779..0000000000000 --- a/core/src/main/scala/spark/CacheTracker.scala +++ /dev/null @@ -1,240 +0,0 @@ -package spark - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet - -import akka.actor._ -import akka.dispatch._ -import akka.pattern.ask -import akka.remote._ -import akka.util.Duration -import akka.util.Timeout -import akka.util.duration._ - -import spark.storage.BlockManager -import spark.storage.StorageLevel -import util.{TimeStampedHashSet, MetadataCleaner, TimeStampedHashMap} - -private[spark] sealed trait CacheTrackerMessage - -private[spark] case class AddedToCache(rddId: Int, partition: Int, host: String, size: Long = 0L) - extends CacheTrackerMessage -private[spark] case class DroppedFromCache(rddId: Int, partition: Int, host: String, size: Long = 0L) - extends CacheTrackerMessage -private[spark] case class MemoryCacheLost(host: String) extends CacheTrackerMessage -private[spark] case class RegisterRDD(rddId: Int, numPartitions: Int) extends CacheTrackerMessage -private[spark] case class SlaveCacheStarted(host: String, size: Long) extends CacheTrackerMessage -private[spark] case object GetCacheStatus extends CacheTrackerMessage -private[spark] case object GetCacheLocations extends CacheTrackerMessage -private[spark] case object StopCacheTracker extends CacheTrackerMessage - -private[spark] class CacheTrackerActor extends Actor with Logging { - // TODO: Should probably store (String, CacheType) tuples - private val locs = new TimeStampedHashMap[Int, Array[List[String]]] - - /** - * A map from the slave's host name to its cache size. - */ - private val slaveCapacity = new HashMap[String, Long] - private val slaveUsage = new HashMap[String, Long] - - private val metadataCleaner = new MetadataCleaner("CacheTrackerActor", locs.clearOldValues) - - private def getCacheUsage(host: String): Long = slaveUsage.getOrElse(host, 0L) - private def getCacheCapacity(host: String): Long = slaveCapacity.getOrElse(host, 0L) - private def getCacheAvailable(host: String): Long = getCacheCapacity(host) - getCacheUsage(host) - - def receive = { - case SlaveCacheStarted(host: String, size: Long) => - slaveCapacity.put(host, size) - slaveUsage.put(host, 0) - sender ! true - - case RegisterRDD(rddId: Int, numPartitions: Int) => - logInfo("Registering RDD " + rddId + " with " + numPartitions + " partitions") - locs(rddId) = Array.fill[List[String]](numPartitions)(Nil) - sender ! true - - case AddedToCache(rddId, partition, host, size) => - slaveUsage.put(host, getCacheUsage(host) + size) - locs(rddId)(partition) = host :: locs(rddId)(partition) - sender ! true - - case DroppedFromCache(rddId, partition, host, size) => - slaveUsage.put(host, getCacheUsage(host) - size) - // Do a sanity check to make sure usage is greater than 0. - locs(rddId)(partition) = locs(rddId)(partition).filterNot(_ == host) - sender ! true - - case MemoryCacheLost(host) => - logInfo("Memory cache lost on " + host) - for ((id, locations) <- locs) { - for (i <- 0 until locations.length) { - locations(i) = locations(i).filterNot(_ == host) - } - } - sender ! true - - case GetCacheLocations => - logInfo("Asked for current cache locations") - sender ! locs.map{case (rrdId, array) => (rrdId -> array.clone())} - - case GetCacheStatus => - val status = slaveCapacity.map { case (host, capacity) => - (host, capacity, getCacheUsage(host)) - }.toSeq - sender ! status - - case StopCacheTracker => - logInfo("Stopping CacheTrackerActor") - sender ! true - metadataCleaner.cancel() - context.stop(self) - } -} - -private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: BlockManager) - extends Logging { - - // Tracker actor on the master, or remote reference to it on workers - val ip: String = System.getProperty("spark.master.host", "localhost") - val port: Int = System.getProperty("spark.master.port", "7077").toInt - val actorName: String = "CacheTracker" - - val timeout = 10.seconds - - var trackerActor: ActorRef = if (isMaster) { - val actor = actorSystem.actorOf(Props[CacheTrackerActor], name = actorName) - logInfo("Registered CacheTrackerActor actor") - actor - } else { - val url = "akka://spark@%s:%s/user/%s".format(ip, port, actorName) - actorSystem.actorFor(url) - } - - // TODO: Consider removing this HashSet completely as locs CacheTrackerActor already - // keeps track of registered RDDs - val registeredRddIds = new TimeStampedHashSet[Int] - - // Remembers which splits are currently being loaded (on worker nodes) - val loading = new HashSet[String] - - val metadataCleaner = new MetadataCleaner("CacheTracker", registeredRddIds.clearOldValues) - - // Send a message to the trackerActor and get its result within a default timeout, or - // throw a SparkException if this fails. - def askTracker(message: Any): Any = { - try { - val future = trackerActor.ask(message)(timeout) - return Await.result(future, timeout) - } catch { - case e: Exception => - throw new SparkException("Error communicating with CacheTracker", e) - } - } - - // Send a one-way message to the trackerActor, to which we expect it to reply with true. - def communicate(message: Any) { - if (askTracker(message) != true) { - throw new SparkException("Error reply received from CacheTracker") - } - } - - // Registers an RDD (on master only) - def registerRDD(rddId: Int, numPartitions: Int) { - registeredRddIds.synchronized { - if (!registeredRddIds.contains(rddId)) { - logInfo("Registering RDD ID " + rddId + " with cache") - registeredRddIds += rddId - communicate(RegisterRDD(rddId, numPartitions)) - } - } - } - - // For BlockManager.scala only - def cacheLost(host: String) { - communicate(MemoryCacheLost(host)) - logInfo("CacheTracker successfully removed entries on " + host) - } - - // Get the usage status of slave caches. Each tuple in the returned sequence - // is in the form of (host name, capacity, usage). - def getCacheStatus(): Seq[(String, Long, Long)] = { - askTracker(GetCacheStatus).asInstanceOf[Seq[(String, Long, Long)]] - } - - // For BlockManager.scala only - def notifyFromBlockManager(t: AddedToCache) { - communicate(t) - } - - // Get a snapshot of the currently known locations - def getLocationsSnapshot(): HashMap[Int, Array[List[String]]] = { - askTracker(GetCacheLocations).asInstanceOf[HashMap[Int, Array[List[String]]]] - } - - // Gets or computes an RDD split - def getOrCompute[T](rdd: RDD[T], split: Split, context: TaskContext, storageLevel: StorageLevel) - : Iterator[T] = { - val key = "rdd_%d_%d".format(rdd.id, split.index) - logInfo("Cache key is " + key) - blockManager.get(key) match { - case Some(cachedValues) => - // Split is in cache, so just return its values - logInfo("Found partition in cache!") - return cachedValues.asInstanceOf[Iterator[T]] - - case None => - // Mark the split as loading (unless someone else marks it first) - loading.synchronized { - if (loading.contains(key)) { - logInfo("Loading contains " + key + ", waiting...") - while (loading.contains(key)) { - try {loading.wait()} catch {case _ =>} - } - logInfo("Loading no longer contains " + key + ", so returning cached result") - // See whether someone else has successfully loaded it. The main way this would fail - // is for the RDD-level cache eviction policy if someone else has loaded the same RDD - // partition but we didn't want to make space for it. However, that case is unlikely - // because it's unlikely that two threads would work on the same RDD partition. One - // downside of the current code is that threads wait serially if this does happen. - blockManager.get(key) match { - case Some(values) => - return values.asInstanceOf[Iterator[T]] - case None => - logInfo("Whoever was loading " + key + " failed; we'll try it ourselves") - loading.add(key) - } - } else { - loading.add(key) - } - } - try { - // If we got here, we have to load the split - val elements = new ArrayBuffer[Any] - logInfo("Computing partition " + split) - elements ++= rdd.compute(split, context) - // Try to put this block in the blockManager - blockManager.put(key, elements, storageLevel, true) - return elements.iterator.asInstanceOf[Iterator[T]] - } finally { - loading.synchronized { - loading.remove(key) - loading.notifyAll() - } - } - } - } - - // Called by the Cache to report that an entry has been dropped from it - def dropEntry(rddId: Int, partition: Int) { - communicate(DroppedFromCache(rddId, partition, Utils.localHostName())) - } - - def stop() { - communicate(StopCacheTracker) - registeredRddIds.clear() - trackerActor = null - } -} diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e0d2eabb1dd65..c79f34342f66c 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -176,7 +176,7 @@ abstract class RDD[T: ClassManifest]( if (isCheckpointed) { checkpointData.get.iterator(split, context) } else if (storageLevel != StorageLevel.NONE) { - SparkEnv.get.cacheTracker.getOrCompute[T](this, split, context, storageLevel) + SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel) } else { compute(split, context) } diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 41441720a7c8f..a080194980df9 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -22,7 +22,7 @@ class SparkEnv ( val actorSystem: ActorSystem, val serializer: Serializer, val closureSerializer: Serializer, - val cacheTracker: CacheTracker, + val cacheManager: CacheManager, val mapOutputTracker: MapOutputTracker, val shuffleFetcher: ShuffleFetcher, val broadcastManager: BroadcastManager, @@ -39,7 +39,6 @@ class SparkEnv ( def stop() { httpFileServer.stop() mapOutputTracker.stop() - cacheTracker.stop() shuffleFetcher.stop() broadcastManager.stop() blockManager.stop() @@ -100,8 +99,7 @@ object SparkEnv extends Logging { val closureSerializer = instantiateClass[Serializer]( "spark.closure.serializer", "spark.JavaSerializer") - val cacheTracker = new CacheTracker(actorSystem, isMaster, blockManager) - blockManager.cacheTracker = cacheTracker + val cacheManager = new CacheManager(blockManager) val mapOutputTracker = new MapOutputTracker(actorSystem, isMaster) @@ -122,7 +120,7 @@ object SparkEnv extends Logging { actorSystem, serializer, closureSerializer, - cacheTracker, + cacheManager, mapOutputTracker, shuffleFetcher, broadcastManager, diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 59f2099e911a5..03d173ac3be46 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -69,8 +69,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with var cacheLocs = new HashMap[Int, Array[List[String]]] val env = SparkEnv.get - val cacheTracker = env.cacheTracker val mapOutputTracker = env.mapOutputTracker + val blockManagerMaster = env.blockManager.master val deadHosts = new HashSet[String] // TODO: The code currently assumes these can't come back; // that's not going to be a realistic assumption in general @@ -95,11 +95,17 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with }.start() def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { + if (!cacheLocs.contains(rdd.id)) { + val blockIds = rdd.splits.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray + cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map { + locations => locations.map(_.ip).toList + }.toArray + } cacheLocs(rdd.id) } - def updateCacheLocs() { - cacheLocs = cacheTracker.getLocationsSnapshot() + def clearCacheLocs() { + cacheLocs.clear } /** @@ -126,7 +132,6 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of splits is unknown logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") - cacheTracker.registerRDD(rdd.id, rdd.splits.size) if (shuffleDep != None) { mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.splits.size) } @@ -148,8 +153,6 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with visited += r // Kind of ugly: need to register RDDs with the cache here since // we can't do it in its constructor because # of splits is unknown - logInfo("Registering parent RDD " + r.id + " (" + r.origin + ")") - cacheTracker.registerRDD(r.id, r.splits.size) for (dep <- r.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => @@ -250,7 +253,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val runId = nextRunId.getAndIncrement() val finalStage = newStage(finalRDD, None, runId) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener) - updateCacheLocs() + clearCacheLocs() logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions") logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") @@ -293,7 +296,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with // on the failed node. if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) { logInfo("Resubmitting failed stages") - updateCacheLocs() + clearCacheLocs() val failed2 = failed.toArray failed.clear() for (stage <- failed2.sortBy(_.priority)) { @@ -443,7 +446,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) } - updateCacheLocs() + clearCacheLocs() if (stage.outputLocs.count(_ == Nil) != 0) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this @@ -519,8 +522,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray mapOutputTracker.registerMapOutputs(shuffleId, locs, true) } - cacheTracker.cacheLost(host) - updateCacheLocs() + clearCacheLocs() } } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 7a8ac10cdd88e..e049565f48aad 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -16,7 +16,7 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.{CacheTracker, Logging, SizeEstimator, SparkEnv, SparkException, Utils} +import spark.{Logging, SizeEstimator, SparkEnv, SparkException, Utils} import spark.network._ import spark.serializer.Serializer import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} @@ -71,9 +71,6 @@ class BlockManager( val connectionManagerId = connectionManager.id val blockManagerId = new BlockManagerId(connectionManagerId.host, connectionManagerId.port) - // TODO: This will be removed after cacheTracker is removed from the code base. - var cacheTracker: CacheTracker = null - // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = @@ -662,10 +659,6 @@ class BlockManager( BlockManager.dispose(bytesAfterPut) - // TODO: This code will be removed when CacheTracker is gone. - if (blockId.startsWith("rdd")) { - notifyCacheTracker(blockId) - } logDebug("Put block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)) return size @@ -733,11 +726,6 @@ class BlockManager( } } - // TODO: This code will be removed when CacheTracker is gone. - if (blockId.startsWith("rdd")) { - notifyCacheTracker(blockId) - } - // If replication had started, then wait for it to finish if (level.replication > 1) { if (replicationFuture == null) { @@ -780,16 +768,6 @@ class BlockManager( } } - // TODO: This code will be removed when CacheTracker is gone. - private def notifyCacheTracker(key: String) { - if (cacheTracker != null) { - val rddInfo = key.split("_") - val rddId: Int = rddInfo(1).toInt - val partition: Int = rddInfo(2).toInt - cacheTracker.notifyFromBlockManager(spark.AddedToCache(rddId, partition, host)) - } - } - /** * Read a block consisting of a single object. */ diff --git a/core/src/test/scala/spark/CacheTrackerSuite.scala b/core/src/test/scala/spark/CacheTrackerSuite.scala deleted file mode 100644 index 467605981b3d9..0000000000000 --- a/core/src/test/scala/spark/CacheTrackerSuite.scala +++ /dev/null @@ -1,131 +0,0 @@ -package spark - -import org.scalatest.FunSuite - -import scala.collection.mutable.HashMap - -import akka.actor._ -import akka.dispatch._ -import akka.pattern.ask -import akka.remote._ -import akka.util.Duration -import akka.util.Timeout -import akka.util.duration._ - -class CacheTrackerSuite extends FunSuite { - // Send a message to an actor and wait for a reply, in a blocking manner - private def ask(actor: ActorRef, message: Any): Any = { - try { - val timeout = 10.seconds - val future = actor.ask(message)(timeout) - return Await.result(future, timeout) - } catch { - case e: Exception => - throw new SparkException("Error communicating with actor", e) - } - } - - test("CacheTrackerActor slave initialization & cache status") { - //System.setProperty("spark.master.port", "1345") - val initialSize = 2L << 20 - - val actorSystem = ActorSystem("test") - val tracker = actorSystem.actorOf(Props[CacheTrackerActor]) - - assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true) - - assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 0L))) - - assert(ask(tracker, StopCacheTracker) === true) - - actorSystem.shutdown() - actorSystem.awaitTermination() - } - - test("RegisterRDD") { - //System.setProperty("spark.master.port", "1345") - val initialSize = 2L << 20 - - val actorSystem = ActorSystem("test") - val tracker = actorSystem.actorOf(Props[CacheTrackerActor]) - - assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true) - - assert(ask(tracker, RegisterRDD(1, 3)) === true) - assert(ask(tracker, RegisterRDD(2, 1)) === true) - - assert(getCacheLocations(tracker) === Map(1 -> List(Nil, Nil, Nil), 2 -> List(Nil))) - - assert(ask(tracker, StopCacheTracker) === true) - - actorSystem.shutdown() - actorSystem.awaitTermination() - } - - test("AddedToCache") { - //System.setProperty("spark.master.port", "1345") - val initialSize = 2L << 20 - - val actorSystem = ActorSystem("test") - val tracker = actorSystem.actorOf(Props[CacheTrackerActor]) - - assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true) - - assert(ask(tracker, RegisterRDD(1, 2)) === true) - assert(ask(tracker, RegisterRDD(2, 1)) === true) - - assert(ask(tracker, AddedToCache(1, 0, "host001", 2L << 15)) === true) - assert(ask(tracker, AddedToCache(1, 1, "host001", 2L << 11)) === true) - assert(ask(tracker, AddedToCache(2, 0, "host001", 3L << 10)) === true) - - assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 72704L))) - - assert(getCacheLocations(tracker) === - Map(1 -> List(List("host001"), List("host001")), 2 -> List(List("host001")))) - - assert(ask(tracker, StopCacheTracker) === true) - - actorSystem.shutdown() - actorSystem.awaitTermination() - } - - test("DroppedFromCache") { - //System.setProperty("spark.master.port", "1345") - val initialSize = 2L << 20 - - val actorSystem = ActorSystem("test") - val tracker = actorSystem.actorOf(Props[CacheTrackerActor]) - - assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true) - - assert(ask(tracker, RegisterRDD(1, 2)) === true) - assert(ask(tracker, RegisterRDD(2, 1)) === true) - - assert(ask(tracker, AddedToCache(1, 0, "host001", 2L << 15)) === true) - assert(ask(tracker, AddedToCache(1, 1, "host001", 2L << 11)) === true) - assert(ask(tracker, AddedToCache(2, 0, "host001", 3L << 10)) === true) - - assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 72704L))) - assert(getCacheLocations(tracker) === - Map(1 -> List(List("host001"), List("host001")), 2 -> List(List("host001")))) - - assert(ask(tracker, DroppedFromCache(1, 1, "host001", 2L << 11)) === true) - - assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 68608L))) - assert(getCacheLocations(tracker) === - Map(1 -> List(List("host001"),List()), 2 -> List(List("host001")))) - - assert(ask(tracker, StopCacheTracker) === true) - - actorSystem.shutdown() - actorSystem.awaitTermination() - } - - /** - * Helper function to get cacheLocations from CacheTracker - */ - def getCacheLocations(tracker: ActorRef): HashMap[Int, List[List[String]]] = { - val answer = ask(tracker, GetCacheLocations).asInstanceOf[HashMap[Int, Array[List[String]]]] - answer.map { case (i, arr) => (i, arr.toList) } - } -} From 43e9ff959645e533bcfa0a5c31e62e32c7e9d0a6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 22 Jan 2013 22:47:26 -0800 Subject: [PATCH 037/696] Add test for driver hanging on exit (SPARK-530). --- core/src/test/scala/spark/DriverSuite.scala | 31 +++++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 core/src/test/scala/spark/DriverSuite.scala diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala new file mode 100644 index 0000000000000..70a7c8bc2f528 --- /dev/null +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -0,0 +1,31 @@ +package spark + +import java.io.File + +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts +import org.scalatest.prop.TableDrivenPropertyChecks._ +import org.scalatest.time.SpanSugar._ + +class DriverSuite extends FunSuite with Timeouts { + test("driver should exit after finishing") { + // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" + val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) + forAll(masters) { (master: String) => + failAfter(10 seconds) { + Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) + } + } + } +} + +/** + * Program that creates a Spark driver but doesn't call SparkContext.stop() or + * Sys.exit() after finishing. + */ +object DriverWithoutCleanup { + def main(args: Array[String]) { + val sc = new SparkContext(args(0), "DriverWithoutCleanup") + sc.parallelize(1 to 100, 4).count() + } +} \ No newline at end of file From bacade6caf7527737dc6f02b1c2ca9114e02d8bc Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 22 Jan 2013 22:55:26 -0800 Subject: [PATCH 038/696] Modified BlockManagerId API to ensure zero duplicate objects. Fixed BlockManagerId testcase in BlockManagerTestSuite. --- .../scala/spark/scheduler/MapStatus.scala | 2 +- .../scala/spark/storage/BlockManager.scala | 2 +- .../scala/spark/storage/BlockManagerId.scala | 33 +++++++++++++++---- .../spark/storage/BlockManagerMessages.scala | 3 +- .../scala/spark/MapOutputTrackerSuite.scala | 22 ++++++------- .../spark/storage/BlockManagerSuite.scala | 18 +++++----- 6 files changed, 51 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala index 4532d9497f303..fae643f3a8247 100644 --- a/core/src/main/scala/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/spark/scheduler/MapStatus.scala @@ -20,7 +20,7 @@ private[spark] class MapStatus(var address: BlockManagerId, var compressedSizes: } def readExternal(in: ObjectInput) { - address = new BlockManagerId(in) + address = BlockManagerId(in) compressedSizes = new Array[Byte](in.readInt()) in.readFully(compressedSizes) } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 7a8ac10cdd88e..596a69c583ecf 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -69,7 +69,7 @@ class BlockManager( implicit val futureExecContext = connectionManager.futureExecContext val connectionManagerId = connectionManager.id - val blockManagerId = new BlockManagerId(connectionManagerId.host, connectionManagerId.port) + val blockManagerId = BlockManagerId(connectionManagerId.host, connectionManagerId.port) // TODO: This will be removed after cacheTracker is removed from the code base. var cacheTracker: CacheTracker = null diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index 488679f0496b4..26c98f2ac8e76 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -3,20 +3,35 @@ package spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap +/** + * This class represent an unique identifier for a BlockManager. + * The first 2 constructors of this class is made private to ensure that + * BlockManagerId objects can be created only using the factory method in + * [[spark.storage.BlockManager$]]. This allows de-duplication of id objects. + * Also, constructor parameters are private to ensure that parameters cannot + * be modified from outside this class. + */ +private[spark] class BlockManagerId private ( + private var ip_ : String, + private var port_ : Int + ) extends Externalizable { + + private def this(in: ObjectInput) = this(in.readUTF(), in.readInt()) -private[spark] class BlockManagerId(var ip: String, var port: Int) extends Externalizable { def this() = this(null, 0) // For deserialization only - def this(in: ObjectInput) = this(in.readUTF(), in.readInt()) + def ip = ip_ + + def port = port_ override def writeExternal(out: ObjectOutput) { - out.writeUTF(ip) - out.writeInt(port) + out.writeUTF(ip_) + out.writeInt(port_) } override def readExternal(in: ObjectInput) { - ip = in.readUTF() - port = in.readInt() + ip_ = in.readUTF() + port_ = in.readInt() } @throws(classOf[IOException]) @@ -35,6 +50,12 @@ private[spark] class BlockManagerId(var ip: String, var port: Int) extends Exter private[spark] object BlockManagerId { + def apply(ip: String, port: Int) = + getCachedBlockManagerId(new BlockManagerId(ip, port)) + + def apply(in: ObjectInput) = + getCachedBlockManagerId(new BlockManagerId(in)) + val blockManagerIdCache = new ConcurrentHashMap[BlockManagerId, BlockManagerId]() def getCachedBlockManagerId(id: BlockManagerId): BlockManagerId = { diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index d73a9b790f479..7437fc63ebf79 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -54,8 +54,7 @@ class UpdateBlockInfo( } override def readExternal(in: ObjectInput) { - blockManagerId = new BlockManagerId() - blockManagerId.readExternal(in) + blockManagerId = BlockManagerId(in) blockId = in.readUTF() storageLevel = new StorageLevel() storageLevel.readExternal(in) diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index d3dd3a8fa4930..095f415978df3 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -47,13 +47,13 @@ class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(new BlockManagerId("hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("hostA", 1000), Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(new BlockManagerId("hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("hostB", 1000), Array(compressedSize10000, compressedSize1000))) val statuses = tracker.getServerStatuses(10, 0) - assert(statuses.toSeq === Seq((new BlockManagerId("hostA", 1000), size1000), - (new BlockManagerId("hostB", 1000), size10000))) + assert(statuses.toSeq === Seq((BlockManagerId("hostA", 1000), size1000), + (BlockManagerId("hostB", 1000), size10000))) tracker.stop() } @@ -65,14 +65,14 @@ class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(new BlockManagerId("hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("hostA", 1000), Array(compressedSize1000, compressedSize1000, compressedSize1000))) - tracker.registerMapOutput(10, 1, new MapStatus(new BlockManagerId("hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("hostB", 1000), Array(compressedSize10000, compressedSize1000, compressedSize1000))) // As if we had two simulatenous fetch failures - tracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000)) - tracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("hostA", 1000)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("hostA", 1000)) // The remaining reduce task might try to grab the output dispite the shuffle failure; // this should cause it to fail, and the scheduler will ignore the failure due to the @@ -95,13 +95,13 @@ class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter { val compressedSize1000 = MapOutputTracker.compressSize(1000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) masterTracker.registerMapOutput(10, 0, new MapStatus( - new BlockManagerId("hostA", 1000), Array(compressedSize1000))) + BlockManagerId("hostA", 1000), Array(compressedSize1000))) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((new BlockManagerId("hostA", 1000), size1000))) + Seq((BlockManagerId("hostA", 1000), size1000))) - masterTracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000)) + masterTracker.unregisterMapOutput(10, 0, BlockManagerId("hostA", 1000)) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 8f86e3170ed0f..a33d3324ba617 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -82,16 +82,18 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("BlockManagerId object caching") { - val id1 = new StorageLevel(false, false, false, 3) - val id2 = new StorageLevel(false, false, false, 3) + val id1 = BlockManagerId("XXX", 1) + val id2 = BlockManagerId("XXX", 1) // this should return the same object as id1 + assert(id2 === id1, "id2 is not same as id1") + assert(id2.eq(id1), "id2 is not the same object as id1") val bytes1 = spark.Utils.serialize(id1) - val id1_ = spark.Utils.deserialize[StorageLevel](bytes1) + val id1_ = spark.Utils.deserialize[BlockManagerId](bytes1) val bytes2 = spark.Utils.serialize(id2) - val id2_ = spark.Utils.deserialize[StorageLevel](bytes2) - assert(id1_ === id1, "Deserialized id1 not same as original id1") - assert(id2_ === id2, "Deserialized id2 not same as original id1") - assert(id1_ === id2_, "Deserialized id1 not same as deserialized id2") - assert(id2_.eq(id1_), "Deserialized id2 not the same object as deserialized level1") + val id2_ = spark.Utils.deserialize[BlockManagerId](bytes2) + assert(id1_ === id1, "Deserialized id1 is not same as original id1") + assert(id1_.eq(id1), "Deserialized id1 is not the same object as original id1") + assert(id2_ === id2, "Deserialized id2 is not same as original id2") + assert(id2_.eq(id1), "Deserialized id2 is not the same object as original id1") } test("master + 1 manager interaction") { From 5e11f1e51f17113abb8d3a5bc261af5ba5ffce94 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 22 Jan 2013 23:42:53 -0800 Subject: [PATCH 039/696] Modified StorageLevel API to ensure zero duplicate objects. --- .../scala/spark/storage/BlockManager.scala | 5 +- .../scala/spark/storage/BlockMessage.scala | 2 +- .../scala/spark/storage/StorageLevel.scala | 47 ++++++++++++------- .../spark/storage/BlockManagerSuite.scala | 16 +++++-- 4 files changed, 44 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 596a69c583ecf..ca7eb13ec8af8 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -191,7 +191,7 @@ class BlockManager( case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) - val storageLevel = new StorageLevel(onDisk, inMem, level.deserialized, level.replication) + val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication) val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L (storageLevel, memSize, diskSize, info.tellMaster) @@ -760,8 +760,7 @@ class BlockManager( */ var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: String, data: ByteBuffer, level: StorageLevel) { - val tLevel: StorageLevel = - new StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1) + val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1) if (cachedPeers == null) { cachedPeers = master.getPeers(blockManagerId, level.replication - 1) } diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index 3f234df654c34..30d7500e01e68 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -64,7 +64,7 @@ private[spark] class BlockMessage() { val booleanInt = buffer.getInt() val replication = buffer.getInt() - level = new StorageLevel(booleanInt, replication) + level = StorageLevel(booleanInt, replication) val dataLength = buffer.getInt() data = ByteBuffer.allocate(dataLength) diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index e3544e5aae28c..f2535ae5aebc2 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -7,25 +7,30 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory * in a serialized format, and whether to replicate the RDD partitions on multiple nodes. * The [[spark.storage.StorageLevel$]] singleton object contains some static constants for - * commonly useful storage levels. + * commonly useful storage levels. The recommended method to create your own storage level + * object is to use `StorageLevel.apply(...)` from the singleton object. */ class StorageLevel( - var useDisk: Boolean, - var useMemory: Boolean, - var deserialized: Boolean, - var replication: Int = 1) + private var useDisk_ : Boolean, + private var useMemory_ : Boolean, + private var deserialized_ : Boolean, + private var replication_ : Int = 1) extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. - - assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") - - def this(flags: Int, replication: Int) { + private def this(flags: Int, replication: Int) { this((flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication) } def this() = this(false, true, false) // For deserialization + def useDisk = useDisk_ + def useMemory = useMemory_ + def deserialized = deserialized_ + def replication = replication_ + + assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") + override def clone(): StorageLevel = new StorageLevel( this.useDisk, this.useMemory, this.deserialized, this.replication) @@ -43,13 +48,13 @@ class StorageLevel( def toInt: Int = { var ret = 0 - if (useDisk) { + if (useDisk_) { ret |= 4 } - if (useMemory) { + if (useMemory_) { ret |= 2 } - if (deserialized) { + if (deserialized_) { ret |= 1 } return ret @@ -57,15 +62,15 @@ class StorageLevel( override def writeExternal(out: ObjectOutput) { out.writeByte(toInt) - out.writeByte(replication) + out.writeByte(replication_) } override def readExternal(in: ObjectInput) { val flags = in.readByte() - useDisk = (flags & 4) != 0 - useMemory = (flags & 2) != 0 - deserialized = (flags & 1) != 0 - replication = in.readByte() + useDisk_ = (flags & 4) != 0 + useMemory_ = (flags & 2) != 0 + deserialized_ = (flags & 1) != 0 + replication_ = in.readByte() } @throws(classOf[IOException]) @@ -91,6 +96,14 @@ object StorageLevel { val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false) val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) + /** Create a new StorageLevel object */ + def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = + getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication)) + + /** Create a new StorageLevel object from its integer representation */ + def apply(flags: Int, replication: Int) = + getCachedStorageLevel(new StorageLevel(flags, replication)) + private[spark] val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index a33d3324ba617..a1aeb12f25ec9 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -69,23 +69,29 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("StorageLevel object caching") { - val level1 = new StorageLevel(false, false, false, 3) - val level2 = new StorageLevel(false, false, false, 3) + val level1 = StorageLevel(false, false, false, 3) + val level2 = StorageLevel(false, false, false, 3) // this should return the same object as level1 + val level3 = StorageLevel(false, false, false, 2) // this should return a different object + assert(level2 === level1, "level2 is not same as level1") + assert(level2.eq(level1), "level2 is not the same object as level1") + assert(level3 != level1, "level3 is same as level1") val bytes1 = spark.Utils.serialize(level1) val level1_ = spark.Utils.deserialize[StorageLevel](bytes1) val bytes2 = spark.Utils.serialize(level2) val level2_ = spark.Utils.deserialize[StorageLevel](bytes2) assert(level1_ === level1, "Deserialized level1 not same as original level1") - assert(level2_ === level2, "Deserialized level2 not same as original level1") - assert(level1_ === level2_, "Deserialized level1 not same as deserialized level2") - assert(level2_.eq(level1_), "Deserialized level2 not the same object as deserialized level1") + assert(level1_.eq(level1), "Deserialized level1 not the same object as original level2") + assert(level2_ === level2, "Deserialized level2 not same as original level2") + assert(level2_.eq(level1), "Deserialized level2 not the same object as original level1") } test("BlockManagerId object caching") { val id1 = BlockManagerId("XXX", 1) val id2 = BlockManagerId("XXX", 1) // this should return the same object as id1 + val id3 = BlockManagerId("XXX", 2) // this should return a different object assert(id2 === id1, "id2 is not same as id1") assert(id2.eq(id1), "id2 is not the same object as id1") + assert(id3 != id1, "id3 is same as id1") val bytes1 = spark.Utils.serialize(id1) val id1_ = spark.Utils.deserialize[BlockManagerId](bytes1) val bytes2 = spark.Utils.serialize(id2) From 155f31398dc83ecb88b4b3e07849a2a8a0a6592f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 23 Jan 2013 01:10:26 -0800 Subject: [PATCH 040/696] Made StorageLevel constructor private, and added StorageLevels.create() to the Java API. Updates scala and java programming guides. --- core/src/main/scala/spark/api/java/StorageLevels.java | 11 +++++++++++ core/src/main/scala/spark/storage/StorageLevel.scala | 6 +++--- docs/java-programming-guide.md | 3 ++- docs/scala-programming-guide.md | 3 ++- 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/spark/api/java/StorageLevels.java index 722af3c06c668..5e5845ac3a65b 100644 --- a/core/src/main/scala/spark/api/java/StorageLevels.java +++ b/core/src/main/scala/spark/api/java/StorageLevels.java @@ -17,4 +17,15 @@ public class StorageLevels { public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2); public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, 1); public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2); + + /** + * Create a new StorageLevel object. + * @param useDisk saved to disk, if true + * @param useMemory saved to memory, if true + * @param deserialized saved as deserialized objects, if true + * @param replication replication factor + */ + public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) { + return StorageLevel.apply(useDisk, useMemory, deserialized, replication); + } } diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index f2535ae5aebc2..45d6ea2656e41 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -7,10 +7,10 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory * in a serialized format, and whether to replicate the RDD partitions on multiple nodes. * The [[spark.storage.StorageLevel$]] singleton object contains some static constants for - * commonly useful storage levels. The recommended method to create your own storage level - * object is to use `StorageLevel.apply(...)` from the singleton object. + * commonly useful storage levels. To create your own storage level object, use the factor method + * of the singleton object (`StorageLevel(...)`). */ -class StorageLevel( +class StorageLevel private( private var useDisk_ : Boolean, private var useMemory_ : Boolean, private var deserialized_ : Boolean, diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 188ca4995ecad..37a906ea1c780 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -75,7 +75,8 @@ class has a single abstract method, `call()`, that must be implemented. ## Storage Levels RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, such as `MEMORY_AND_DISK`, are -declared in the [spark.api.java.StorageLevels](api/core/index.html#spark.api.java.StorageLevels) class. +declared in the [spark.api.java.StorageLevels](api/core/index.html#spark.api.java.StorageLevels) class. To +define your own storage level, you can use StorageLevels.create(...). # Other Features diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 7350eca83796f..301b330a79e93 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -301,7 +301,8 @@ We recommend going through the following process to select one: * Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones let you continue running tasks on the RDD without waiting to recompute a lost partition. - + +If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#spark.storage.StorageLevel$) singleton object. # Shared Variables From 9a27062260490336a3bfa97c6efd39b1e7e81573 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 23 Jan 2013 01:34:44 -0800 Subject: [PATCH 041/696] Force generation increment after shuffle map stage --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 39a1e6d6c6f58..d8a9049e81518 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -445,9 +445,16 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with logInfo("waiting: " + waiting) logInfo("failed: " + failed) if (stage.shuffleDep != None) { + // We supply true to increment the generation number here in case this is a + // recomputation of the map outputs. In that case, some nodes may have cached + // locations with holes (from when we detected the error) and will need the + // generation incremented to refetch them. + // TODO: Only increment the generation number if this is not the first time + // we registered these map outputs. mapOutputTracker.registerMapOutputs( stage.shuffleDep.get.shuffleId, - stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray) + stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + true) } updateCacheLocs() if (stage.outputLocs.count(_ == Nil) != 0) { From d209b6b7641059610f734414ea05e0494b5510b0 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 23 Jan 2013 01:35:14 -0800 Subject: [PATCH 042/696] Extra debugging from hostLost() --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index d8a9049e81518..740aec2e611c7 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -528,7 +528,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val currentGeneration = maybeGeneration.getOrElse(mapOutputTracker.getGeneration) if (!failedGeneration.contains(host) || failedGeneration(host) < currentGeneration) { failedGeneration(host) = currentGeneration - logInfo("Host lost: " + host) + logInfo("Host lost: " + host + " (generation " + currentGeneration + ")") env.blockManager.master.notifyADeadHost(host) // TODO: This will be really slow if we keep accumulating shuffle map stages for ((shuffleId, stage) <- shuffleToMapStage) { @@ -541,6 +541,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } cacheTracker.cacheLost(host) updateCacheLocs() + } else { + logDebug("Additional host lost message for " + host + + "(generation " + currentGeneration + ")") } } From 0b506dd2ecec909cd514143389d0846db2d194ed Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 23 Jan 2013 01:37:51 -0800 Subject: [PATCH 043/696] Add tests of various node failure scenarios. --- .../test/scala/spark/DistributedSuite.scala | 72 +++++++++++++++++++ 1 file changed, 72 insertions(+) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index cacc2796b6fb1..0d6b265e54df7 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -188,4 +188,76 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter val values = sc.parallelize(1 to 2, 2).map(x => System.getenv("TEST_VAR")).collect() assert(values.toSeq === Seq("TEST_VALUE", "TEST_VALUE")) } + + test("recover from node failures") { + import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity} + DistributedSuite.amMaster = true + sc = new SparkContext(clusterUrl, "test") + val data = sc.parallelize(Seq(true, true), 2) + val singleton = sc.parallelize(Seq(true), 1) + assert(data.count === 2) // force executors to start + val masterId = SparkEnv.get.blockManager.blockManagerId + assert(data.map(markNodeIfIdentity).collect.size === 2) + assert(data.map(failOnMarkedIdentity).collect.size === 2) + } + + test("recover from repeated node failures during shuffle-map") { + import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity} + DistributedSuite.amMaster = true + sc = new SparkContext(clusterUrl, "test") + for (i <- 1 to 3) { + val data = sc.parallelize(Seq(true, false), 2) + val singleton = sc.parallelize(Seq(false), 1) + assert(data.count === 2) + assert(data.map(markNodeIfIdentity).collect.size === 2) + assert(data.map(failOnMarkedIdentity).map(x => x -> x).groupByKey.count === 2) + } + } + + test("recover from repeated node failures during shuffle-reduce") { + import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity} + DistributedSuite.amMaster = true + sc = new SparkContext(clusterUrl, "test") + for (i <- 1 to 3) { + val data = sc.parallelize(Seq(true, true), 2) + val singleton = sc.parallelize(Seq(false), 1) + assert(data.count === 2) + assert(data.map(markNodeIfIdentity).collect.size === 2) + // This relies on mergeCombiners being used to perform the actual reduce for this + // test to actually be testing what it claims. + val grouped = data.map(x => x -> x).combineByKey( + x => x, + (x: Boolean, y: Boolean) => x, + (x: Boolean, y: Boolean) => failOnMarkedIdentity(x) + ) + assert(grouped.collect.size === 1) + } + } +} + +object DistributedSuite { + // Indicates whether this JVM is marked for failure. + var mark = false + + // Set by test to remember if we are in the driver program so we can assert + // that we are not. + var amMaster = false + + // Act like an identity function, but if the argument is true, set mark to true. + def markNodeIfIdentity(item: Boolean): Boolean = { + if (item) { + assert(!amMaster) + mark = true + } + item + } + + // Act like an identity function, but if mark was set to true previously, fail, + // crashing the entire JVM. + def failOnMarkedIdentity(item: Boolean): Boolean = { + if (mark) { + System.exit(42) + } + item + } } From 79d55700ce2559051ac61cc2fb72a67fd7035926 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 23 Jan 2013 01:57:09 -0800 Subject: [PATCH 044/696] One more fix. Made even default constructor of BlockManagerId private to prevent such problems in the future. --- .../src/main/scala/spark/storage/BlockManagerId.scala | 11 ++++++----- .../scala/spark/storage/BlockManagerMessages.scala | 3 +-- core/src/main/scala/spark/storage/StorageLevel.scala | 7 +++++++ 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index 26c98f2ac8e76..abb8b45a1f48b 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -16,9 +16,7 @@ private[spark] class BlockManagerId private ( private var port_ : Int ) extends Externalizable { - private def this(in: ObjectInput) = this(in.readUTF(), in.readInt()) - - def this() = this(null, 0) // For deserialization only + private def this() = this(null, 0) // For deserialization only def ip = ip_ @@ -53,8 +51,11 @@ private[spark] object BlockManagerId { def apply(ip: String, port: Int) = getCachedBlockManagerId(new BlockManagerId(ip, port)) - def apply(in: ObjectInput) = - getCachedBlockManagerId(new BlockManagerId(in)) + def apply(in: ObjectInput) = { + val obj = new BlockManagerId() + obj.readExternal(in) + getCachedBlockManagerId(obj) + } val blockManagerIdCache = new ConcurrentHashMap[BlockManagerId, BlockManagerId]() diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 7437fc63ebf79..30483b0b37e5c 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -56,8 +56,7 @@ class UpdateBlockInfo( override def readExternal(in: ObjectInput) { blockManagerId = BlockManagerId(in) blockId = in.readUTF() - storageLevel = new StorageLevel() - storageLevel.readExternal(in) + storageLevel = StorageLevel(in) memSize = in.readInt() diskSize = in.readInt() } diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index 45d6ea2656e41..d1d1c61c1cb37 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -104,6 +104,13 @@ object StorageLevel { def apply(flags: Int, replication: Int) = getCachedStorageLevel(new StorageLevel(flags, replication)) + /** Read StorageLevel object from ObjectInput stream */ + def apply(in: ObjectInput) = { + val obj = new StorageLevel() + obj.readExternal(in) + getCachedStorageLevel(obj) + } + private[spark] val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() From 666ce431aa03239d580a8c78b3a2f34a851eb413 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 23 Jan 2013 03:15:36 -0800 Subject: [PATCH 045/696] Added support for rescheduling unprocessed batches on master failure. --- .../scala/spark/streaming/Checkpoint.scala | 3 +- .../scala/spark/streaming/JobManager.scala | 30 ++++++++++++++++++- .../scala/spark/streaming/Scheduler.scala | 5 +++- .../spark/streaming/StreamingContext.scala | 4 +-- .../spark/streaming/InputStreamsSuite.scala | 23 +++++++++----- 5 files changed, 53 insertions(+), 12 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 2f3adb39c2506..b9eb7f8ec4a00 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -17,7 +17,8 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val jars = ssc.sc.jars val graph = ssc.graph val checkpointDir = ssc.checkpointDir - val checkpointDuration: Duration = ssc.checkpointDuration + val checkpointDuration = ssc.checkpointDuration + val pendingTimes = ssc.scheduler.jobManager.getPendingTimes() def validate() { assert(master != null, "Checkpoint.master is null") diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala index 3b910538e0289..5acdd01e5816c 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/spark/streaming/JobManager.scala @@ -3,6 +3,8 @@ package spark.streaming import spark.Logging import spark.SparkEnv import java.util.concurrent.Executors +import collection.mutable.HashMap +import collection.mutable.ArrayBuffer private[streaming] @@ -19,15 +21,41 @@ class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging { case e: Exception => logError("Running " + job + " failed", e) } + clearJob(job) } } initLogging() val jobExecutor = Executors.newFixedThreadPool(numThreads) - + val jobs = new HashMap[Time, ArrayBuffer[Job]] + def runJob(job: Job) { + jobs.synchronized { + jobs.getOrElseUpdate(job.time, new ArrayBuffer[Job]) += job + } jobExecutor.execute(new JobHandler(ssc, job)) logInfo("Added " + job + " to queue") } + + private def clearJob(job: Job) { + jobs.synchronized { + val jobsOfTime = jobs.get(job.time) + if (jobsOfTime.isDefined) { + jobsOfTime.get -= job + if (jobsOfTime.get.isEmpty) { + jobs -= job.time + } + } else { + throw new Exception("Job finished for time " + job.time + + " but time does not exist in jobs") + } + } + } + + def getPendingTimes(): Array[Time] = { + jobs.synchronized { + jobs.keySet.toArray + } + } } diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala index c04ed37de8b21..b77986a3ba420 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala @@ -35,10 +35,13 @@ class Scheduler(ssc: StreamingContext) extends Logging { // either set the manual clock to the last checkpointed time, // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { - val lastTime = ssc.getInitialCheckpoint.checkpointTime.milliseconds + val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } + // Reschedule the batches that were received but not processed before failure + ssc.initialCheckpoint.pendingTimes.foreach(time => generateRDDs(time)) + // Restart the timer timer.restart(graph.zeroTime.milliseconds) logInfo("Scheduler's timer restarted") } else { diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 2cf00e3baab88..5781b1cc720b9 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -133,7 +133,7 @@ class StreamingContext private ( } } - protected[streaming] def getInitialCheckpoint(): Checkpoint = { + protected[streaming] def initialCheckpoint: Checkpoint = { if (isCheckpointPresent) cp_ else null } @@ -367,7 +367,7 @@ class StreamingContext private ( } /** - * Sstops the execution of the streams. + * Stops the execution of the streams. */ def stop() { try { diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 4f6204f2058b4..34e51e95628b3 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -44,7 +44,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.master.port") } - + /* test("network input stream") { // Start the server testServer = new TestServer(testPort) @@ -236,8 +236,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i).head.toString === expectedOutput(i)) } } - - test("file input stream with checkpoint") { + */ + test("file input stream with master failure") { // Create a temporary directory testDir = { var temp = File.createTempFile(".temp.", Random.nextInt().toString) @@ -251,11 +251,17 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { var ssc = new StreamingContext(master, framework, batchDuration) ssc.checkpoint(checkpointDir, checkpointInterval) val fileStream = ssc.textFileStream(testDir.toString) - val outputBuffer = new ArrayBuffer[Seq[Int]] - // Reduced over a large window to ensure that recovery from master failure + // Making value 3 take large time to process, to ensure that the master + // shuts down in the middle of processing the 3rd batch + val mappedStream = fileStream.map(s => { + val i = s.toInt + if (i == 3) Thread.sleep(1000) + i + }) + // Reducing over a large window to ensure that recovery from master failure // requires reprocessing of all the files seen before the failure - val reducedStream = fileStream.map(_.toInt) - .reduceByWindow(_ + _, batchDuration * 30, batchDuration) + val reducedStream = mappedStream.reduceByWindow(_ + _, batchDuration * 30, batchDuration) + val outputBuffer = new ArrayBuffer[Seq[Int]] var outputStream = new TestOutputStream(reducedStream, outputBuffer) ssc.registerOutputStream(outputStream) ssc.start() @@ -275,6 +281,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(outputStream.output.size > 0, "No files processed before restart") ssc.stop() + // Create files while the master is down for (i <- Seq(4, 5, 6)) { FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") Thread.sleep(1000) @@ -293,6 +300,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { Thread.sleep(500) } Thread.sleep(1000) + logInfo("Output = " + outputStream.output.mkString(",")) assert(outputStream.output.size > 0, "No files processed after restart") ssc.stop() @@ -316,6 +324,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(outputBuffer(i).head === expectedOutput(i)) } } + } From 9c8ff1e55fb97980e7f0bb7f305c1ed0e59b749e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 23 Jan 2013 07:31:49 -0800 Subject: [PATCH 046/696] Fixed checkpoint testcases --- streaming/src/test/java/JavaAPISuite.java | 23 +-- .../spark/streaming/CheckpointSuite.scala | 115 +++++++++++- .../spark/streaming/InputStreamsSuite.scala | 163 +----------------- 3 files changed, 129 insertions(+), 172 deletions(-) diff --git a/streaming/src/test/java/JavaAPISuite.java b/streaming/src/test/java/JavaAPISuite.java index c84e7331c75e8..7a189d85b48bd 100644 --- a/streaming/src/test/java/JavaAPISuite.java +++ b/streaming/src/test/java/JavaAPISuite.java @@ -45,7 +45,7 @@ public void tearDown() { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.master.port"); } - + /* @Test public void testCount() { List> inputData = Arrays.asList( @@ -434,7 +434,7 @@ public void testUnion() { assertOrderInvariantEquals(expected, result); } - + */ /* * Performs an order-invariant comparison of lists representing two RDD streams. This allows * us to account for ordering variation within individual RDD's which occurs during windowing. @@ -450,7 +450,7 @@ public static void assertOrderInvariantEquals( Assert.assertEquals(expected, actual); } - + /* // PairDStream Functions @Test public void testPairFilter() { @@ -897,7 +897,7 @@ public void testJoin() { Assert.assertEquals(expected, result); } - + */ @Test public void testCheckpointMasterRecovery() throws InterruptedException { List> inputData = Arrays.asList( @@ -911,7 +911,6 @@ public void testCheckpointMasterRecovery() throws InterruptedException { Arrays.asList(1,4), Arrays.asList(8,7)); - File tempDir = Files.createTempDir(); ssc.checkpoint(tempDir.getAbsolutePath(), new Duration(1000)); @@ -927,14 +926,16 @@ public Integer call(String s) throws Exception { assertOrderInvariantEquals(expectedInitial, initialResult); Thread.sleep(1000); - ssc.stop(); + ssc = new JavaStreamingContext(tempDir.getAbsolutePath()); - ssc.start(); - List> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 2); - assertOrderInvariantEquals(expectedFinal, finalResult); + // Tweak to take into consideration that the last batch before failure + // will be re-processed after recovery + List> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3); + assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3)); } + /** TEST DISABLED: Pending a discussion about checkpoint() semantics with TD @Test public void testCheckpointofIndividualStream() throws InterruptedException { @@ -963,7 +964,7 @@ public Integer call(String s) throws Exception { assertOrderInvariantEquals(expected, result1); } */ - + /* // Input stream tests. These mostly just test that we can instantiate a given InputStream with // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the // InputStream functionality is deferred to the existing Scala tests. @@ -1025,5 +1026,5 @@ public void testFlumeStream() { public void testFileStream() { JavaPairDStream foo = ssc.fileStream("/tmp/foo"); - } + }*/ } diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index 58da4ee539816..04ccca4c011e0 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -7,6 +7,8 @@ import org.scalatest.BeforeAndAfter import org.apache.commons.io.FileUtils import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import util.{Clock, ManualClock} +import scala.util.Random +import com.google.common.io.Files class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { @@ -32,7 +34,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { override def actuallyWait = true - test("basic stream+rdd recovery") { + test("basic rdd checkpoints + dstream graph checkpoint recovery") { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") assert(checkpointInterval === batchDuration, "checkpointInterval for this test much be same as batchDuration") @@ -117,7 +119,10 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ssc = null } - test("map and reduceByKey") { + // This tests whether the systm can recover from a master failure with simple + // non-stateful operations. This assumes as reliable, replayable input + // source - TestInputDStream. + test("recovery with map and reduceByKey operations") { testCheckpointedOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq() ), (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _), @@ -126,7 +131,11 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ) } - test("reduceByKeyAndWindowInv") { + + // This tests whether the ReduceWindowedDStream's RDD checkpoints works correctly such + // that the system can recover from a master failure. This assumes as reliable, + // replayable input source - TestInputDStream. + test("recovery with invertible reduceByKeyAndWindow operation") { val n = 10 val w = 4 val input = (1 to n).map(_ => Seq("a")).toSeq @@ -139,7 +148,11 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { testCheckpointedOperation(input, operation, output, 7) } - test("updateStateByKey") { + + // This tests whether the StateDStream's RDD checkpoints works correctly such + // that the system can recover from a master failure. This assumes as reliable, + // replayable input source - TestInputDStream. + test("recovery with updateStateByKey operation") { val input = (1 to 10).map(_ => Seq("a")).toSeq val output = (1 to 10).map(x => Seq(("a", x))).toSeq val operation = (st: DStream[String]) => { @@ -154,11 +167,99 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { testCheckpointedOperation(input, operation, output, 7) } + // This tests whether file input stream remembers what files were seen before + // the master failure and uses them again to process a large window operatoin. + // It also tests whether batches, whose processing was incomplete due to the + // failure, are re-processed or not. + test("recovery with file input stream") { + // Set up the streaming context and input streams + val testDir = Files.createTempDir() + var ssc = new StreamingContext(master, framework, batchDuration) + ssc.checkpoint(checkpointDir, checkpointInterval) + val fileStream = ssc.textFileStream(testDir.toString) + // Making value 3 take large time to process, to ensure that the master + // shuts down in the middle of processing the 3rd batch + val mappedStream = fileStream.map(s => { + val i = s.toInt + if (i == 3) Thread.sleep(1000) + i + }) + // Reducing over a large window to ensure that recovery from master failure + // requires reprocessing of all the files seen before the failure + val reducedStream = mappedStream.reduceByWindow(_ + _, batchDuration * 30, batchDuration) + val outputBuffer = new ArrayBuffer[Seq[Int]] + var outputStream = new TestOutputStream(reducedStream, outputBuffer) + ssc.registerOutputStream(outputStream) + ssc.start() + + // Create files and advance manual clock to process them + var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + Thread.sleep(1000) + for (i <- Seq(1, 2, 3)) { + FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") + // wait to make sure that the file is written such that it gets shown in the file listings + Thread.sleep(500) + clock.addToTime(batchDuration.milliseconds) + // wait to make sure that FileInputDStream picks up this file only and not any other file + Thread.sleep(500) + } + logInfo("Output = " + outputStream.output.mkString(",")) + assert(outputStream.output.size > 0, "No files processed before restart") + ssc.stop() + + // Create files while the master is down + for (i <- Seq(4, 5, 6)) { + FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") + Thread.sleep(1000) + } + + // Restart stream computation from checkpoint and create more files to see whether + // they are being processed + logInfo("*********** RESTARTING ************") + ssc = new StreamingContext(checkpointDir) + ssc.start() + clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + for (i <- Seq(7, 8, 9)) { + FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") + Thread.sleep(500) + clock.addToTime(batchDuration.milliseconds) + Thread.sleep(500) + } + Thread.sleep(1000) + logInfo("Output = " + outputStream.output.mkString(",")) + assert(outputStream.output.size > 0, "No files processed after restart") + ssc.stop() + + // Append the new output to the old buffer + outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] + outputBuffer ++= outputStream.output + + // Verify whether data received by Spark Streaming was as expected + val expectedOutput = Seq(1, 3, 6, 28, 36, 45) + logInfo("--------------------------------") + logInfo("output, size = " + outputBuffer.size) + outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("expected output, size = " + expectedOutput.size) + expectedOutput.foreach(x => logInfo("[" + x + "]")) + logInfo("--------------------------------") + + // Verify whether all the elements received are as expected + assert(outputBuffer.size === expectedOutput.size) + for (i <- 0 until outputBuffer.size) { + assert(outputBuffer(i).size === 1) + assert(outputBuffer(i).head === expectedOutput(i)) + } + } + + /** - * Tests a streaming operation under checkpointing, by restart the operation + * Tests a streaming operation under checkpointing, by restarting the operation * from checkpoint file and verifying whether the final output is correct. * The output is assumed to have come from a reliable queue which an replay * data as required. + * + * NOTE: This takes into consideration that the last batch processed before + * master failure will be re-processed after restart/recovery. */ def testCheckpointedOperation[U: ClassManifest, V: ClassManifest]( input: Seq[Seq[U]], @@ -172,7 +273,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { val totalNumBatches = input.size val nextNumBatches = totalNumBatches - initialNumBatches val initialNumExpectedOutputs = initialNumBatches - val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs + val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs + 1 + // because the last batch will be processed again // Do the computation for initial number of batches, create checkpoint file and quit ssc = setupStreams[U, V](input, operation) @@ -188,6 +290,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ) ssc = new StreamingContext(checkpointDir) val outputNew = runStreams[V](ssc, nextNumBatches, nextNumExpectedOutputs) + // the first element will be re-processed data of the last batch before restart verifyOutput[V](outputNew, expectedOutput.takeRight(nextNumExpectedOutputs), true) ssc = null } diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 34e51e95628b3..aa08ea114125b 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -19,35 +19,24 @@ import org.apache.avro.ipc.specific.SpecificRequestor import java.nio.ByteBuffer import collection.JavaConversions._ import java.nio.charset.Charset +import com.google.common.io.Files class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") - val testPort = 9999 - var testServer: TestServer = null - var testDir: File = null - override def checkpointDir = "checkpoint" after { - FileUtils.deleteDirectory(new File(checkpointDir)) - if (testServer != null) { - testServer.stop() - testServer = null - } - if (testDir != null && testDir.exists()) { - FileUtils.deleteDirectory(testDir) - testDir = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.master.port") } - /* + + test("network input stream") { // Start the server - testServer = new TestServer(testPort) + val testPort = 9999 + val testServer = new TestServer(testPort) testServer.start() // Set up the streaming context and input streams @@ -93,46 +82,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } - test("network input stream with checkpoint") { - // Start the server - testServer = new TestServer(testPort) - testServer.start() - - // Set up the streaming context and input streams - var ssc = new StreamingContext(master, framework, batchDuration) - ssc.checkpoint(checkpointDir, checkpointInterval) - val networkStream = ssc.networkTextStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) - var outputStream = new TestOutputStream(networkStream, new ArrayBuffer[Seq[String]]) - ssc.registerOutputStream(outputStream) - ssc.start() - - // Feed data to the server to send to the network receiver - var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - for (i <- Seq(1, 2, 3)) { - testServer.send(i.toString + "\n") - Thread.sleep(100) - clock.addToTime(batchDuration.milliseconds) - } - Thread.sleep(500) - assert(outputStream.output.size > 0) - ssc.stop() - - // Restart stream computation from checkpoint and feed more data to see whether - // they are being received and processed - logInfo("*********** RESTARTING ************") - ssc = new StreamingContext(checkpointDir) - ssc.start() - clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - for (i <- Seq(4, 5, 6)) { - testServer.send(i.toString + "\n") - Thread.sleep(100) - clock.addToTime(batchDuration.milliseconds) - } - Thread.sleep(500) - outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[String]] - assert(outputStream.output.size > 0) - ssc.stop() - } test("flume input stream") { // Set up the streaming context and input streams @@ -182,18 +131,10 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } - test("file input stream") { - - // Create a temporary directory - testDir = { - var temp = File.createTempFile(".temp.", Random.nextInt().toString) - temp.delete() - temp.mkdirs() - logInfo("Created temp dir " + temp) - temp - } + test("file input stream") { // Set up the streaming context and input streams + val testDir = Files.createTempDir() val ssc = new StreamingContext(master, framework, batchDuration) val filestream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] @@ -235,96 +176,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i).size === 1) assert(output(i).head.toString === expectedOutput(i)) } + FileUtils.deleteDirectory(testDir) } - */ - test("file input stream with master failure") { - // Create a temporary directory - testDir = { - var temp = File.createTempFile(".temp.", Random.nextInt().toString) - temp.delete() - temp.mkdirs() - logInfo("Created temp dir " + temp) - temp - } - - // Set up the streaming context and input streams - var ssc = new StreamingContext(master, framework, batchDuration) - ssc.checkpoint(checkpointDir, checkpointInterval) - val fileStream = ssc.textFileStream(testDir.toString) - // Making value 3 take large time to process, to ensure that the master - // shuts down in the middle of processing the 3rd batch - val mappedStream = fileStream.map(s => { - val i = s.toInt - if (i == 3) Thread.sleep(1000) - i - }) - // Reducing over a large window to ensure that recovery from master failure - // requires reprocessing of all the files seen before the failure - val reducedStream = mappedStream.reduceByWindow(_ + _, batchDuration * 30, batchDuration) - val outputBuffer = new ArrayBuffer[Seq[Int]] - var outputStream = new TestOutputStream(reducedStream, outputBuffer) - ssc.registerOutputStream(outputStream) - ssc.start() - - // Create files and advance manual clock to process them - var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - Thread.sleep(1000) - for (i <- Seq(1, 2, 3)) { - FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") - // wait to make sure that the file is written such that it gets shown in the file listings - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - // wait to make sure that FileInputDStream picks up this file only and not any other file - Thread.sleep(500) - } - logInfo("Output = " + outputStream.output.mkString(",")) - assert(outputStream.output.size > 0, "No files processed before restart") - ssc.stop() - - // Create files while the master is down - for (i <- Seq(4, 5, 6)) { - FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") - Thread.sleep(1000) - } - - // Restart stream computation from checkpoint and create more files to see whether - // they are being processed - logInfo("*********** RESTARTING ************") - ssc = new StreamingContext(checkpointDir) - ssc.start() - clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - for (i <- Seq(7, 8, 9)) { - FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - Thread.sleep(500) - } - Thread.sleep(1000) - logInfo("Output = " + outputStream.output.mkString(",")) - assert(outputStream.output.size > 0, "No files processed after restart") - ssc.stop() - - // Append the new output to the old buffer - outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] - outputBuffer ++= outputStream.output - - // Verify whether data received by Spark Streaming was as expected - val expectedOutput = Seq(1, 3, 6, 28, 36, 45) - logInfo("--------------------------------") - logInfo("output, size = " + outputBuffer.size) - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output, size = " + expectedOutput.size) - expectedOutput.foreach(x => logInfo("[" + x + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - assert(outputBuffer.size === expectedOutput.size) - for (i <- 0 until outputBuffer.size) { - assert(outputBuffer(i).size === 1) - assert(outputBuffer(i).head === expectedOutput(i)) - } - } - } From ae2ed2947d43860c74a8d40767e289ca78073977 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 23 Jan 2013 10:36:18 -0800 Subject: [PATCH 047/696] Allow PySpark's SparkFiles to be used from driver Fix minor documentation formatting issues. --- core/src/main/scala/spark/SparkFiles.java | 8 +++---- python/pyspark/context.py | 27 ++++++++++++++++++----- python/pyspark/files.py | 20 ++++++++++++++--- python/pyspark/tests.py | 23 +++++++++++++++++++ python/pyspark/worker.py | 1 + python/test_support/hello.txt | 1 + 6 files changed, 67 insertions(+), 13 deletions(-) create mode 100755 python/test_support/hello.txt diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/spark/SparkFiles.java index b59d8ce93f23d..566aec622c096 100644 --- a/core/src/main/scala/spark/SparkFiles.java +++ b/core/src/main/scala/spark/SparkFiles.java @@ -3,23 +3,23 @@ import java.io.File; /** - * Resolves paths to files added through `addFile(). + * Resolves paths to files added through `SparkContext.addFile()`. */ public class SparkFiles { private SparkFiles() {} /** - * Get the absolute path of a file added through `addFile()`. + * Get the absolute path of a file added through `SparkContext.addFile()`. */ public static String get(String filename) { return new File(getRootDirectory(), filename).getAbsolutePath(); } /** - * Get the root directory that contains files added through `addFile()`. + * Get the root directory that contains files added through `SparkContext.addFile()`. */ public static String getRootDirectory() { return SparkEnv.get().sparkFilesDir(); } -} \ No newline at end of file +} diff --git a/python/pyspark/context.py b/python/pyspark/context.py index b8d7dc05af43d..3e33776af0add 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1,12 +1,15 @@ import os import atexit import shutil +import sys import tempfile +from threading import Lock from tempfile import NamedTemporaryFile from pyspark import accumulators from pyspark.accumulators import Accumulator from pyspark.broadcast import Broadcast +from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import dump_pickle, write_with_length, batched from pyspark.rdd import RDD @@ -27,6 +30,8 @@ class SparkContext(object): _writeIteratorToPickleFile = jvm.PythonRDD.writeIteratorToPickleFile _takePartition = jvm.PythonRDD.takePartition _next_accum_id = 0 + _active_spark_context = None + _lock = Lock() def __init__(self, master, jobName, sparkHome=None, pyFiles=None, environment=None, batchSize=1024): @@ -46,6 +51,11 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, Java object. Set 1 to disable batching or -1 to use an unlimited batch size. """ + with SparkContext._lock: + if SparkContext._active_spark_context: + raise ValueError("Cannot run multiple SparkContexts at once") + else: + SparkContext._active_spark_context = self self.master = master self.jobName = jobName self.sparkHome = sparkHome or None # None becomes null in Py4J @@ -75,6 +85,8 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, # Deploy any code dependencies specified in the constructor for path in (pyFiles or []): self.addPyFile(path) + SparkFiles._sc = self + sys.path.append(SparkFiles.getRootDirectory()) @property def defaultParallelism(self): @@ -85,17 +97,20 @@ def defaultParallelism(self): return self._jsc.sc().defaultParallelism() def __del__(self): - if self._jsc: - self._jsc.stop() - if self._accumulatorServer: - self._accumulatorServer.shutdown() + self.stop() def stop(self): """ Shut down the SparkContext. """ - self._jsc.stop() - self._jsc = None + if self._jsc: + self._jsc.stop() + self._jsc = None + if self._accumulatorServer: + self._accumulatorServer.shutdown() + self._accumulatorServer = None + with SparkContext._lock: + SparkContext._active_spark_context = None def parallelize(self, c, numSlices=None): """ diff --git a/python/pyspark/files.py b/python/pyspark/files.py index de1334f046c6b..98f6a399cc26b 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -4,13 +4,15 @@ class SparkFiles(object): """ Resolves paths to files added through - L{addFile()}. + L{SparkContext.addFile()}. SparkFiles contains only classmethods; users should not create SparkFiles instances. """ _root_directory = None + _is_running_on_worker = False + _sc = None def __init__(self): raise NotImplementedError("Do not construct SparkFiles objects") @@ -18,7 +20,19 @@ def __init__(self): @classmethod def get(cls, filename): """ - Get the absolute path of a file added through C{addFile()}. + Get the absolute path of a file added through C{SparkContext.addFile()}. """ - path = os.path.join(SparkFiles._root_directory, filename) + path = os.path.join(SparkFiles.getRootDirectory(), filename) return os.path.abspath(path) + + @classmethod + def getRootDirectory(cls): + """ + Get the root directory that contains files added through + C{SparkContext.addFile()}. + """ + if cls._is_running_on_worker: + return cls._root_directory + else: + # This will have to change if we support multiple SparkContexts: + return cls._sc.jvm.spark.SparkFiles.getRootDirectory() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 4d70ee4f12500..46ab34f063b2b 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -4,22 +4,26 @@ """ import os import shutil +import sys from tempfile import NamedTemporaryFile import time import unittest from pyspark.context import SparkContext +from pyspark.files import SparkFiles from pyspark.java_gateway import SPARK_HOME class PySparkTestCase(unittest.TestCase): def setUp(self): + self._old_sys_path = list(sys.path) class_name = self.__class__.__name__ self.sc = SparkContext('local[4]', class_name , batchSize=2) def tearDown(self): self.sc.stop() + sys.path = self._old_sys_path # To avoid Akka rebinding to the same port, since it doesn't unbind # immediately on shutdown self.sc.jvm.System.clearProperty("spark.master.port") @@ -84,6 +88,25 @@ def func(x): res = self.sc.parallelize(range(2)).map(func).first() self.assertEqual("Hello World!", res) + def test_add_file_locally(self): + path = os.path.join(SPARK_HOME, "python/test_support/hello.txt") + self.sc.addFile(path) + download_path = SparkFiles.get("hello.txt") + self.assertNotEqual(path, download_path) + with open(download_path) as test_file: + self.assertEquals("Hello World!\n", test_file.readline()) + + def test_add_py_file_locally(self): + # To ensure that we're actually testing addPyFile's effects, check that + # this fails due to `userlibrary` not being on the Python path: + def func(): + from userlibrary import UserClass + self.assertRaises(ImportError, func) + path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") + self.sc.addFile(path) + from userlibrary import UserClass + self.assertEqual("Hello World!", UserClass().hello()) + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4bf643da66d1d..d33d6dd15f0de 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -26,6 +26,7 @@ def main(): split_index = read_int(sys.stdin) spark_files_dir = load_pickle(read_with_length(sys.stdin)) SparkFiles._root_directory = spark_files_dir + SparkFiles._is_running_on_worker = True sys.path.append(spark_files_dir) num_broadcast_variables = read_int(sys.stdin) for _ in range(num_broadcast_variables): diff --git a/python/test_support/hello.txt b/python/test_support/hello.txt new file mode 100755 index 0000000000000..980a0d5f19a64 --- /dev/null +++ b/python/test_support/hello.txt @@ -0,0 +1 @@ +Hello World! From b47d054cfc5ef45b92a1c970388722ffa0283e66 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 23 Jan 2013 11:18:25 -0800 Subject: [PATCH 048/696] Remove use of abc.ABCMeta due to cloudpickle issue. cloudpickle runs into issues while pickling subclasses of AccumulatorParam, which may be related to this Python issue: http://bugs.python.org/issue7689 This seems hard to fix and the ABCMeta wasn't necessary, so I removed it. --- python/pyspark/accumulators.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 5a9269f9bb251..61fcbbd37679f 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -25,7 +25,8 @@ >>> a.value 13 ->>> class VectorAccumulatorParam(object): +>>> from pyspark.accumulators import AccumulatorParam +>>> class VectorAccumulatorParam(AccumulatorParam): ... def zero(self, value): ... return [0.0] * len(value) ... def addInPlace(self, val1, val2): @@ -61,7 +62,6 @@ Exception:... """ -from abc import ABCMeta, abstractmethod import struct import SocketServer import threading @@ -138,23 +138,20 @@ class AccumulatorParam(object): """ Helper object that defines how to accumulate values of a given type. """ - __metaclass__ = ABCMeta - @abstractmethod def zero(self, value): """ Provide a "zero value" for the type, compatible in dimensions with the provided C{value} (e.g., a zero vector) """ - return + raise NotImplementedError - @abstractmethod def addInPlace(self, value1, value2): """ Add two values of the accumulator's data type, returning a new value; for efficiency, can also update C{value1} in place and return it. """ - return + raise NotImplementedError class AddingAccumulatorParam(AccumulatorParam): From e1027ca6398fd5b1a99a2203df840911c4dccb27 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 23 Jan 2013 12:22:11 -0800 Subject: [PATCH 049/696] Actually add CacheManager. --- core/src/main/scala/spark/CacheManager.scala | 65 ++++++++++++++++++++ 1 file changed, 65 insertions(+) create mode 100644 core/src/main/scala/spark/CacheManager.scala diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala new file mode 100644 index 0000000000000..a0b53fd9d67c5 --- /dev/null +++ b/core/src/main/scala/spark/CacheManager.scala @@ -0,0 +1,65 @@ +package spark + +import scala.collection.mutable.{ArrayBuffer, HashSet} +import spark.storage.{BlockManager, StorageLevel} + + +/** Spark class responsible for passing RDDs split contents to the BlockManager and making + sure a node doesn't load two copies of an RDD at once. + */ +private[spark] class CacheManager(blockManager: BlockManager) extends Logging { + private val loading = new HashSet[String] + + /** Gets or computes an RDD split. Used by RDD.iterator() when a RDD is cached. */ + def getOrCompute[T](rdd: RDD[T], split: Split, context: TaskContext, storageLevel: StorageLevel) + : Iterator[T] = { + val key = "rdd_%d_%d".format(rdd.id, split.index) + logInfo("Cache key is " + key) + blockManager.get(key) match { + case Some(cachedValues) => + // Split is in cache, so just return its values + logInfo("Found partition in cache!") + return cachedValues.asInstanceOf[Iterator[T]] + + case None => + // Mark the split as loading (unless someone else marks it first) + loading.synchronized { + if (loading.contains(key)) { + logInfo("Loading contains " + key + ", waiting...") + while (loading.contains(key)) { + try {loading.wait()} catch {case _ =>} + } + logInfo("Loading no longer contains " + key + ", so returning cached result") + // See whether someone else has successfully loaded it. The main way this would fail + // is for the RDD-level cache eviction policy if someone else has loaded the same RDD + // partition but we didn't want to make space for it. However, that case is unlikely + // because it's unlikely that two threads would work on the same RDD partition. One + // downside of the current code is that threads wait serially if this does happen. + blockManager.get(key) match { + case Some(values) => + return values.asInstanceOf[Iterator[T]] + case None => + logInfo("Whoever was loading " + key + " failed; we'll try it ourselves") + loading.add(key) + } + } else { + loading.add(key) + } + } + try { + // If we got here, we have to load the split + val elements = new ArrayBuffer[Any] + logInfo("Computing partition " + split) + elements ++= rdd.compute(split, context) + // Try to put this block in the blockManager + blockManager.put(key, elements, storageLevel, true) + return elements.iterator.asInstanceOf[Iterator[T]] + } finally { + loading.synchronized { + loading.remove(key) + loading.notifyAll() + } + } + } + } +} From 88b9d240fda7ca34c08752dfa66797eecb6db872 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 23 Jan 2013 12:40:38 -0800 Subject: [PATCH 050/696] Remove dead code in test. --- core/src/test/scala/spark/DistributedSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 0d6b265e54df7..af66d33aa3cf7 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -194,7 +194,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter DistributedSuite.amMaster = true sc = new SparkContext(clusterUrl, "test") val data = sc.parallelize(Seq(true, true), 2) - val singleton = sc.parallelize(Seq(true), 1) assert(data.count === 2) // force executors to start val masterId = SparkEnv.get.blockManager.blockManagerId assert(data.map(markNodeIfIdentity).collect.size === 2) @@ -207,7 +206,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter sc = new SparkContext(clusterUrl, "test") for (i <- 1 to 3) { val data = sc.parallelize(Seq(true, false), 2) - val singleton = sc.parallelize(Seq(false), 1) assert(data.count === 2) assert(data.map(markNodeIfIdentity).collect.size === 2) assert(data.map(failOnMarkedIdentity).map(x => x -> x).groupByKey.count === 2) From be4a115a7ec7fb6ec0d34f1a1a1bb2c9bbe7600e Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 23 Jan 2013 12:48:45 -0800 Subject: [PATCH 051/696] Clarify TODO. --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 740aec2e611c7..14a3ef8ad7751 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -76,7 +76,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with // sent with every task. When we detect a node failing, we note the current generation number // and failed host, increment it for new tasks, and use this to ignore stray ShuffleMapTask // results. - // TODO: Garbage collect information about failure generations when new stages start. + // TODO: Garbage collect information about failure generations when we know there are no more + // stray messages to detect. val failedGeneration = new HashMap[String, Long] val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done From e1985bfa04ad4583ac1f0f421cbe0182ce7c53df Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 21 Jan 2013 16:21:14 -0800 Subject: [PATCH 052/696] be sure to set class loader of kryo instances --- core/src/main/scala/spark/KryoSerializer.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index 93d7327324ed3..56919544e8ef8 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -206,5 +206,8 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { kryo } - def newInstance(): SerializerInstance = new KryoSerializerInstance(this) + def newInstance(): SerializerInstance = { + this.kryo.setClassLoader(Thread.currentThread().getContextClassLoader) + new KryoSerializerInstance(this) + } } From 5c7422292ecace947f78e5ebe97e83a355531af7 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 23 Jan 2013 12:59:51 -0800 Subject: [PATCH 053/696] Remove more dead code from test. --- core/src/test/scala/spark/DistributedSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index af66d33aa3cf7..0487e06d12b10 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -218,7 +218,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter sc = new SparkContext(clusterUrl, "test") for (i <- 1 to 3) { val data = sc.parallelize(Seq(true, true), 2) - val singleton = sc.parallelize(Seq(false), 1) assert(data.count === 2) assert(data.map(markNodeIfIdentity).collect.size === 2) // This relies on mergeCombiners being used to perform the actual reduce for this From 1dd82743e09789f8fdae2f5628545c0cb9f79245 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 23 Jan 2013 13:07:27 -0800 Subject: [PATCH 054/696] Fix compile error due to cherry-pick --- core/src/main/scala/spark/KryoSerializer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index 56919544e8ef8..0bd73e936bd05 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -207,7 +207,7 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { } def newInstance(): SerializerInstance = { - this.kryo.setClassLoader(Thread.currentThread().getContextClassLoader) + this.kryo.get().setClassLoader(Thread.currentThread().getContextClassLoader) new KryoSerializerInstance(this) } } From eb222b720647c9e92a867c591cc4914b9a6cb5c1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 15:29:02 -0800 Subject: [PATCH 055/696] Added pruntSplits method to RDD. --- core/src/main/scala/spark/RDD.scala | 10 ++++++++ .../scala/spark/rdd/SplitsPruningRDD.scala | 24 +++++++++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 22 +++++++++++------ 3 files changed, 49 insertions(+), 7 deletions(-) create mode 100644 core/src/main/scala/spark/rdd/SplitsPruningRDD.scala diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e0d2eabb1dd65..3d93ff33bbf7b 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -40,6 +40,7 @@ import spark.rdd.MapPartitionsRDD import spark.rdd.MapPartitionsWithSplitRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD +import spark.rdd.SplitsPruningRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD import spark.storage.StorageLevel @@ -543,6 +544,15 @@ abstract class RDD[T: ClassManifest]( map(x => (f(x), x)) } + /** + * Prune splits (partitions) so Spark can avoid launching tasks on + * all splits. An example use case: If we know the RDD is partitioned by range, + * and the execution DAG has a filter on the key, we can avoid launching tasks + * on splits that don't have the range covering the key. + */ + def pruneSplits(splitsFilterFunc: Int => Boolean): RDD[T] = + new SplitsPruningRDD(this, splitsFilterFunc) + /** A private method for tests, to look at the contents of each partition */ private[spark] def collectPartitions(): Array[Array[T]] = { sc.runJob(this, (iter: Iterator[T]) => iter.toArray) diff --git a/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala b/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala new file mode 100644 index 0000000000000..74e10265fc54c --- /dev/null +++ b/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala @@ -0,0 +1,24 @@ +package spark.rdd + +import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext} + +/** + * A RDD used to prune RDD splits so we can avoid launching tasks on + * all splits. An example use case: If we know the RDD is partitioned by range, + * and the execution DAG has a filter on the key, we can avoid launching tasks + * on splits that don't have the range covering the key. + */ +class SplitsPruningRDD[T: ClassManifest]( + prev: RDD[T], + @transient splitsFilterFunc: Int => Boolean) + extends RDD[T](prev) { + + @transient + val _splits: Array[Split] = prev.splits.filter(s => splitsFilterFunc(s.index)) + + override def compute(split: Split, context: TaskContext) = prev.iterator(split, context) + + override protected def getSplits = _splits + + override val partitioner = prev.partitioner +} diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index db217f8482510..03aa2845f4630 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -1,11 +1,9 @@ package spark import scala.collection.mutable.HashMap -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter - +import org.scalatest.{BeforeAndAfter, FunSuite} +import spark.SparkContext._ import spark.rdd.CoalescedRDD -import SparkContext._ class RDDSuite extends FunSuite with BeforeAndAfter { @@ -104,7 +102,7 @@ class RDDSuite extends FunSuite with BeforeAndAfter { } test("caching with failures") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test") val onlySplit = new Split { override def index: Int = 0 } var shouldFail = true val rdd = new RDD[Int](sc, Nil) { @@ -136,8 +134,10 @@ class RDDSuite extends FunSuite with BeforeAndAfter { List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10))) // Check that the narrow dependency is also specified correctly - assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(0).toList === List(0, 1, 2, 3, 4)) - assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList === List(5, 6, 7, 8, 9)) + assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(0).toList === + List(0, 1, 2, 3, 4)) + assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList === + List(5, 6, 7, 8, 9)) val coalesced2 = new CoalescedRDD(data, 3) assert(coalesced2.collect().toList === (1 to 10).toList) @@ -168,4 +168,12 @@ class RDDSuite extends FunSuite with BeforeAndAfter { nums.zip(sc.parallelize(1 to 4, 1)).collect() } } + + test("split pruning") { + sc = new SparkContext("local", "test") + val data = sc.parallelize(1 to 10, 10) + // Note that split number starts from 0, so > 8 means only 10th partition left. + val prunedData = data.pruneSplits(splitNum => splitNum > 8).collect + assert(prunedData.size == 1 && prunedData(0) == 10) + } } From c24b3819dd474e13d6098150c174b2e7e4bc6498 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 15:34:59 -0800 Subject: [PATCH 056/696] Added an extra assert for split size check. --- core/src/test/scala/spark/RDDSuite.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 03aa2845f4630..ef74c99246728 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -173,7 +173,10 @@ class RDDSuite extends FunSuite with BeforeAndAfter { sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) // Note that split number starts from 0, so > 8 means only 10th partition left. - val prunedData = data.pruneSplits(splitNum => splitNum > 8).collect - assert(prunedData.size == 1 && prunedData(0) == 10) + val prunedRdd = data.pruneSplits(splitNum => splitNum > 8) + assert(prunedRdd.splits.size == 1) + val prunedData = prunedRdd.collect + assert(prunedData.size == 1) + assert(prunedData(0) == 10) } } From 45cd50d5fe40869cdc237157e073cfb5ac47b27c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 16:06:58 -0800 Subject: [PATCH 057/696] Updated assert == to ===. --- core/src/test/scala/spark/RDDSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ef74c99246728..5a3a12dfff8e3 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -174,9 +174,9 @@ class RDDSuite extends FunSuite with BeforeAndAfter { val data = sc.parallelize(1 to 10, 10) // Note that split number starts from 0, so > 8 means only 10th partition left. val prunedRdd = data.pruneSplits(splitNum => splitNum > 8) - assert(prunedRdd.splits.size == 1) + assert(prunedRdd.splits.size === 1) val prunedData = prunedRdd.collect - assert(prunedData.size == 1) - assert(prunedData(0) == 10) + assert(prunedData.size === 1) + assert(prunedData(0) === 10) } } From 636e912f3289e422be9550752f5279d519062b75 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 21:21:55 -0800 Subject: [PATCH 058/696] Created a PruneDependency to properly assign dependency for SplitsPruningRDD. --- core/src/main/scala/spark/Dependency.scala | 24 ++++++++++++++++--- .../scala/spark/rdd/SplitsPruningRDD.scala | 8 +++---- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index b85d2732db919..7d5858e88ec10 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -5,6 +5,7 @@ package spark */ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable + /** * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. @@ -12,12 +13,13 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. - * @param outputPartition a partition of the child RDD + * @param partitionId a partition of the child RDD * @return the partitions of the parent RDD that the child partition depends upon */ - def getParents(outputPartition: Int): Seq[Int] + def getParents(partitionId: Int): Seq[Int] } + /** * Represents a dependency on the output of a shuffle stage. * @param shuffleId the shuffle id @@ -32,6 +34,7 @@ class ShuffleDependency[K, V]( val shuffleId: Int = rdd.context.newShuffleId() } + /** * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ @@ -39,6 +42,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } + /** * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD @@ -48,7 +52,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { */ class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { - + override def getParents(partitionId: Int) = { if (partitionId >= outStart && partitionId < outStart + length) { List(partitionId - outStart + inStart) @@ -57,3 +61,17 @@ class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) } } } + + +/** + * Represents a dependency between the SplitsPruningRDD and its parent. In this + * case, the child RDD contains a subset of splits of the parents'. + */ +class PruneDependency[T](rdd: RDD[T], @transient splitsFilterFunc: Int => Boolean) + extends NarrowDependency[T](rdd) { + + @transient + val splits: Array[Split] = rdd.splits.filter(s => splitsFilterFunc(s.index)) + + override def getParents(partitionId: Int) = List(splits(partitionId).index) +} diff --git a/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala b/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala index 74e10265fc54c..7b44d85bb5127 100644 --- a/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala @@ -1,6 +1,6 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext} +import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} /** * A RDD used to prune RDD splits so we can avoid launching tasks on @@ -11,12 +11,12 @@ import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext} class SplitsPruningRDD[T: ClassManifest]( prev: RDD[T], @transient splitsFilterFunc: Int => Boolean) - extends RDD[T](prev) { + extends RDD[T](prev.context, List(new PruneDependency(prev, splitsFilterFunc))) { @transient - val _splits: Array[Split] = prev.splits.filter(s => splitsFilterFunc(s.index)) + val _splits: Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].splits - override def compute(split: Split, context: TaskContext) = prev.iterator(split, context) + override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context) override protected def getSplits = _splits From 81004b967e838fca0790727a3fea5a265ddbc69a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 21:54:27 -0800 Subject: [PATCH 059/696] Marked prev RDD as transient in SplitsPruningRDD. --- core/src/main/scala/spark/rdd/SplitsPruningRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala b/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala index 7b44d85bb5127..9b1a210ba3dc0 100644 --- a/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala @@ -9,7 +9,7 @@ import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} * on splits that don't have the range covering the key. */ class SplitsPruningRDD[T: ClassManifest]( - prev: RDD[T], + @transient prev: RDD[T], @transient splitsFilterFunc: Int => Boolean) extends RDD[T](prev.context, List(new PruneDependency(prev, splitsFilterFunc))) { @@ -20,5 +20,5 @@ class SplitsPruningRDD[T: ClassManifest]( override protected def getSplits = _splits - override val partitioner = prev.partitioner + override val partitioner = firstParent[T].partitioner } From eedc542a0276a5248c81446ee84f56d691e5f488 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 22:14:23 -0800 Subject: [PATCH 060/696] Removed pruneSplits method in RDD and renamed SplitsPruningRDD to PartitionPruningRDD. --- core/src/main/scala/spark/RDD.scala | 10 -------- .../scala/spark/rdd/PartitionPruningRDD.scala | 24 +++++++++++++++++++ .../scala/spark/rdd/SplitsPruningRDD.scala | 24 ------------------- core/src/test/scala/spark/RDDSuite.scala | 6 ++--- 4 files changed, 27 insertions(+), 37 deletions(-) create mode 100644 core/src/main/scala/spark/rdd/PartitionPruningRDD.scala delete mode 100644 core/src/main/scala/spark/rdd/SplitsPruningRDD.scala diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 3d93ff33bbf7b..e0d2eabb1dd65 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -40,7 +40,6 @@ import spark.rdd.MapPartitionsRDD import spark.rdd.MapPartitionsWithSplitRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD -import spark.rdd.SplitsPruningRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD import spark.storage.StorageLevel @@ -544,15 +543,6 @@ abstract class RDD[T: ClassManifest]( map(x => (f(x), x)) } - /** - * Prune splits (partitions) so Spark can avoid launching tasks on - * all splits. An example use case: If we know the RDD is partitioned by range, - * and the execution DAG has a filter on the key, we can avoid launching tasks - * on splits that don't have the range covering the key. - */ - def pruneSplits(splitsFilterFunc: Int => Boolean): RDD[T] = - new SplitsPruningRDD(this, splitsFilterFunc) - /** A private method for tests, to look at the contents of each partition */ private[spark] def collectPartitions(): Array[Array[T]] = { sc.runJob(this, (iter: Iterator[T]) => iter.toArray) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala new file mode 100644 index 0000000000000..3048949ef2f87 --- /dev/null +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -0,0 +1,24 @@ +package spark.rdd + +import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} + +/** + * A RDD used to prune RDD partitions/splits so we can avoid launching tasks on + * all partitions. An example use case: If we know the RDD is partitioned by range, + * and the execution DAG has a filter on the key, we can avoid launching tasks + * on partitions that don't have the range covering the key. + */ +class PartitionPruningRDD[T: ClassManifest]( + @transient prev: RDD[T], + @transient partitionFilterFunc: Int => Boolean) + extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { + + @transient + val partitions_ : Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].splits + + override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context) + + override protected def getSplits = partitions_ + + override val partitioner = firstParent[T].partitioner +} diff --git a/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala b/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala deleted file mode 100644 index 9b1a210ba3dc0..0000000000000 --- a/core/src/main/scala/spark/rdd/SplitsPruningRDD.scala +++ /dev/null @@ -1,24 +0,0 @@ -package spark.rdd - -import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} - -/** - * A RDD used to prune RDD splits so we can avoid launching tasks on - * all splits. An example use case: If we know the RDD is partitioned by range, - * and the execution DAG has a filter on the key, we can avoid launching tasks - * on splits that don't have the range covering the key. - */ -class SplitsPruningRDD[T: ClassManifest]( - @transient prev: RDD[T], - @transient splitsFilterFunc: Int => Boolean) - extends RDD[T](prev.context, List(new PruneDependency(prev, splitsFilterFunc))) { - - @transient - val _splits: Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].splits - - override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context) - - override protected def getSplits = _splits - - override val partitioner = firstParent[T].partitioner -} diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 5a3a12dfff8e3..73846131a9dfc 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -3,7 +3,7 @@ package spark import scala.collection.mutable.HashMap import org.scalatest.{BeforeAndAfter, FunSuite} import spark.SparkContext._ -import spark.rdd.CoalescedRDD +import spark.rdd.{CoalescedRDD, PartitionPruningRDD} class RDDSuite extends FunSuite with BeforeAndAfter { @@ -169,11 +169,11 @@ class RDDSuite extends FunSuite with BeforeAndAfter { } } - test("split pruning") { + test("partition pruning") { sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) // Note that split number starts from 0, so > 8 means only 10th partition left. - val prunedRdd = data.pruneSplits(splitNum => splitNum > 8) + val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8) assert(prunedRdd.splits.size === 1) val prunedData = prunedRdd.collect assert(prunedData.size === 1) From c109f29c97c9606dee45e6300d01a272dbb560aa Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 22:22:03 -0800 Subject: [PATCH 061/696] Updated PruneDependency to change "split" to "partition". --- core/src/main/scala/spark/Dependency.scala | 10 +++++----- .../src/main/scala/spark/rdd/PartitionPruningRDD.scala | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 7d5858e88ec10..647aee6eb54cc 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -64,14 +64,14 @@ class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) /** - * Represents a dependency between the SplitsPruningRDD and its parent. In this - * case, the child RDD contains a subset of splits of the parents'. + * Represents a dependency between the PartitionPruningRDD and its parent. In this + * case, the child RDD contains a subset of partitions of the parents'. */ -class PruneDependency[T](rdd: RDD[T], @transient splitsFilterFunc: Int => Boolean) +class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends NarrowDependency[T](rdd) { @transient - val splits: Array[Split] = rdd.splits.filter(s => splitsFilterFunc(s.index)) + val partitions: Array[Split] = rdd.splits.filter(s => partitionFilterFunc(s.index)) - override def getParents(partitionId: Int) = List(splits(partitionId).index) + override def getParents(partitionId: Int) = List(partitions(partitionId).index) } diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 3048949ef2f87..787b59ae8c7b4 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -14,7 +14,7 @@ class PartitionPruningRDD[T: ClassManifest]( extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { @transient - val partitions_ : Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].splits + val partitions_ : Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].partitions override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context) From 67a43bc7e622e4dd9d53ccf80b441740d6ff4df5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 23 Jan 2013 23:06:52 -0800 Subject: [PATCH 062/696] Added a clearDependencies method in PartitionPruningRDD. --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 787b59ae8c7b4..97dd37950e018 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -14,11 +14,16 @@ class PartitionPruningRDD[T: ClassManifest]( extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { @transient - val partitions_ : Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].partitions + var partitions_ : Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].partitions override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context) override protected def getSplits = partitions_ override val partitioner = firstParent[T].partitioner + + override def clearDependencies() { + super.clearDependencies() + partitions_ = null + } } From 230bda204778e6f3c0f5a20ad341f643146d97cb Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 15 Jan 2013 14:01:19 -0600 Subject: [PATCH 063/696] Add LocalSparkContext to manage common sc variable. --- .../test/scala/spark/AccumulatorSuite.scala | 32 ++------ .../src/test/scala/spark/BroadcastSuite.scala | 14 +--- .../test/scala/spark/CheckpointSuite.scala | 19 ++--- .../scala/spark/ClosureCleanerSuite.scala | 73 +++++++++---------- .../test/scala/spark/DistributedSuite.scala | 23 ++---- core/src/test/scala/spark/FailureSuite.scala | 14 +--- .../test/scala/spark/FileServerSuite.scala | 16 ++-- core/src/test/scala/spark/FileSuite.scala | 16 +--- .../test/scala/spark/LocalSparkContext.scala | 41 +++++++++++ .../scala/spark/MapOutputTrackerSuite.scala | 7 +- .../test/scala/spark/PartitioningSuite.scala | 15 +--- core/src/test/scala/spark/PipedRDDSuite.scala | 16 +--- core/src/test/scala/spark/RDDSuite.scala | 14 +--- core/src/test/scala/spark/ShuffleSuite.scala | 14 +--- core/src/test/scala/spark/SortingSuite.scala | 13 +--- .../src/test/scala/spark/ThreadingSuite.scala | 14 +--- .../spark/scheduler/TaskContextSuite.scala | 14 +--- 17 files changed, 109 insertions(+), 246 deletions(-) create mode 100644 core/src/test/scala/spark/LocalSparkContext.scala diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index d8be99dde71f8..78d64a44aed0c 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -1,6 +1,5 @@ package spark -import org.scalatest.BeforeAndAfter import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers import collection.mutable @@ -9,18 +8,7 @@ import scala.math.exp import scala.math.signum import spark.SparkContext._ -class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { - - var sc: SparkContext = null - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext { test ("basic accumulation"){ sc = new SparkContext("local", "test") @@ -53,10 +41,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter for (i <- 1 to maxI) { v should contain(i) } - sc.stop() - sc = null - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + resetSparkContext() } } @@ -86,10 +71,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter x => acc.value += x } } should produce [SparkException] - sc.stop() - sc = null - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + resetSparkContext() } } @@ -115,10 +97,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter bufferAcc.value should contain(i) mapAcc.value should contain (i -> i.toString) } - sc.stop() - sc = null - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + resetSparkContext() } } @@ -134,8 +113,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter x => acc.localValue ++= x } acc.value should be ( (0 to maxI).toSet) - sc.stop() - sc = null + resetSparkContext() } } diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/spark/BroadcastSuite.scala index 2d3302f0aa228..362a31fb0d971 100644 --- a/core/src/test/scala/spark/BroadcastSuite.scala +++ b/core/src/test/scala/spark/BroadcastSuite.scala @@ -1,20 +1,8 @@ package spark import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter -class BroadcastSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class BroadcastSuite extends FunSuite with LocalSparkContext { test("basic broadcast") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 51573254cac7d..33c317720c651 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -1,34 +1,27 @@ package spark -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.FunSuite import java.io.File import spark.rdd._ import spark.SparkContext._ import storage.StorageLevel -class CheckpointSuite extends FunSuite with BeforeAndAfter with Logging { +class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { initLogging() - var sc: SparkContext = _ var checkpointDir: File = _ val partitioner = new HashPartitioner(2) - before { + override def beforeEach() { + super.beforeEach() checkpointDir = File.createTempFile("temp", "") checkpointDir.delete() - sc = new SparkContext("local", "test") sc.setCheckpointDir(checkpointDir.toString) } - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - + override def afterEach() { + super.afterEach() if (checkpointDir != null) { checkpointDir.delete() } diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/spark/ClosureCleanerSuite.scala index dfa2de80e6533..b2d0dd4627e06 100644 --- a/core/src/test/scala/spark/ClosureCleanerSuite.scala +++ b/core/src/test/scala/spark/ClosureCleanerSuite.scala @@ -3,6 +3,7 @@ package spark import java.io.NotSerializableException import org.scalatest.FunSuite +import spark.LocalSparkContext._ import SparkContext._ class ClosureCleanerSuite extends FunSuite { @@ -43,13 +44,10 @@ object TestObject { def run(): Int = { var nonSer = new NonSerializable var x = 5 - val sc = new SparkContext("local", "test") - val nums = sc.parallelize(Array(1, 2, 3, 4)) - val answer = nums.map(_ + x).reduce(_ + _) - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - return answer + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + x).reduce(_ + _) + } } } @@ -60,11 +58,10 @@ class TestClass extends Serializable { def run(): Int = { var nonSer = new NonSerializable - val sc = new SparkContext("local", "test") - val nums = sc.parallelize(Array(1, 2, 3, 4)) - val answer = nums.map(_ + getX).reduce(_ + _) - sc.stop() - return answer + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + getX).reduce(_ + _) + } } } @@ -73,11 +70,10 @@ class TestClassWithoutDefaultConstructor(x: Int) extends Serializable { def run(): Int = { var nonSer = new NonSerializable - val sc = new SparkContext("local", "test") - val nums = sc.parallelize(Array(1, 2, 3, 4)) - val answer = nums.map(_ + getX).reduce(_ + _) - sc.stop() - return answer + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + getX).reduce(_ + _) + } } } @@ -89,11 +85,10 @@ class TestClassWithoutFieldAccess { def run(): Int = { var nonSer2 = new NonSerializable var x = 5 - val sc = new SparkContext("local", "test") - val nums = sc.parallelize(Array(1, 2, 3, 4)) - val answer = nums.map(_ + x).reduce(_ + _) - sc.stop() - return answer + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + x).reduce(_ + _) + } } } @@ -102,16 +97,16 @@ object TestObjectWithNesting { def run(): Int = { var nonSer = new NonSerializable var answer = 0 - val sc = new SparkContext("local", "test") - val nums = sc.parallelize(Array(1, 2, 3, 4)) - var y = 1 - for (i <- 1 to 4) { - var nonSer2 = new NonSerializable - var x = i - answer += nums.map(_ + x + y).reduce(_ + _) + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + var y = 1 + for (i <- 1 to 4) { + var nonSer2 = new NonSerializable + var x = i + answer += nums.map(_ + x + y).reduce(_ + _) + } + answer } - sc.stop() - return answer } } @@ -121,14 +116,14 @@ class TestClassWithNesting(val y: Int) extends Serializable { def run(): Int = { var nonSer = new NonSerializable var answer = 0 - val sc = new SparkContext("local", "test") - val nums = sc.parallelize(Array(1, 2, 3, 4)) - for (i <- 1 to 4) { - var nonSer2 = new NonSerializable - var x = i - answer += nums.map(_ + x + getY).reduce(_ + _) + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + for (i <- 1 to 4) { + var nonSer2 = new NonSerializable + var x = i + answer += nums.map(_ + x + getY).reduce(_ + _) + } + answer } - sc.stop() - return answer } } diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index cacc2796b6fb1..83a2a549a92e6 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -15,41 +15,28 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ import storage.StorageLevel -class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { +class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" - @transient var sc: SparkContext = _ - after { - if (sc != null) { - sc.stop() - sc = null - } System.clearProperty("spark.reducer.maxMbInFlight") System.clearProperty("spark.storage.memoryFraction") - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") } test("local-cluster format") { sc = new SparkContext("local-cluster[2,1,512]", "test") assert(sc.parallelize(1 to 2, 2).count() == 2) - sc.stop() - System.clearProperty("spark.master.port") + resetSparkContext() sc = new SparkContext("local-cluster[2 , 1 , 512]", "test") assert(sc.parallelize(1 to 2, 2).count() == 2) - sc.stop() - System.clearProperty("spark.master.port") + resetSparkContext() sc = new SparkContext("local-cluster[2, 1, 512]", "test") assert(sc.parallelize(1 to 2, 2).count() == 2) - sc.stop() - System.clearProperty("spark.master.port") + resetSparkContext() sc = new SparkContext("local-cluster[ 2, 1, 512 ]", "test") assert(sc.parallelize(1 to 2, 2).count() == 2) - sc.stop() - System.clearProperty("spark.master.port") - sc = null + resetSparkContext() } test("simple groupByKey") { diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index a3454f25f6f81..8c1445a4656af 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -1,7 +1,6 @@ package spark import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import org.scalatest.prop.Checkers import scala.collection.mutable.ArrayBuffer @@ -23,18 +22,7 @@ object FailureSuiteState { } } -class FailureSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class FailureSuite extends FunSuite with LocalSparkContext { // Run a 3-task map job in which task 1 deterministically fails once, and check // whether the job completes successfully and we ran 4 tasks in total. diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index b4283d960488b..8215cbde02898 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -2,17 +2,16 @@ package spark import com.google.common.io.Files import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import java.io.{File, PrintWriter, FileReader, BufferedReader} import SparkContext._ -class FileServerSuite extends FunSuite with BeforeAndAfter { +class FileServerSuite extends FunSuite with LocalSparkContext { - @transient var sc: SparkContext = _ @transient var tmpFile : File = _ @transient var testJarFile : File = _ - before { + override def beforeEach() { + super.beforeEach() // Create a sample text file val tmpdir = new File(Files.createTempDir(), "test") tmpdir.mkdir() @@ -22,17 +21,12 @@ class FileServerSuite extends FunSuite with BeforeAndAfter { pw.close() } - after { - if (sc != null) { - sc.stop() - sc = null - } + override def afterEach() { + super.afterEach() // Clean up downloaded file if (tmpFile.exists) { tmpFile.delete() } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") } test("Distributing files locally") { diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index 554bea53a9181..91b48c745659b 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -6,24 +6,12 @@ import scala.io.Source import com.google.common.io.Files import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import org.apache.hadoop.io._ import SparkContext._ -class FileSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } - +class FileSuite extends FunSuite with LocalSparkContext { + test("text files") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala new file mode 100644 index 0000000000000..b5e31ddae3a9f --- /dev/null +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -0,0 +1,41 @@ +package spark + +import org.scalatest.Suite +import org.scalatest.BeforeAndAfterEach + +/** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ +trait LocalSparkContext extends BeforeAndAfterEach { self: Suite => + + @transient var sc: SparkContext = _ + + override def afterEach() { + resetSparkContext() + super.afterEach() + } + + def resetSparkContext() = { + if (sc != null) { + LocalSparkContext.stop(sc) + sc = null + } + } + +} + +object LocalSparkContext { + def stop(sc: SparkContext) { + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.master.port") + } + + /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ + def withSpark[T](sc: SparkContext)(f: SparkContext => T) = { + try { + f(sc) + } finally { + stop(sc) + } + } + +} \ No newline at end of file diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index d3dd3a8fa4930..774bbd65b199c 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -1,17 +1,13 @@ package spark import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import akka.actor._ import spark.scheduler.MapStatus import spark.storage.BlockManagerId import spark.util.AkkaUtils -class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter { - after { - System.clearProperty("spark.master.port") - } +class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("compressSize") { assert(MapOutputTracker.compressSize(0L) === 0) @@ -81,7 +77,6 @@ class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter { } test("remote fetch") { - System.clearProperty("spark.master.host") val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0) System.setProperty("spark.master.port", boundPort.toString) diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index eb3c8f238f187..af1107cd197b4 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -1,25 +1,12 @@ package spark import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import scala.collection.mutable.ArrayBuffer import SparkContext._ -class PartitioningSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if(sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } - +class PartitioningSuite extends FunSuite with LocalSparkContext { test("HashPartitioner equality") { val p2 = new HashPartitioner(2) diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index 9b84b29227933..a6344edf8f161 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -1,21 +1,9 @@ package spark import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import SparkContext._ -class PipedRDDSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class PipedRDDSuite extends FunSuite with LocalSparkContext { test("basic pipe") { sc = new SparkContext("local", "test") @@ -51,5 +39,3 @@ class PipedRDDSuite extends FunSuite with BeforeAndAfter { } } - - diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index db217f8482510..592427e97a526 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -2,23 +2,11 @@ package spark import scala.collection.mutable.HashMap import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import spark.rdd.CoalescedRDD import SparkContext._ -class RDDSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class RDDSuite extends FunSuite with LocalSparkContext { test("basic operations") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index bebb8ebe86d65..3493b9511f6c2 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -3,7 +3,6 @@ package spark import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers import org.scalatest.prop.Checkers import org.scalacheck.Arbitrary._ @@ -15,18 +14,7 @@ import com.google.common.io.Files import spark.rdd.ShuffledRDD import spark.SparkContext._ -class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { test("groupByKey") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index 1ad11ff4c3df7..edb8c839fcb70 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -5,18 +5,7 @@ import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers import SparkContext._ -class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with Logging { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers with Logging { test("sortByKey") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/spark/ThreadingSuite.scala index e9b1837d89404..ff315b66935dc 100644 --- a/core/src/test/scala/spark/ThreadingSuite.scala +++ b/core/src/test/scala/spark/ThreadingSuite.scala @@ -22,19 +22,7 @@ object ThreadingSuiteState { } } -class ThreadingSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if(sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } - +class ThreadingSuite extends FunSuite with LocalSparkContext { test("accessing SparkContext form a different thread") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index ba6f8b588ff82..a5db7103f5ce8 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -6,19 +6,9 @@ import spark.TaskContext import spark.RDD import spark.SparkContext import spark.Split +import spark.LocalSparkContext -class TaskContextSuite extends FunSuite with BeforeAndAfter { - - var sc: SparkContext = _ - - after { - if (sc != null) { - sc.stop() - sc = null - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") - } +class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { test("Calls executeOnCompleteCallbacks after failure") { var completed = false From b6fc6e67521e8a9a5291693cce3dc766da244395 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 24 Jan 2013 14:28:05 -0800 Subject: [PATCH 064/696] SPARK-541: Adding a warning for invalid Master URL Right now Spark silently parses master URL's which do not match any known regex as a Mesos URL. The Mesos error message when an invalid URL gets passed is really confusing, so this warns the user when the implicit conversion is happening. --- core/src/main/scala/spark/SparkContext.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 66bdbe7cdae96..bc9fdee8b66ab 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -112,6 +112,8 @@ class SparkContext( val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r // Regular expression for connecting to Spark deploy clusters val SPARK_REGEX = """(spark://.*)""".r + //Regular expression for connection to Mesos cluster + val MESOS_REGEX = """(mesos://.*)""".r master match { case "local" => @@ -152,6 +154,9 @@ class SparkContext( scheduler case _ => + if (MESOS_REGEX.findFirstIn(master).isEmpty) { + logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master)) + } MesosNativeLibrary.load() val scheduler = new ClusterScheduler(this) val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean From 4496bf197bb1137cc1224d8e5cb5f55625dd5851 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 25 Jan 2013 14:34:38 +0530 Subject: [PATCH 065/696] Improved document comment in example --- .../scala/spark/streaming/examples/ActorWordCount.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index c3d3755953242..0ffa45a4c6ac2 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -84,12 +84,18 @@ class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) /** * A sample word count program demonstrating the use of plugging in * Actor as Receiver + * Usage: ActorWordCount + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and describe the AkkaSystem that Spark Sample feeder would work on. + * + * and then run the example + * `$ ./run spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { if (args.length < 3) { System.err.println( - "Usage: ActorWordCount " + + "Usage: ActorWordCount " + "In local mode, should be 'local[n]' with n > 1") System.exit(1) } From 7dfb82a992d47491174d7929e31351d26cadfcda Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 22 Jan 2013 15:25:41 -0600 Subject: [PATCH 066/696] Replace old 'master' term with 'driver'. --- bagel/src/test/scala/bagel/BagelSuite.scala | 2 +- .../main/scala/spark/MapOutputTracker.scala | 10 +-- core/src/main/scala/spark/SparkContext.scala | 20 ++--- core/src/main/scala/spark/SparkEnv.scala | 22 +++--- .../spark/broadcast/BitTorrentBroadcast.scala | 24 +++--- .../scala/spark/broadcast/Broadcast.scala | 6 +- .../spark/broadcast/BroadcastFactory.scala | 4 +- .../scala/spark/broadcast/HttpBroadcast.scala | 6 +- .../scala/spark/broadcast/MultiTracker.scala | 35 +++++---- .../scala/spark/broadcast/TreeBroadcast.scala | 52 ++++++------- .../spark/deploy/LocalSparkCluster.scala | 34 ++++----- .../spark/deploy/client/ClientListener.scala | 4 +- .../scala/spark/deploy/master/JobInfo.scala | 2 +- .../scala/spark/deploy/master/Master.scala | 18 ++--- .../executor/StandaloneExecutorBackend.scala | 26 +++---- .../cluster/SparkDeploySchedulerBackend.scala | 33 +++++---- .../cluster/StandaloneClusterMessage.scala | 8 +- .../cluster/StandaloneSchedulerBackend.scala | 74 +++++++++---------- .../mesos/CoarseMesosSchedulerBackend.scala | 6 +- .../spark/storage/BlockManagerMaster.scala | 69 +++++++++-------- .../scala/spark/storage/ThreadingTest.scala | 6 +- core/src/test/scala/spark/JavaAPISuite.java | 2 +- .../test/scala/spark/LocalSparkContext.scala | 2 +- .../scala/spark/MapOutputTrackerSuite.scala | 2 +- docs/configuration.md | 12 +-- python/pyspark/tests.py | 2 +- .../src/test/scala/spark/repl/ReplSuite.scala | 2 +- .../dstream/NetworkInputDStream.scala | 4 +- .../java/spark/streaming/JavaAPISuite.java | 2 +- .../streaming/BasicOperationsSuite.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 +- .../scala/spark/streaming/FailureSuite.scala | 2 +- .../spark/streaming/InputStreamsSuite.scala | 2 +- .../streaming/WindowOperationsSuite.scala | 2 +- 34 files changed, 248 insertions(+), 251 deletions(-) diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala index ca59f46843798..3c2f9c4616fe2 100644 --- a/bagel/src/test/scala/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/bagel/BagelSuite.scala @@ -23,7 +23,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter { sc = null } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") } test("halting by voting") { diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index ac02f3363afa8..d4f5164f7d968 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -38,10 +38,7 @@ private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Ac } } -private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logging { - val ip: String = System.getProperty("spark.master.host", "localhost") - val port: Int = System.getProperty("spark.master.port", "7077").toInt - val actorName: String = "MapOutputTracker" +private[spark] class MapOutputTracker(actorSystem: ActorSystem, isDriver: Boolean) extends Logging { val timeout = 10.seconds @@ -56,11 +53,14 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea var cacheGeneration = generation val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] - var trackerActor: ActorRef = if (isMaster) { + val actorName: String = "MapOutputTracker" + var trackerActor: ActorRef = if (isDriver) { val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName) logInfo("Registered MapOutputTrackerActor actor") actor } else { + val ip = System.getProperty("spark.driver.host", "localhost") + val port = System.getProperty("spark.driver.port", "7077").toInt val url = "akka://spark@%s:%s/user/%s".format(ip, port, actorName) actorSystem.actorFor(url) } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index bc9fdee8b66ab..d4991cb1e0992 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -66,20 +66,20 @@ class SparkContext( // Ensure logging is initialized before we spawn any threads initLogging() - // Set Spark master host and port system properties - if (System.getProperty("spark.master.host") == null) { - System.setProperty("spark.master.host", Utils.localIpAddress) + // Set Spark driver host and port system properties + if (System.getProperty("spark.driver.host") == null) { + System.setProperty("spark.driver.host", Utils.localIpAddress) } - if (System.getProperty("spark.master.port") == null) { - System.setProperty("spark.master.port", "0") + if (System.getProperty("spark.driver.port") == null) { + System.setProperty("spark.driver.port", "0") } private val isLocal = (master == "local" || master.startsWith("local[")) // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( - System.getProperty("spark.master.host"), - System.getProperty("spark.master.port").toInt, + System.getProperty("spark.driver.host"), + System.getProperty("spark.driver.port").toInt, true, isLocal) SparkEnv.set(env) @@ -396,14 +396,14 @@ class SparkContext( /** * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values - * to using the `+=` method. Only the master can access the accumulator's `value`. + * to using the `+=` method. Only the driver can access the accumulator's `value`. */ def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = new Accumulator(initialValue, param) /** * Create an [[spark.Accumulable]] shared variable, to which tasks can add values with `+=`. - * Only the master can access the accumuable's `value`. + * Only the driver can access the accumuable's `value`. * @tparam T accumulator type * @tparam R type that can be added to the accumulator */ @@ -530,7 +530,7 @@ class SparkContext( /** * Run a function on a given set of partitions in an RDD and return the results. This is the main * entry point to the scheduler, by which all actions get launched. The allowLocal flag specifies - * whether the scheduler can run the computation on the master rather than shipping it out to the + * whether the scheduler can run the computation on the driver rather than shipping it out to the * cluster, for short actions like first(). */ def runJob[T, U: ClassManifest]( diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 2a7a8af83d3e8..4034af610c5c0 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -60,15 +60,15 @@ object SparkEnv extends Logging { def createFromSystemProperties( hostname: String, port: Int, - isMaster: Boolean, + isDriver: Boolean, isLocal: Boolean ) : SparkEnv = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port) - // Bit of a hack: If this is the master and our port was 0 (meaning bind to any free port), - // figure out which port number Akka actually bound to and set spark.master.port to it. - if (isMaster && port == 0) { - System.setProperty("spark.master.port", boundPort.toString) + // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port), + // figure out which port number Akka actually bound to and set spark.driver.port to it. + if (isDriver && port == 0) { + System.setProperty("spark.driver.port", boundPort.toString) } val classLoader = Thread.currentThread.getContextClassLoader @@ -82,22 +82,22 @@ object SparkEnv extends Logging { val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") - val masterIp: String = System.getProperty("spark.master.host", "localhost") - val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt + val driverIp: String = System.getProperty("spark.driver.host", "localhost") + val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt val blockManagerMaster = new BlockManagerMaster( - actorSystem, isMaster, isLocal, masterIp, masterPort) + actorSystem, isDriver, isLocal, driverIp, driverPort) val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer) val connectionManager = blockManager.connectionManager - val broadcastManager = new BroadcastManager(isMaster) + val broadcastManager = new BroadcastManager(isDriver) val closureSerializer = instantiateClass[Serializer]( "spark.closure.serializer", "spark.JavaSerializer") val cacheManager = new CacheManager(blockManager) - val mapOutputTracker = new MapOutputTracker(actorSystem, isMaster) + val mapOutputTracker = new MapOutputTracker(actorSystem, isDriver) val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher") @@ -109,7 +109,7 @@ object SparkEnv extends Logging { // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working // directory. - val sparkFilesDir: String = if (isMaster) { + val sparkFilesDir: String = if (isDriver) { Utils.createTempDir().getAbsolutePath } else { "." diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala index 386f505f2a9db..adcb2d2415eb3 100644 --- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala @@ -31,7 +31,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: @transient var totalBlocks = -1 @transient var hasBlocks = new AtomicInteger(0) - // Used ONLY by Master to track how many unique blocks have been sent out + // Used ONLY by driver to track how many unique blocks have been sent out @transient var sentBlocks = new AtomicInteger(0) @transient var listenPortLock = new Object @@ -42,7 +42,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: @transient var serveMR: ServeMultipleRequests = null - // Used only in Master + // Used only in driver @transient var guideMR: GuideMultipleRequests = null // Used only in Workers @@ -99,14 +99,14 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: } // Must always come AFTER listenPort is created - val masterSource = + val driverSource = SourceInfo(hostAddress, listenPort, totalBlocks, totalBytes) hasBlocksBitVector.synchronized { - masterSource.hasBlocksBitVector = hasBlocksBitVector + driverSource.hasBlocksBitVector = hasBlocksBitVector } // In the beginning, this is the only known source to Guide - listOfSources += masterSource + listOfSources += driverSource // Register with the Tracker MultiTracker.registerBroadcast(id, @@ -122,7 +122,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: case None => logInfo("Started reading broadcast variable " + id) - // Initializing everything because Master will only send null/0 values + // Initializing everything because driver will only send null/0 values // Only the 1st worker in a node can be here. Others will get from cache initializeWorkerVariables() @@ -151,7 +151,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: } } - // Initialize variables in the worker node. Master sends everything as 0/null + // Initialize variables in the worker node. Driver sends everything as 0/null private def initializeWorkerVariables() { arrayOfBlocks = null hasBlocksBitVector = null @@ -248,7 +248,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: // Receive source information from Guide var suitableSources = oisGuide.readObject.asInstanceOf[ListBuffer[SourceInfo]] - logDebug("Received suitableSources from Master " + suitableSources) + logDebug("Received suitableSources from Driver " + suitableSources) addToListOfSources(suitableSources) @@ -532,7 +532,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: oosSource.writeObject(blockToAskFor) oosSource.flush() - // CHANGED: Master might send some other block than the one + // CHANGED: Driver might send some other block than the one // requested to ensure fast spreading of all blocks. val recvStartTime = System.currentTimeMillis val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] @@ -982,9 +982,9 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: // Receive which block to send var blockToSend = ois.readObject.asInstanceOf[Int] - // If it is master AND at least one copy of each block has not been + // If it is driver AND at least one copy of each block has not been // sent out already, MODIFY blockToSend - if (MultiTracker.isMaster && sentBlocks.get < totalBlocks) { + if (MultiTracker.isDriver && sentBlocks.get < totalBlocks) { blockToSend = sentBlocks.getAndIncrement } @@ -1031,7 +1031,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: private[spark] class BitTorrentBroadcastFactory extends BroadcastFactory { - def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) } + def initialize(isDriver: Boolean) { MultiTracker.initialize(isDriver) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new BitTorrentBroadcast[T](value_, isLocal, id) diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala index 2ffe7f741d8d0..415bde5d677cb 100644 --- a/core/src/main/scala/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/spark/broadcast/Broadcast.scala @@ -15,7 +15,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { } private[spark] -class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializable { +class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable { private var initialized = false private var broadcastFactory: BroadcastFactory = null @@ -33,7 +33,7 @@ class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializabl Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isMaster) + broadcastFactory.initialize(isDriver) initialized = true } @@ -49,5 +49,5 @@ class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializabl def newBroadcast[T](value_ : T, isLocal: Boolean) = broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) - def isMaster = isMaster_ + def isDriver = _isDriver } diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala index ab6d302827d7d..5c6184c3c7159 100644 --- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala @@ -7,7 +7,7 @@ package spark.broadcast * entire Spark job. */ private[spark] trait BroadcastFactory { - def initialize(isMaster: Boolean): Unit - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] + def initialize(isDriver: Boolean): Unit + def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] def stop(): Unit } diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala index 8e490e6bad8ba..7e30b8f7d21b2 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala @@ -48,7 +48,7 @@ extends Broadcast[T](id) with Logging with Serializable { } private[spark] class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isMaster: Boolean) { HttpBroadcast.initialize(isMaster) } + def initialize(isDriver: Boolean) { HttpBroadcast.initialize(isDriver) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) @@ -69,12 +69,12 @@ private object HttpBroadcast extends Logging { private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup) - def initialize(isMaster: Boolean) { + def initialize(isDriver: Boolean) { synchronized { if (!initialized) { bufferSize = System.getProperty("spark.buffer.size", "65536").toInt compress = System.getProperty("spark.broadcast.compress", "true").toBoolean - if (isMaster) { + if (isDriver) { createServer() } serverUri = System.getProperty("spark.httpBroadcast.uri") diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala index 5e76dedb942db..3fd77af73f176 100644 --- a/core/src/main/scala/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala @@ -23,25 +23,24 @@ extends Logging { var ranGen = new Random private var initialized = false - private var isMaster_ = false + private var _isDriver = false private var stopBroadcast = false private var trackMV: TrackMultipleValues = null - def initialize(isMaster__ : Boolean) { + def initialize(__isDriver: Boolean) { synchronized { if (!initialized) { + _isDriver = __isDriver - isMaster_ = isMaster__ - - if (isMaster) { + if (isDriver) { trackMV = new TrackMultipleValues trackMV.setDaemon(true) trackMV.start() - // Set masterHostAddress to the master's IP address for the slaves to read - System.setProperty("spark.MultiTracker.MasterHostAddress", Utils.localIpAddress) + // Set DriverHostAddress to the driver's IP address for the slaves to read + System.setProperty("spark.MultiTracker.DriverHostAddress", Utils.localIpAddress) } initialized = true @@ -54,10 +53,10 @@ extends Logging { } // Load common parameters - private var MasterHostAddress_ = System.getProperty( - "spark.MultiTracker.MasterHostAddress", "") - private var MasterTrackerPort_ = System.getProperty( - "spark.broadcast.masterTrackerPort", "11111").toInt + private var DriverHostAddress_ = System.getProperty( + "spark.MultiTracker.DriverHostAddress", "") + private var DriverTrackerPort_ = System.getProperty( + "spark.broadcast.driverTrackerPort", "11111").toInt private var BlockSize_ = System.getProperty( "spark.broadcast.blockSize", "4096").toInt * 1024 private var MaxRetryCount_ = System.getProperty( @@ -91,11 +90,11 @@ extends Logging { private var EndGameFraction_ = System.getProperty( "spark.broadcast.endGameFraction", "0.95").toDouble - def isMaster = isMaster_ + def isDriver = _isDriver // Common config params - def MasterHostAddress = MasterHostAddress_ - def MasterTrackerPort = MasterTrackerPort_ + def DriverHostAddress = DriverHostAddress_ + def DriverTrackerPort = DriverTrackerPort_ def BlockSize = BlockSize_ def MaxRetryCount = MaxRetryCount_ @@ -123,7 +122,7 @@ extends Logging { var threadPool = Utils.newDaemonCachedThreadPool() var serverSocket: ServerSocket = null - serverSocket = new ServerSocket(MasterTrackerPort) + serverSocket = new ServerSocket(DriverTrackerPort) logInfo("TrackMultipleValues started at " + serverSocket) try { @@ -235,7 +234,7 @@ extends Logging { try { // Connect to the tracker to find out GuideInfo clientSocketToTracker = - new Socket(MultiTracker.MasterHostAddress, MultiTracker.MasterTrackerPort) + new Socket(MultiTracker.DriverHostAddress, MultiTracker.DriverTrackerPort) oosTracker = new ObjectOutputStream(clientSocketToTracker.getOutputStream) oosTracker.flush() @@ -276,7 +275,7 @@ extends Logging { } def registerBroadcast(id: Long, gInfo: SourceInfo) { - val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort) + val socket = new Socket(MultiTracker.DriverHostAddress, DriverTrackerPort) val oosST = new ObjectOutputStream(socket.getOutputStream) oosST.flush() val oisST = new ObjectInputStream(socket.getInputStream) @@ -303,7 +302,7 @@ extends Logging { } def unregisterBroadcast(id: Long) { - val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort) + val socket = new Socket(MultiTracker.DriverHostAddress, DriverTrackerPort) val oosST = new ObjectOutputStream(socket.getOutputStream) oosST.flush() val oisST = new ObjectInputStream(socket.getInputStream) diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala index f573512835d03..c55c4761172db 100644 --- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala @@ -98,7 +98,7 @@ extends Broadcast[T](id) with Logging with Serializable { case None => logInfo("Started reading broadcast variable " + id) - // Initializing everything because Master will only send null/0 values + // Initializing everything because Driver will only send null/0 values // Only the 1st worker in a node can be here. Others will get from cache initializeWorkerVariables() @@ -157,55 +157,55 @@ extends Broadcast[T](id) with Logging with Serializable { listenPortLock.synchronized { listenPortLock.wait() } } - var clientSocketToMaster: Socket = null - var oosMaster: ObjectOutputStream = null - var oisMaster: ObjectInputStream = null + var clientSocketToDriver: Socket = null + var oosDriver: ObjectOutputStream = null + var oisDriver: ObjectInputStream = null // Connect and receive broadcast from the specified source, retrying the // specified number of times in case of failures var retriesLeft = MultiTracker.MaxRetryCount do { - // Connect to Master and send this worker's Information - clientSocketToMaster = new Socket(MultiTracker.MasterHostAddress, gInfo.listenPort) - oosMaster = new ObjectOutputStream(clientSocketToMaster.getOutputStream) - oosMaster.flush() - oisMaster = new ObjectInputStream(clientSocketToMaster.getInputStream) + // Connect to Driver and send this worker's Information + clientSocketToDriver = new Socket(MultiTracker.DriverHostAddress, gInfo.listenPort) + oosDriver = new ObjectOutputStream(clientSocketToDriver.getOutputStream) + oosDriver.flush() + oisDriver = new ObjectInputStream(clientSocketToDriver.getInputStream) - logDebug("Connected to Master's guiding object") + logDebug("Connected to Driver's guiding object") // Send local source information - oosMaster.writeObject(SourceInfo(hostAddress, listenPort)) - oosMaster.flush() + oosDriver.writeObject(SourceInfo(hostAddress, listenPort)) + oosDriver.flush() - // Receive source information from Master - var sourceInfo = oisMaster.readObject.asInstanceOf[SourceInfo] + // Receive source information from Driver + var sourceInfo = oisDriver.readObject.asInstanceOf[SourceInfo] totalBlocks = sourceInfo.totalBlocks arrayOfBlocks = new Array[BroadcastBlock](totalBlocks) totalBlocksLock.synchronized { totalBlocksLock.notifyAll() } totalBytes = sourceInfo.totalBytes - logDebug("Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) + logDebug("Received SourceInfo from Driver:" + sourceInfo + " My Port: " + listenPort) val start = System.nanoTime val receptionSucceeded = receiveSingleTransmission(sourceInfo) val time = (System.nanoTime - start) / 1e9 - // Updating some statistics in sourceInfo. Master will be using them later + // Updating some statistics in sourceInfo. Driver will be using them later if (!receptionSucceeded) { sourceInfo.receptionFailed = true } - // Send back statistics to the Master - oosMaster.writeObject(sourceInfo) + // Send back statistics to the Driver + oosDriver.writeObject(sourceInfo) - if (oisMaster != null) { - oisMaster.close() + if (oisDriver != null) { + oisDriver.close() } - if (oosMaster != null) { - oosMaster.close() + if (oosDriver != null) { + oosDriver.close() } - if (clientSocketToMaster != null) { - clientSocketToMaster.close() + if (clientSocketToDriver != null) { + clientSocketToDriver.close() } retriesLeft -= 1 @@ -552,7 +552,7 @@ extends Broadcast[T](id) with Logging with Serializable { } private def sendObject() { - // Wait till receiving the SourceInfo from Master + // Wait till receiving the SourceInfo from Driver while (totalBlocks == -1) { totalBlocksLock.synchronized { totalBlocksLock.wait() } } @@ -576,7 +576,7 @@ extends Broadcast[T](id) with Logging with Serializable { private[spark] class TreeBroadcastFactory extends BroadcastFactory { - def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) } + def initialize(isDriver: Boolean) { MultiTracker.initialize(isDriver) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new TreeBroadcast[T](value_, isLocal, id) diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 4211d805967a9..ae083efc8db6e 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -10,7 +10,7 @@ import spark.{Logging, Utils} import scala.collection.mutable.ArrayBuffer private[spark] -class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) extends Logging { +class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging { val localIpAddress = Utils.localIpAddress @@ -19,33 +19,31 @@ class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) var masterPort : Int = _ var masterUrl : String = _ - val slaveActorSystems = ArrayBuffer[ActorSystem]() - val slaveActors = ArrayBuffer[ActorRef]() + val workerActorSystems = ArrayBuffer[ActorSystem]() + val workerActors = ArrayBuffer[ActorRef]() def start() : String = { - logInfo("Starting a local Spark cluster with " + numSlaves + " slaves.") + logInfo("Starting a local Spark cluster with " + numWorkers + " workers.") /* Start the Master */ val (actorSystem, masterPort) = AkkaUtils.createActorSystem("sparkMaster", localIpAddress, 0) masterActorSystem = actorSystem masterUrl = "spark://" + localIpAddress + ":" + masterPort - val actor = masterActorSystem.actorOf( + masterActor = masterActorSystem.actorOf( Props(new Master(localIpAddress, masterPort, 0)), name = "Master") - masterActor = actor - /* Start the Slaves */ - for (slaveNum <- 1 to numSlaves) { - /* We can pretend to test distributed stuff by giving the slaves distinct hostnames. + /* Start the Workers */ + for (workerNum <- 1 to numWorkers) { + /* We can pretend to test distributed stuff by giving the workers distinct hostnames. All of 127/8 should be a loopback, we use 127.100.*.* in hopes that it is sufficiently distinctive. */ - val slaveIpAddress = "127.100.0." + (slaveNum % 256) + val workerIpAddress = "127.100.0." + (workerNum % 256) val (actorSystem, boundPort) = - AkkaUtils.createActorSystem("sparkWorker" + slaveNum, slaveIpAddress, 0) - slaveActorSystems += actorSystem - val actor = actorSystem.actorOf( - Props(new Worker(slaveIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)), + AkkaUtils.createActorSystem("sparkWorker" + workerNum, workerIpAddress, 0) + workerActorSystems += actorSystem + workerActors += actorSystem.actorOf( + Props(new Worker(workerIpAddress, boundPort, 0, coresPerWorker, memoryPerWorker, masterUrl)), name = "Worker") - slaveActors += actor } return masterUrl @@ -53,9 +51,9 @@ class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) def stop() { logInfo("Shutting down local Spark cluster.") - // Stop the slaves before the master so they don't get upset that it disconnected - slaveActorSystems.foreach(_.shutdown()) - slaveActorSystems.foreach(_.awaitTermination()) + // Stop the workers before the master so they don't get upset that it disconnected + workerActorSystems.foreach(_.shutdown()) + workerActorSystems.foreach(_.awaitTermination()) masterActorSystem.shutdown() masterActorSystem.awaitTermination() } diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index da6abcc9c26c0..7035f4b394242 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -12,7 +12,7 @@ private[spark] trait ClientListener { def disconnected(): Unit - def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int): Unit + def executorAdded(fullId: String, workerId: String, host: String, cores: Int, memory: Int): Unit - def executorRemoved(id: String, message: String, exitStatus: Option[Int]): Unit + def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit } diff --git a/core/src/main/scala/spark/deploy/master/JobInfo.scala b/core/src/main/scala/spark/deploy/master/JobInfo.scala index 130b031a2af6c..a274b21c346f2 100644 --- a/core/src/main/scala/spark/deploy/master/JobInfo.scala +++ b/core/src/main/scala/spark/deploy/master/JobInfo.scala @@ -10,7 +10,7 @@ private[spark] class JobInfo( val id: String, val desc: JobDescription, val submitDate: Date, - val actor: ActorRef) + val driver: ActorRef) { var state = JobState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 2c2cd0231be70..3347207c6dd58 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -88,7 +88,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor execOption match { case Some(exec) => { exec.state = state - exec.job.actor ! ExecutorUpdated(execId, state, message, exitStatus) + exec.job.driver ! ExecutorUpdated(execId, state, message, exitStatus) if (ExecutorState.isFinished(state)) { val jobInfo = idToJob(jobId) // Remove this executor from the worker and job @@ -199,7 +199,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(exec.job.id, exec.id, exec.job.desc, exec.cores, exec.memory, sparkHome) - exec.job.actor ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory) + exec.job.driver ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory) } def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int, @@ -221,19 +221,19 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor actorToWorker -= worker.actor addressToWorker -= worker.actor.path.address for (exec <- worker.executors.values) { - exec.job.actor ! ExecutorStateChanged(exec.job.id, exec.id, ExecutorState.LOST, None, None) + exec.job.driver ! ExecutorStateChanged(exec.job.id, exec.id, ExecutorState.LOST, None, None) exec.job.executors -= exec.id } } - def addJob(desc: JobDescription, actor: ActorRef): JobInfo = { + def addJob(desc: JobDescription, driver: ActorRef): JobInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val job = new JobInfo(now, newJobId(date), desc, date, actor) + val job = new JobInfo(now, newJobId(date), desc, date, driver) jobs += job idToJob(job.id) = job - actorToJob(sender) = job - addressToJob(sender.path.address) = job + actorToJob(driver) = job + addressToJob(driver.path.address) = job return job } @@ -242,8 +242,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor logInfo("Removing job " + job.id) jobs -= job idToJob -= job.id - actorToJob -= job.actor - addressToWorker -= job.actor.path.address + actorToJob -= job.driver + addressToWorker -= job.driver.path.address completedJobs += job // Remember it in our history waitingJobs -= job for (exec <- job.executors.values) { diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index a29bf974d247e..f80f1b5274176 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -16,33 +16,33 @@ import spark.scheduler.cluster.RegisterSlave private[spark] class StandaloneExecutorBackend( executor: Executor, - masterUrl: String, - slaveId: String, + driverUrl: String, + workerId: String, hostname: String, cores: Int) extends Actor with ExecutorBackend with Logging { - var master: ActorRef = null + var driver: ActorRef = null override def preStart() { try { - logInfo("Connecting to master: " + masterUrl) - master = context.actorFor(masterUrl) - master ! RegisterSlave(slaveId, hostname, cores) + logInfo("Connecting to driver: " + driverUrl) + driver = context.actorFor(driverUrl) + driver ! RegisterSlave(workerId, hostname, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing + context.watch(driver) // Doesn't work with remote actors, but useful for testing } catch { case e: Exception => - logError("Failed to connect to master", e) + logError("Failed to connect to driver", e) System.exit(1) } } override def receive = { case RegisteredSlave(sparkProperties) => - logInfo("Successfully registered with master") + logInfo("Successfully registered with driver") executor.initialize(hostname, sparkProperties) case RegisterSlaveFailed(message) => @@ -55,24 +55,24 @@ private[spark] class StandaloneExecutorBackend( } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { - master ! StatusUpdate(slaveId, taskId, state, data) + driver ! StatusUpdate(workerId, taskId, state, data) } } private[spark] object StandaloneExecutorBackend { - def run(masterUrl: String, slaveId: String, hostname: String, cores: Int) { + def run(driverUrl: String, workerId: String, hostname: String, cores: Int) { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0) val actor = actorSystem.actorOf( - Props(new StandaloneExecutorBackend(new Executor, masterUrl, slaveId, hostname, cores)), + Props(new StandaloneExecutorBackend(new Executor, driverUrl, workerId, hostname, cores)), name = "Executor") actorSystem.awaitTermination() } def main(args: Array[String]) { if (args.length != 4) { - System.err.println("Usage: StandaloneExecutorBackend ") + System.err.println("Usage: StandaloneExecutorBackend ") System.exit(1) } run(args(0), args(1), args(2), args(3).toInt) diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 4f82cd96dd66a..866beb6d01efa 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -19,7 +19,7 @@ private[spark] class SparkDeploySchedulerBackend( var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt - val executorIdToSlaveId = new HashMap[String, String] + val executorIdToWorkerId = new HashMap[String, String] // Memory used by each executor (in megabytes) val executorMemory = { @@ -34,10 +34,11 @@ private[spark] class SparkDeploySchedulerBackend( override def start() { super.start() - val masterUrl = "akka://spark@%s:%s/user/%s".format( - System.getProperty("spark.master.host"), System.getProperty("spark.master.port"), + // The endpoint for executors to talk to us + val driverUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) - val args = Seq(masterUrl, "{{SLAVEID}}", "{{HOSTNAME}}", "{{CORES}}") + val args = Seq(driverUrl, "{{SLAVEID}}", "{{HOSTNAME}}", "{{CORES}}") val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(throw new IllegalArgumentException("must supply spark home for spark standalone")) val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command, sparkHome) @@ -55,35 +56,35 @@ private[spark] class SparkDeploySchedulerBackend( } } - def connected(jobId: String) { + override def connected(jobId: String) { logInfo("Connected to Spark cluster with job ID " + jobId) } - def disconnected() { + override def disconnected() { if (!stopping) { logError("Disconnected from Spark cluster!") scheduler.error("Disconnected from Spark cluster") } } - def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int) { - executorIdToSlaveId += id -> workerId + override def executorAdded(fullId: String, workerId: String, host: String, cores: Int, memory: Int) { + executorIdToWorkerId += fullId -> workerId logInfo("Granted executor ID %s on host %s with %d cores, %s RAM".format( - id, host, cores, Utils.memoryMegabytesToString(memory))) + fullId, host, cores, Utils.memoryMegabytesToString(memory))) } - def executorRemoved(id: String, message: String, exitStatus: Option[Int]) { + override def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]) { val reason: ExecutorLossReason = exitStatus match { case Some(code) => ExecutorExited(code) case None => SlaveLost(message) } - logInfo("Executor %s removed: %s".format(id, message)) - executorIdToSlaveId.get(id) match { - case Some(slaveId) => - executorIdToSlaveId.remove(id) - scheduler.slaveLost(slaveId, reason) + logInfo("Executor %s removed: %s".format(fullId, message)) + executorIdToWorkerId.get(fullId) match { + case Some(workerId) => + executorIdToWorkerId.remove(fullId) + scheduler.slaveLost(workerId, reason) case None => - logInfo("No slave ID known for executor %s".format(id)) + logInfo("No worker ID known for executor %s".format(fullId)) } } } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index 1386cd9d4402e..bea9dc4f23a76 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -6,7 +6,7 @@ import spark.util.SerializableBuffer private[spark] sealed trait StandaloneClusterMessage extends Serializable -// Master to slaves +// Driver to executors private[spark] case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage @@ -16,7 +16,7 @@ case class RegisteredSlave(sparkProperties: Seq[(String, String)]) extends Stand private[spark] case class RegisterSlaveFailed(message: String) extends StandaloneClusterMessage -// Slaves to master +// Executors to driver private[spark] case class RegisterSlave(slaveId: String, host: String, cores: Int) extends StandaloneClusterMessage @@ -32,6 +32,6 @@ object StatusUpdate { } } -// Internal messages in master +// Internal messages in driver private[spark] case object ReviveOffers extends StandaloneClusterMessage -private[spark] case object StopMaster extends StandaloneClusterMessage +private[spark] case object StopDriver extends StandaloneClusterMessage diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index eeaae23dc8697..d742a7b2bf848 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -23,7 +23,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) - class MasterActor(sparkProperties: Seq[(String, String)]) extends Actor { + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { val slaveActor = new HashMap[String, ActorRef] val slaveAddress = new HashMap[String, Address] val slaveHost = new HashMap[String, String] @@ -37,34 +37,34 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } def receive = { - case RegisterSlave(slaveId, host, cores) => - if (slaveActor.contains(slaveId)) { - sender ! RegisterSlaveFailed("Duplicate slave ID: " + slaveId) + case RegisterSlave(workerId, host, cores) => + if (slaveActor.contains(workerId)) { + sender ! RegisterSlaveFailed("Duplicate slave ID: " + workerId) } else { - logInfo("Registered slave: " + sender + " with ID " + slaveId) + logInfo("Registered slave: " + sender + " with ID " + workerId) sender ! RegisteredSlave(sparkProperties) context.watch(sender) - slaveActor(slaveId) = sender - slaveHost(slaveId) = host - freeCores(slaveId) = cores - slaveAddress(slaveId) = sender.path.address - actorToSlaveId(sender) = slaveId - addressToSlaveId(sender.path.address) = slaveId + slaveActor(workerId) = sender + slaveHost(workerId) = host + freeCores(workerId) = cores + slaveAddress(workerId) = sender.path.address + actorToSlaveId(sender) = workerId + addressToSlaveId(sender.path.address) = workerId totalCoreCount.addAndGet(cores) makeOffers() } - case StatusUpdate(slaveId, taskId, state, data) => + case StatusUpdate(workerId, taskId, state, data) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { - freeCores(slaveId) += 1 - makeOffers(slaveId) + freeCores(workerId) += 1 + makeOffers(workerId) } case ReviveOffers => makeOffers() - case StopMaster => + case StopDriver => sender ! true context.stop(self) @@ -85,9 +85,9 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } // Make fake resource offers on just one slave - def makeOffers(slaveId: String) { + def makeOffers(workerId: String) { launchTasks(scheduler.resourceOffers( - Seq(new WorkerOffer(slaveId, slaveHost(slaveId), freeCores(slaveId))))) + Seq(new WorkerOffer(workerId, slaveHost(workerId), freeCores(workerId))))) } // Launch tasks returned by a set of resource offers @@ -99,24 +99,24 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } // Remove a disconnected slave from the cluster - def removeSlave(slaveId: String, reason: String) { - logInfo("Slave " + slaveId + " disconnected, so removing it") - val numCores = freeCores(slaveId) - actorToSlaveId -= slaveActor(slaveId) - addressToSlaveId -= slaveAddress(slaveId) - slaveActor -= slaveId - slaveHost -= slaveId - freeCores -= slaveId - slaveHost -= slaveId + def removeSlave(workerId: String, reason: String) { + logInfo("Slave " + workerId + " disconnected, so removing it") + val numCores = freeCores(workerId) + actorToSlaveId -= slaveActor(workerId) + addressToSlaveId -= slaveAddress(workerId) + slaveActor -= workerId + slaveHost -= workerId + freeCores -= workerId + slaveHost -= workerId totalCoreCount.addAndGet(-numCores) - scheduler.slaveLost(slaveId, SlaveLost(reason)) + scheduler.slaveLost(workerId, SlaveLost(reason)) } } - var masterActor: ActorRef = null + var driverActor: ActorRef = null val taskIdsOnSlave = new HashMap[String, HashSet[String]] - def start() { + override def start() { val properties = new ArrayBuffer[(String, String)] val iterator = System.getProperties.entrySet.iterator while (iterator.hasNext) { @@ -126,15 +126,15 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor properties += ((key, value)) } } - masterActor = actorSystem.actorOf( - Props(new MasterActor(properties)), name = StandaloneSchedulerBackend.ACTOR_NAME) + driverActor = actorSystem.actorOf( + Props(new DriverActor(properties)), name = StandaloneSchedulerBackend.ACTOR_NAME) } - def stop() { + override def stop() { try { - if (masterActor != null) { + if (driverActor != null) { val timeout = 5.seconds - val future = masterActor.ask(StopMaster)(timeout) + val future = driverActor.ask(StopDriver)(timeout) Await.result(future, timeout) } } catch { @@ -143,11 +143,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } } - def reviveOffers() { - masterActor ! ReviveOffers + override def reviveOffers() { + driverActor ! ReviveOffers } - def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2) + override def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2) } private[spark] object StandaloneSchedulerBackend { diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 014906b02850d..7bf56a05d64c0 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -104,11 +104,11 @@ private[spark] class CoarseMesosSchedulerBackend( def createCommand(offer: Offer, numCores: Int): CommandInfo = { val runScript = new File(sparkHome, "run").getCanonicalPath - val masterUrl = "akka://spark@%s:%s/user/%s".format( - System.getProperty("spark.master.host"), System.getProperty("spark.master.port"), + val driverUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) val command = "\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format( - runScript, masterUrl, offer.getSlaveId.getValue, offer.getHostname, numCores) + runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores) val environment = Environment.newBuilder() sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index a3d8671834dbc..9fd2b454a43dc 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -11,52 +11,51 @@ import akka.util.duration._ import spark.{Logging, SparkException, Utils} - private[spark] class BlockManagerMaster( val actorSystem: ActorSystem, - isMaster: Boolean, + isDriver: Boolean, isLocal: Boolean, - masterIp: String, - masterPort: Int) + driverIp: String, + driverPort: Int) extends Logging { val AKKA_RETRY_ATTEMPS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt - val MASTER_AKKA_ACTOR_NAME = "BlockMasterManager" + val DRIVER_AKKA_ACTOR_NAME = "BlockMasterManager" val SLAVE_AKKA_ACTOR_NAME = "BlockSlaveManager" val DEFAULT_MANAGER_IP: String = Utils.localHostName() val timeout = 10.seconds - var masterActor: ActorRef = { - if (isMaster) { - val masterActor = actorSystem.actorOf(Props(new BlockManagerMasterActor(isLocal)), - name = MASTER_AKKA_ACTOR_NAME) + var driverActor: ActorRef = { + if (isDriver) { + val driverActor = actorSystem.actorOf(Props(new BlockManagerMasterActor(isLocal)), + name = DRIVER_AKKA_ACTOR_NAME) logInfo("Registered BlockManagerMaster Actor") - masterActor + driverActor } else { - val url = "akka://spark@%s:%s/user/%s".format(masterIp, masterPort, MASTER_AKKA_ACTOR_NAME) + val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, DRIVER_AKKA_ACTOR_NAME) logInfo("Connecting to BlockManagerMaster: " + url) actorSystem.actorFor(url) } } - /** Remove a dead host from the master actor. This is only called on the master side. */ + /** Remove a dead host from the driver actor. This is only called on the driver side. */ def notifyADeadHost(host: String) { tell(RemoveHost(host)) logInfo("Removed " + host + " successfully in notifyADeadHost") } /** - * Send the master actor a heart beat from the slave. Returns true if everything works out, - * false if the master does not know about the given block manager, which means the block + * Send the driver actor a heart beat from the slave. Returns true if everything works out, + * false if the driver does not know about the given block manager, which means the block * manager should re-register. */ def sendHeartBeat(blockManagerId: BlockManagerId): Boolean = { - askMasterWithRetry[Boolean](HeartBeat(blockManagerId)) + askDriverWithReply[Boolean](HeartBeat(blockManagerId)) } - /** Register the BlockManager's id with the master. */ + /** Register the BlockManager's id with the driver. */ def registerBlockManager( blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") @@ -70,25 +69,25 @@ private[spark] class BlockManagerMaster( storageLevel: StorageLevel, memSize: Long, diskSize: Long): Boolean = { - val res = askMasterWithRetry[Boolean]( + val res = askDriverWithReply[Boolean]( UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) logInfo("Updated info of block " + blockId) res } - /** Get locations of the blockId from the master */ + /** Get locations of the blockId from the driver */ def getLocations(blockId: String): Seq[BlockManagerId] = { - askMasterWithRetry[Seq[BlockManagerId]](GetLocations(blockId)) + askDriverWithReply[Seq[BlockManagerId]](GetLocations(blockId)) } - /** Get locations of multiple blockIds from the master */ + /** Get locations of multiple blockIds from the driver */ def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = { - askMasterWithRetry[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) + askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) } - /** Get ids of other nodes in the cluster from the master */ + /** Get ids of other nodes in the cluster from the driver */ def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { - val result = askMasterWithRetry[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) + val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) if (result.length != numPeers) { throw new SparkException( "Error getting peers, only got " + result.size + " instead of " + numPeers) @@ -98,10 +97,10 @@ private[spark] class BlockManagerMaster( /** * Remove a block from the slaves that have it. This can only be used to remove - * blocks that the master knows about. + * blocks that the driver knows about. */ def removeBlock(blockId: String) { - askMasterWithRetry(RemoveBlock(blockId)) + askDriverWithReply(RemoveBlock(blockId)) } /** @@ -111,33 +110,33 @@ private[spark] class BlockManagerMaster( * amount of remaining memory. */ def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = { - askMasterWithRetry[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) + askDriverWithReply[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) } - /** Stop the master actor, called only on the Spark master node */ + /** Stop the driver actor, called only on the Spark driver node */ def stop() { - if (masterActor != null) { + if (driverActor != null) { tell(StopBlockManagerMaster) - masterActor = null + driverActor = null logInfo("BlockManagerMaster stopped") } } /** Send a one-way message to the master actor, to which we expect it to reply with true. */ private def tell(message: Any) { - if (!askMasterWithRetry[Boolean](message)) { + if (!askDriverWithReply[Boolean](message)) { throw new SparkException("BlockManagerMasterActor returned false, expected true.") } } /** - * Send a message to the master actor and get its result within a default timeout, or + * Send a message to the driver actor and get its result within a default timeout, or * throw a SparkException if this fails. */ - private def askMasterWithRetry[T](message: Any): T = { + private def askDriverWithReply[T](message: Any): T = { // TODO: Consider removing multiple attempts - if (masterActor == null) { - throw new SparkException("Error sending message to BlockManager as masterActor is null " + + if (driverActor == null) { + throw new SparkException("Error sending message to BlockManager as driverActor is null " + "[message = " + message + "]") } var attempts = 0 @@ -145,7 +144,7 @@ private[spark] class BlockManagerMaster( while (attempts < AKKA_RETRY_ATTEMPS) { attempts += 1 try { - val future = masterActor.ask(message)(timeout) + val future = driverActor.ask(message)(timeout) val result = Await.result(future, timeout) if (result == null) { throw new Exception("BlockManagerMaster returned null") diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 689f07b9692fb..0b8f6d43031c4 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -75,9 +75,9 @@ private[spark] object ThreadingTest { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") val serializer = new KryoSerializer - val masterIp: String = System.getProperty("spark.master.host", "localhost") - val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt - val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true, masterIp, masterPort) + val driverIp: String = System.getProperty("spark.driver.host", "localhost") + val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt + val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true, driverIp, driverPort) val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 01351de4aeb1a..42ce6f3c749f9 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -46,7 +46,7 @@ public void tearDown() { sc.stop(); sc = null; // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port"); + System.clearProperty("spark.driver.port"); } static class ReverseIntComparator implements Comparator, Serializable { diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index b5e31ddae3a9f..ff00dd05dd787 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -26,7 +26,7 @@ object LocalSparkContext { def stop(sc: SparkContext) { sc.stop() // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") } /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 7d5305f1e0369..718107d2b53f1 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -79,7 +79,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0) - System.setProperty("spark.master.port", boundPort.toString) + System.setProperty("spark.driver.port", boundPort.toString) val masterTracker = new MapOutputTracker(actorSystem, true) val slaveTracker = new MapOutputTracker(actorSystem, false) masterTracker.registerShuffle(10, 1) diff --git a/docs/configuration.md b/docs/configuration.md index 036a0df480e3f..a7054b4321bcf 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -202,7 +202,7 @@ Apart from these, the following properties are also available, and may be useful 10 Maximum message size to allow in "control plane" communication (for serialized tasks and task - results), in MB. Increase this if your tasks need to send back large results to the master + results), in MB. Increase this if your tasks need to send back large results to the driver (e.g. using collect() on a large dataset). @@ -211,7 +211,7 @@ Apart from these, the following properties are also available, and may be useful 4 Number of actor threads to use for communication. Can be useful to increase on large clusters - when the master has a lot of CPU cores. + when the driver has a lot of CPU cores. @@ -222,17 +222,17 @@ Apart from these, the following properties are also available, and may be useful - spark.master.host + spark.driver.host (local hostname) - Hostname or IP address for the master to listen on. + Hostname or IP address for the driver to listen on. - spark.master.port + spark.driver.port (random) - Port for the master to listen on. + Port for the driver to listen on. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 46ab34f063b2b..df7235756d37d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -26,7 +26,7 @@ def tearDown(self): sys.path = self._old_sys_path # To avoid Akka rebinding to the same port, since it doesn't unbind # immediately on shutdown - self.sc.jvm.System.clearProperty("spark.master.port") + self.sc.jvm.System.clearProperty("spark.driver.port") class TestCheckpoint(PySparkTestCase): diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index db78d06d4f0b6..43559b96d3a80 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -31,7 +31,7 @@ class ReplSuite extends FunSuite { if (interp.sparkContext != null) interp.sparkContext.stop() // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") return out.toString } diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index aa6be95f30f7a..8c322dd698099 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -153,8 +153,8 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = System.getProperty("spark.master.host", "localhost") - val port = System.getProperty("spark.master.port", "7077").toInt + val ip = System.getProperty("spark.driver.host", "localhost") + val port = System.getProperty("spark.driver.port", "7077").toInt val url = "akka://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorFor(url) val timeout = 5.seconds diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index c84e7331c75e8..79d60934297f3 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -43,7 +43,7 @@ public void tearDown() { ssc = null; // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port"); + System.clearProperty("spark.driver.port"); } @Test diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index bfdf32c73eeea..4a036f071074f 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -10,7 +10,7 @@ class BasicOperationsSuite extends TestSuiteBase { after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") } test("map") { diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index d2f32c189bc98..563a7d14587b5 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -19,7 +19,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { FileUtils.deleteDirectory(new File(checkpointDir)) // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") } var ssc: StreamingContext = null diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index 7493ac12073ea..c4cfffbfc1ef3 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -24,7 +24,7 @@ class FailureSuite extends TestSuiteBase with BeforeAndAfter { FileUtils.deleteDirectory(new File(checkpointDir)) // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") } override def framework = "CheckpointSuite" diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index d7ba7a5d17da8..70ae6e3934cfd 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -42,7 +42,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") } test("network input stream") { diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index 0c6e928835df1..cd9608df530af 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -13,7 +13,7 @@ class WindowOperationsSuite extends TestSuiteBase { after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.master.port") + System.clearProperty("spark.driver.port") } val largerSlideInput = Seq( From 539491bbc333834b9ae2721ae6cf3524cefb91ea Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 25 Jan 2013 09:29:59 -0800 Subject: [PATCH 067/696] code reformatting --- core/src/main/scala/spark/RDD.scala | 4 ++-- core/src/main/scala/spark/storage/BlockManagerUI.scala | 8 +++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 870cc5ca786c4..4fcab9279a5e1 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -94,7 +94,7 @@ abstract class RDD[T: ClassManifest]( /** How this RDD depends on any parent RDDs. */ protected def getDependencies(): List[Dependency[_]] = dependencies_ - // A friendly name for this RDD + /** A friendly name for this RDD */ var name: String = null /** Optionally overridden by subclasses to specify placement preferences. */ @@ -111,7 +111,7 @@ abstract class RDD[T: ClassManifest]( /** A unique ID for this RDD (within its SparkContext). */ val id = sc.newRddId() - /* Assign a name to this RDD */ + /** Assign a name to this RDD */ def setName(_name: String) = { name = _name this diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 35cbd59280dce..1003cc7a61160 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -57,7 +57,8 @@ class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - spark.storage.html.index.render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) + spark.storage.html.index. + render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) } }}} ~ get { path("rdd") { parameter("id") { id => { completeWith { @@ -67,9 +68,10 @@ class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray - val filteredStorageStatusList = StorageUtils.filterStorageStatusByPrefix(storageStatusList, prefix) + val filteredStorageStatusList = StorageUtils. + filterStorageStatusByPrefix(storageStatusList, prefix) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).first + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) From 1cadaa164e9f078e4ca483edb9db7fd5507c9e64 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 25 Jan 2013 09:30:21 -0800 Subject: [PATCH 068/696] switch to TimeStampedHashMap for storing persistent Rdds --- core/src/main/scala/spark/SparkContext.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index d9946488990b9..10ceeb30283e1 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -44,6 +44,7 @@ import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import util.TimeStampedHashMap /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -110,7 +111,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds = new ConcurrentHashMap[Int, RDD[_]]() + private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]() // Add each JAR given through the constructor jars.foreach { addJar(_) } From a1d9d1767d821c1e25e485e32d9356b12aba6a01 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 25 Jan 2013 10:05:26 -0800 Subject: [PATCH 069/696] fixup 1cadaa1, changed api of map --- core/src/main/scala/spark/storage/StorageUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 63ad5c125bd33..a10e3a95c60d3 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -56,8 +56,8 @@ object StorageUtils { // Find the id of the RDD, e.g. rdd_1 => 1 val rddId = rddKey.split("_").last.toInt // Get the friendly name for the rdd, if available. - val rddName = Option(sc.persistentRdds.get(rddId).name).getOrElse(rddKey) - val rddStorageLevel = sc.persistentRdds.get(rddId).getStorageLevel + val rddName = Option(sc.persistentRdds(rddId).name).getOrElse(rddKey) + val rddStorageLevel = sc.persistentRdds(rddId).getStorageLevel RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, memSize, diskSize) }.toArray From 8efbda0b179e3821a1221c6d78681fc74248cdac Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 25 Jan 2013 14:55:33 -0600 Subject: [PATCH 070/696] Call executeOnCompleteCallbacks in more finally blocks. --- .../scala/spark/scheduler/DAGScheduler.scala | 13 +++--- .../spark/scheduler/ShuffleMapTask.scala | 46 +++++++++---------- 2 files changed, 30 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b320be8863a2f..f599eb00bdff6 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -40,7 +40,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with eventQueue.put(HostLost(host)) } - // Called by TaskScheduler to cancel an entier TaskSet due to repeated failures. + // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures. override def taskSetFailed(taskSet: TaskSet, reason: String) { eventQueue.put(TaskSetFailed(taskSet, reason)) } @@ -54,8 +54,6 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with // resubmit failed stages val POLL_TIMEOUT = 10L - private val lock = new Object // Used for access to the entire DAGScheduler - private val eventQueue = new LinkedBlockingQueue[DAGSchedulerEvent] val nextRunId = new AtomicInteger(0) @@ -337,9 +335,12 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val rdd = job.finalStage.rdd val split = rdd.splits(job.partitions(0)) val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) - val result = job.func(taskContext, rdd.iterator(split, taskContext)) - taskContext.executeOnCompleteCallbacks() - job.listener.taskSucceeded(0, result) + try { + val result = job.func(taskContext, rdd.iterator(split, taskContext)) + job.listener.taskSucceeded(0, result) + } finally { + taskContext.executeOnCompleteCallbacks() + } } catch { case e: Exception => job.listener.jobFailed(e) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 19f5328eee087..83641a2a8427b 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -81,7 +81,7 @@ private[spark] class ShuffleMapTask( with Externalizable with Logging { - def this() = this(0, null, null, 0, null) + protected def this() = this(0, null, null, 0, null) var split = if (rdd == null) { null @@ -117,34 +117,34 @@ private[spark] class ShuffleMapTask( override def run(attemptId: Long): MapStatus = { val numOutputSplits = dep.partitioner.numPartitions - val partitioner = dep.partitioner val taskContext = new TaskContext(stageId, partition, attemptId) + try { + // Partition the map output. + val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) + for (elem <- rdd.iterator(split, taskContext)) { + val pair = elem.asInstanceOf[(Any, Any)] + val bucketId = dep.partitioner.getPartition(pair._1) + buckets(bucketId) += pair + } + val bucketIterators = buckets.map(_.iterator) - // Partition the map output. - val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) - for (elem <- rdd.iterator(split, taskContext)) { - val pair = elem.asInstanceOf[(Any, Any)] - val bucketId = partitioner.getPartition(pair._1) - buckets(bucketId) += pair - } - val bucketIterators = buckets.map(_.iterator) + val compressedSizes = new Array[Byte](numOutputSplits) - val compressedSizes = new Array[Byte](numOutputSplits) + val blockManager = SparkEnv.get.blockManager + for (i <- 0 until numOutputSplits) { + val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i + // Get a Scala iterator from Java map + val iter: Iterator[(Any, Any)] = bucketIterators(i) + val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) + compressedSizes(i) = MapOutputTracker.compressSize(size) + } - val blockManager = SparkEnv.get.blockManager - for (i <- 0 until numOutputSplits) { - val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i - // Get a Scala iterator from Java map - val iter: Iterator[(Any, Any)] = bucketIterators(i) - val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) - compressedSizes(i) = MapOutputTracker.compressSize(size) + return new MapStatus(blockManager.blockManagerId, compressedSizes) + } finally { + // Execute the callbacks on task completion. + taskContext.executeOnCompleteCallbacks() } - - // Execute the callbacks on task completion. - taskContext.executeOnCompleteCallbacks() - - return new MapStatus(blockManager.blockManagerId, compressedSizes) } override def preferredLocations: Seq[String] = locs From 49c05608f5f27354da120e2367b6d4a63ec38948 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 25 Jan 2013 17:04:16 -0800 Subject: [PATCH 071/696] add metadatacleaner for persisentRdd map --- core/src/main/scala/spark/SparkContext.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 10ceeb30283e1..bff54dbdd14a6 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -44,7 +44,7 @@ import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import util.TimeStampedHashMap +import util.{MetadataCleaner, TimeStampedHashMap} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -113,6 +113,9 @@ class SparkContext( // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]() + private[spark] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) + + // Add each JAR given through the constructor jars.foreach { addJar(_) } @@ -512,6 +515,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { if (dagScheduler != null) { + metadataCleaner.cancel() dagScheduler.stop() dagScheduler = null taskScheduler = null @@ -654,6 +658,12 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + + private[spark] def cleanup(cleanupTime: Long) { + var sizeBefore = persistentRdds.size + persistentRdds.clearOldValues(cleanupTime) + logInfo("idToStage " + sizeBefore + " --> " + persistentRdds.size) + } } /** From d49cf0e587b7cbbd31917d9bb69f98466feb0f9f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 26 Jan 2013 15:57:01 -0800 Subject: [PATCH 072/696] Fix JavaRDDLike.flatMap(PairFlatMapFunction) (SPARK-668). This workaround is easier than rewriting JavaRDDLike in Java. --- .../scala/spark/api/java/JavaRDDLike.scala | 7 ++--- .../spark/api/java/PairFlatMapWorkaround.java | 20 +++++++++++++ core/src/test/scala/spark/JavaAPISuite.java | 28 +++++++++++++++++++ 3 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index b3698ffa44d57..4c95c989b5367 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -12,7 +12,7 @@ import spark.storage.StorageLevel import com.google.common.base.Optional -trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { +trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround[T] { def wrapRDD(rdd: RDD[T]): This implicit val classManifest: ClassManifest[T] @@ -82,10 +82,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Return a new RDD by first applying a function to all elements of this - * RDD, and then flattening the results. + * Part of the workaround for SPARK-668; called in PairFlatMapWorkaround.java. */ - def flatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairRDD[K, V] = { + private[spark] def doFlatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairRDD[K, V] = { import scala.collection.JavaConverters._ def fn = (x: T) => f.apply(x).asScala def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] diff --git a/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java b/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java new file mode 100644 index 0000000000000..68b6fd6622742 --- /dev/null +++ b/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java @@ -0,0 +1,20 @@ +package spark.api.java; + +import spark.api.java.JavaPairRDD; +import spark.api.java.JavaRDDLike; +import spark.api.java.function.PairFlatMapFunction; + +import java.io.Serializable; + +/** + * Workaround for SPARK-668. + */ +class PairFlatMapWorkaround implements Serializable { + /** + * Return a new RDD by first applying a function to all elements of this + * RDD, and then flattening the results. + */ + public JavaPairRDD flatMap(PairFlatMapFunction f) { + return ((JavaRDDLike ) this).doFlatMap(f); + } +} diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 01351de4aeb1a..f50ba093e90ae 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -355,6 +355,34 @@ public Iterable call(String s) { Assert.assertEquals(11, pairs.count()); } + @Test + public void mapsFromPairsToPairs() { + List> pairs = Arrays.asList( + new Tuple2(1, "a"), + new Tuple2(2, "aa"), + new Tuple2(3, "aaa") + ); + JavaPairRDD pairRDD = sc.parallelizePairs(pairs); + + // Regression test for SPARK-668: + JavaPairRDD swapped = pairRDD.flatMap( + new PairFlatMapFunction, String, Integer>() { + @Override + public Iterable> call(Tuple2 item) throws Exception { + return Collections.singletonList(item.swap()); + } + }); + swapped.collect(); + + // There was never a bug here, but it's worth testing: + pairRDD.map(new PairFunction, String, Integer>() { + @Override + public Tuple2 call(Tuple2 item) throws Exception { + return item.swap(); + } + }).collect(); + } + @Test public void mapPartitions() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); From ad4232b4dadc6290d3c4696d3cc007d3f01cb236 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Sat, 26 Jan 2013 18:07:14 -0800 Subject: [PATCH 073/696] Fix deadlock in BlockManager reregistration triggered by failed updates. --- .../scala/spark/storage/BlockManager.scala | 35 +++++++++++++++- .../spark/storage/BlockManagerSuite.scala | 40 ++++++++++++++++++- 2 files changed, 72 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 19cdaaa984225..19d35b8667883 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -90,7 +90,10 @@ class BlockManager( val slaveActor = master.actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) - @volatile private var shuttingDown = false + // Pending reregistration action being executed asynchronously or null if none + // is pending. Accesses should synchronize on asyncReregisterLock. + var asyncReregisterTask: Future[Unit] = null + val asyncReregisterLock = new Object private def heartBeat() { if (!master.sendHeartBeat(blockManagerId)) { @@ -147,6 +150,8 @@ class BlockManager( /** * Reregister with the master and report all blocks to it. This will be called by the heart beat * thread if our heartbeat to the block amnager indicates that we were not registered. + * + * Note that this method must be called without any BlockInfo locks held. */ def reregister() { // TODO: We might need to rate limit reregistering. @@ -155,6 +160,32 @@ class BlockManager( reportAllBlocks() } + /** + * Reregister with the master sometime soon. + */ + def asyncReregister() { + asyncReregisterLock.synchronized { + if (asyncReregisterTask == null) { + asyncReregisterTask = Future[Unit] { + reregister() + asyncReregisterLock.synchronized { + asyncReregisterTask = null + } + } + } + } + } + + /** + * For testing. Wait for any pending asynchronous reregistration; otherwise, do nothing. + */ + def waitForAsyncReregister() { + val task = asyncReregisterTask + if (task != null) { + Await.ready(task, Duration.Inf) + } + } + /** * Get storage level of local block. If no info exists for the block, then returns null. */ @@ -170,7 +201,7 @@ class BlockManager( if (needReregister) { logInfo("Got told to reregister updating block " + blockId) // Reregistering will report our new block for free. - reregister() + asyncReregister() } logDebug("Told master about block " + blockId) } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index a1aeb12f25ec9..216574468951a 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -219,18 +219,56 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a2 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - assert(master.getLocations("a1").size > 0, "master was not told about a1") master.notifyADeadHost(store.blockManagerId.ip) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") store.putSingle("a2", a1, StorageLevel.MEMORY_ONLY) + store.waitForAsyncReregister() assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") assert(master.getLocations("a2").size > 0, "master was not told about a2") } + test("reregistration doesn't dead lock") { + val heartBeat = PrivateMethod[Unit]('heartBeat) + store = new BlockManager(actorSystem, master, serializer, 2000) + val a1 = new Array[Byte](400) + val a2 = List(new Array[Byte](400)) + + // try many times to trigger any deadlocks + for (i <- 1 to 100) { + master.notifyADeadHost(store.blockManagerId.ip) + val t1 = new Thread { + override def run = { + store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, true) + } + } + val t2 = new Thread { + override def run = { + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + } + } + val t3 = new Thread { + override def run = { + store invokePrivate heartBeat() + } + } + + t1.start + t2.start + t3.start + t1.join + t2.join + t3.join + + store.dropFromMemory("a1", null) + store.dropFromMemory("a2", null) + store.waitForAsyncReregister() + } + } + test("in-memory LRU storage") { store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) From 58fc6b2bed9f660fbf134aab188827b7d8975a62 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Sat, 26 Jan 2013 18:07:53 -0800 Subject: [PATCH 074/696] Handle duplicate registrations better. --- core/src/main/scala/spark/storage/BlockManagerMasterActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index f4d026da3329c..2216c33b76651 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -183,7 +183,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { if (blockManagerId.ip == Utils.localHostName() && !isLocal) { logInfo("Got Register Msg from master node, don't register it") - } else { + } else if (!blockManagerInfo.contains(blockManagerId)) { blockManagerIdByHost.get(blockManagerId.ip) match { case Some(managers) => // A block manager of the same host name already exists. From d012cfa0e8b29a1a7412bdff41eb159c4afe7d34 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 26 Jan 2013 22:48:39 -0800 Subject: [PATCH 075/696] Update spark_ec2.py to use new spark-ec2 scripts --- .../root/spark-ec2/ec2-variables.sh | 11 +++++++ ec2/spark_ec2.py | 33 ++++++++++++------- 2 files changed, 32 insertions(+), 12 deletions(-) create mode 100644 ec2/deploy.generic/root/spark-ec2/ec2-variables.sh diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh new file mode 100644 index 0000000000000..948cb5b1ecd7b --- /dev/null +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -0,0 +1,11 @@ +#!/bin/bash + +# These variables are automatically filled in by the mesos-ec2 script. +export MESOS_MASTERS="{{master_list}}" +export MESOS_SLAVES="{{slave_list}}" +export MESOS_ZOO_LIST="{{zoo_list}}" +export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}" +export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" +export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}" +export MODULES="{{modules}}" +export SWAP="{{swap}}" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index a5384d3bda1c6..f2385b5b56d79 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -358,25 +358,31 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_key): - print "Deploying files to master..." - deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, zoo_nodes) + if opts.cluster_type == "mesos": + modules = ['ephemeral-hdfs', 'persistent-hdfs', 'mesos'] + elif opts.cluster_type == "standalone": + modules = ['ephemeral-hdfs', 'persistent-hdfs', 'spark-standalone'] + master = master_nodes[0].public_dns_name if deploy_ssh_key: print "Copying SSH key %s to master..." % opts.identity_file ssh(master, opts, 'mkdir -p ~/.ssh') scp(master, opts, opts.identity_file, '~/.ssh/id_rsa') ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa') + + # NOTE: We should clone the repository before running deploy_files to prevent + # ec2-variables.sh from being overwritten + ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/shivaram/spark-ec2.git") + print "Deploying files to master..." + deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, + zoo_nodes, modules) print "Running setup on master..." - if opts.cluster_type == "mesos": - setup_mesos_cluster(master, opts) - elif opts.cluster_type == "standalone": - setup_standalone_cluster(master, slave_nodes, opts) + setup_spark_cluster(master, opts) print "Done!" -def setup_mesos_cluster(master, opts): - ssh(master, opts, "chmod u+x mesos-ec2/setup") - ssh(master, opts, "mesos-ec2/setup %s %s %s %s" % - ("generic", "none", "master", opts.swap)) +def setup_spark_cluster(master, opts): + ssh(master, opts, "chmod u+x spark-ec2/setup.sh") + ssh(master, opts, "spark-ec2/setup.sh") def setup_standalone_cluster(master, slave_nodes, opts): slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) @@ -427,7 +433,8 @@ def get_num_disks(instance_type): # cluster (e.g. lists of masters and slaves). Files are only deployed to # the first master instance in the cluster, and we expect the setup # script to be run on that instance to copy them to other nodes. -def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes): +def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes, + modules): active_master = master_nodes[0].public_dns_name num_disks = get_num_disks(opts.instance_type) @@ -459,7 +466,9 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes): "cluster_url": cluster_url, "hdfs_data_dirs": hdfs_data_dirs, "mapred_local_dirs": mapred_local_dirs, - "spark_local_dirs": spark_local_dirs + "spark_local_dirs": spark_local_dirs, + "swap": str(opts.swap), + "modules": '\n'.join(modules) } # Create a temp directory in which we will place all the files to be From 717b221cca79cb8a1603e9dcf7f0bb50e215ac41 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 26 Jan 2013 22:59:22 -0800 Subject: [PATCH 076/696] Detect whether we run on EC2 using ec2-metadata as well --- bin/start-master.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bin/start-master.sh b/bin/start-master.sh index a901b1c26068e..87feb261fe86b 100755 --- a/bin/start-master.sh +++ b/bin/start-master.sh @@ -26,7 +26,8 @@ fi # Set SPARK_PUBLIC_DNS so the master report the correct webUI address to the slaves if [ "$SPARK_PUBLIC_DNS" = "" ]; then # If we appear to be running on EC2, use the public address by default: - if [[ `hostname` == *ec2.internal ]]; then + # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname + if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname` fi fi From 0243b081ce4348c3d2955f2c16c0d3a61620be34 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 27 Jan 2013 00:25:29 -0800 Subject: [PATCH 077/696] Fix swap variable name --- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 948cb5b1ecd7b..166a884c889e4 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -8,4 +8,4 @@ export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}" export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}" export MODULES="{{modules}}" -export SWAP="{{swap}}" +export SWAP_MB="{{swap}}" From dc9d3ab6ed7ec2122ec9fdd248e236393601725c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 27 Jan 2013 00:26:00 -0800 Subject: [PATCH 078/696] Add option to start ganglia. Also enable Hadoop ports even if cluster type is not mesos --- ec2/spark_ec2.py | 23 +++++++++++++++-------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index f2385b5b56d79..dfccb6c238411 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -84,6 +84,9 @@ def parse_args(): "maximum price (in dollars)") parser.add_option("-c", "--cluster-type", default="mesos", help="'mesos' for a mesos cluster, 'standalone' for a standalone spark cluster (default: mesos)") + parser.add_option("-g", "--ganglia", action="store_true", default=False, + help="Setup ganglia monitoring for the cluster. NOTE: The ganglia " + + "monitoring page will be publicly accessible") parser.add_option("-u", "--user", default="root", help="The ssh user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, @@ -164,22 +167,23 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize(src_group=zoo_group) master_group.authorize('tcp', 22, 22, '0.0.0.0/0') master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') + master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') + master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') + master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') if opts.cluster_type == "mesos": - master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') - master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') - master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') + if opts.ganglia: + master_group.authorize('tcp', 80, 80, '0.0.0.0/0') if slave_group.rules == []: # Group was just now created slave_group.authorize(src_group=master_group) slave_group.authorize(src_group=slave_group) slave_group.authorize(src_group=zoo_group) slave_group.authorize('tcp', 22, 22, '0.0.0.0/0') slave_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') - if opts.cluster_type == "mesos": - slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0') - slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0') - slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0') - slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0') + slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0') + slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0') + slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0') + slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0') if zoo_group.rules == []: # Group was just now created zoo_group.authorize(src_group=master_group) zoo_group.authorize(src_group=slave_group) @@ -363,6 +367,9 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k elif opts.cluster_type == "standalone": modules = ['ephemeral-hdfs', 'persistent-hdfs', 'spark-standalone'] + if opts.ganglia: + modules.append('ganglia') + master = master_nodes[0].public_dns_name if deploy_ssh_key: print "Copying SSH key %s to master..." % opts.identity_file From 44b4a0f88fcb31727347b755ae8ec14d69571b52 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 27 Jan 2013 19:23:49 -0800 Subject: [PATCH 079/696] Track workers by executor ID instead of hostname to allow multiple executors per machine and remove the need for multiple IP addresses in unit tests. --- .../main/scala/spark/MapOutputTracker.scala | 4 +- core/src/main/scala/spark/SparkContext.scala | 6 +- core/src/main/scala/spark/SparkEnv.scala | 9 +- .../spark/deploy/LocalSparkCluster.scala | 16 +-- .../scala/spark/deploy/master/Master.scala | 4 +- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- .../main/scala/spark/executor/Executor.scala | 4 +- .../spark/executor/MesosExecutorBackend.scala | 3 +- .../executor/StandaloneExecutorBackend.scala | 14 +-- .../scala/spark/scheduler/DAGScheduler.scala | 44 +++---- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../scala/spark/scheduler/MapStatus.scala | 6 +- .../main/scala/spark/scheduler/Stage.scala | 11 +- .../scheduler/TaskSchedulerListener.scala | 2 +- .../scheduler/cluster/ClusterScheduler.scala | 110 ++++++++++-------- .../cluster/SparkDeploySchedulerBackend.scala | 4 +- .../cluster/StandaloneSchedulerBackend.scala | 64 +++++----- .../scheduler/cluster/TaskDescription.scala | 2 +- .../spark/scheduler/cluster/TaskInfo.scala | 7 +- .../scheduler/cluster/TaskSetManager.scala | 38 +++--- .../spark/scheduler/cluster/WorkerOffer.scala | 4 +- .../mesos/MesosSchedulerBackend.scala | 2 +- .../scala/spark/storage/BlockManager.scala | 10 +- .../scala/spark/storage/BlockManagerId.scala | 27 +++-- .../spark/storage/BlockManagerMaster.scala | 12 +- .../storage/BlockManagerMasterActor.scala | 66 +++++------ .../spark/storage/BlockManagerMessages.scala | 2 +- .../scala/spark/storage/BlockManagerUI.scala | 7 +- .../scala/spark/storage/ThreadingTest.scala | 3 +- .../src/main/scala/spark/util/AkkaUtils.scala | 6 +- .../scala/spark/util/TimeStampedHashMap.scala | 4 +- core/src/test/scala/spark/DriverSuite.scala | 5 +- .../scala/spark/MapOutputTrackerSuite.scala | 69 ++++++----- .../spark/storage/BlockManagerSuite.scala | 86 +++++++------- sbt/sbt | 2 +- 35 files changed, 343 insertions(+), 314 deletions(-) diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index ac02f3363afa8..c1f012b4195fe 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -114,7 +114,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea var array = mapStatuses(shuffleId) if (array != null) { array.synchronized { - if (array(mapId) != null && array(mapId).address == bmAddress) { + if (array(mapId) != null && array(mapId).location == bmAddress) { array(mapId) = null } } @@ -277,7 +277,7 @@ private[spark] object MapOutputTracker { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Missing an output location for shuffle " + shuffleId)) } else { - (status.address, decompressSize(status.compressedSizes(reduceId))) + (status.location, decompressSize(status.compressedSizes(reduceId))) } } } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 4581c0adcf72a..39721b47aee6b 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -80,6 +80,7 @@ class SparkContext( // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( + "", System.getProperty("spark.master.host"), System.getProperty("spark.master.port").toInt, true, @@ -97,7 +98,7 @@ class SparkContext( // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]() - private[spark] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) + private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) // Add each JAR given through the constructor @@ -649,10 +650,9 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { - var sizeBefore = persistentRdds.size persistentRdds.clearOldValues(cleanupTime) - logInfo("idToStage " + sizeBefore + " --> " + persistentRdds.size) } } diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 2a7a8af83d3e8..0c094edcf3e6c 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -19,6 +19,7 @@ import spark.util.AkkaUtils * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. */ class SparkEnv ( + val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, val closureSerializer: Serializer, @@ -58,11 +59,12 @@ object SparkEnv extends Logging { } def createFromSystemProperties( + executorId: String, hostname: String, port: Int, isMaster: Boolean, - isLocal: Boolean - ) : SparkEnv = { + isLocal: Boolean): SparkEnv = { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port) // Bit of a hack: If this is the master and our port was 0 (meaning bind to any free port), @@ -86,7 +88,7 @@ object SparkEnv extends Logging { val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt val blockManagerMaster = new BlockManagerMaster( actorSystem, isMaster, isLocal, masterIp, masterPort) - val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) val connectionManager = blockManager.connectionManager @@ -122,6 +124,7 @@ object SparkEnv extends Logging { } new SparkEnv( + executorId, actorSystem, serializer, closureSerializer, diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 4211d805967a9..8f51051e394e3 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -9,6 +9,12 @@ import spark.{Logging, Utils} import scala.collection.mutable.ArrayBuffer +/** + * Testing class that creates a Spark standalone process in-cluster (that is, running the + * spark.deploy.master.Master and spark.deploy.worker.Workers in the same JVMs). Executors launched + * by the Workers still run in separate JVMs. This can be used to test distributed operation and + * fault recovery without spinning up a lot of processes. + */ private[spark] class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) extends Logging { @@ -35,16 +41,12 @@ class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) /* Start the Slaves */ for (slaveNum <- 1 to numSlaves) { - /* We can pretend to test distributed stuff by giving the slaves distinct hostnames. - All of 127/8 should be a loopback, we use 127.100.*.* in hopes that it is - sufficiently distinctive. */ - val slaveIpAddress = "127.100.0." + (slaveNum % 256) val (actorSystem, boundPort) = - AkkaUtils.createActorSystem("sparkWorker" + slaveNum, slaveIpAddress, 0) + AkkaUtils.createActorSystem("sparkWorker" + slaveNum, localIpAddress, 0) slaveActorSystems += actorSystem val actor = actorSystem.actorOf( - Props(new Worker(slaveIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)), - name = "Worker") + Props(new Worker(localIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)), + name = "Worker") slaveActors += actor } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 2c2cd0231be70..2e7e868579a5b 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -97,10 +97,10 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor exec.worker.removeExecutor(exec) // Only retry certain number of times so we don't go into an infinite loop. - if (jobInfo.incrementRetryCount <= JobState.MAX_NUM_RETRY) { + if (jobInfo.incrementRetryCount < JobState.MAX_NUM_RETRY) { schedule() } else { - val e = new SparkException("Job %s wth ID %s failed %d times.".format( + val e = new SparkException("Job %s with ID %s failed %d times.".format( jobInfo.desc.name, jobInfo.id, jobInfo.retryCount)) logError(e.getMessage, e) throw e diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 0d1fe2a6b4970..af3acfecb6270 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -67,7 +67,7 @@ private[spark] class ExecutorRunner( /** Replace variables such as {{SLAVEID}} and {{CORES}} in a command argument passed to us */ def substituteVariables(argument: String): String = argument match { - case "{{SLAVEID}}" => workerId + case "{{EXECUTOR_ID}}" => execId.toString case "{{HOSTNAME}}" => hostname case "{{CORES}}" => cores.toString case other => other diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 28d9d40d43b92..bd21ba719a77c 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -30,7 +30,7 @@ private[spark] class Executor extends Logging { initLogging() - def initialize(slaveHostname: String, properties: Seq[(String, String)]) { + def initialize(executorId: String, slaveHostname: String, properties: Seq[(String, String)]) { // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) @@ -64,7 +64,7 @@ private[spark] class Executor extends Logging { ) // Initialize Spark environment (using system properties read above) - env = SparkEnv.createFromSystemProperties(slaveHostname, 0, false, false) + env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) SparkEnv.set(env) // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index eeab3959c6628..1ef88075ad1e2 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -29,9 +29,10 @@ private[spark] class MesosExecutorBackend(executor: Executor) executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { + logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) this.driver = driver val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor.initialize(slaveInfo.getHostname, properties) + executor.initialize(executorInfo.getExecutorId.getValue, slaveInfo.getHostname, properties) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index a29bf974d247e..435ee5743e7cf 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -17,7 +17,7 @@ import spark.scheduler.cluster.RegisterSlave private[spark] class StandaloneExecutorBackend( executor: Executor, masterUrl: String, - slaveId: String, + executorId: String, hostname: String, cores: Int) extends Actor @@ -30,7 +30,7 @@ private[spark] class StandaloneExecutorBackend( try { logInfo("Connecting to master: " + masterUrl) master = context.actorFor(masterUrl) - master ! RegisterSlave(slaveId, hostname, cores) + master ! RegisterSlave(executorId, hostname, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { @@ -43,7 +43,7 @@ private[spark] class StandaloneExecutorBackend( override def receive = { case RegisteredSlave(sparkProperties) => logInfo("Successfully registered with master") - executor.initialize(hostname, sparkProperties) + executor.initialize(executorId, hostname, sparkProperties) case RegisterSlaveFailed(message) => logError("Slave registration failed: " + message) @@ -55,24 +55,24 @@ private[spark] class StandaloneExecutorBackend( } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { - master ! StatusUpdate(slaveId, taskId, state, data) + master ! StatusUpdate(executorId, taskId, state, data) } } private[spark] object StandaloneExecutorBackend { - def run(masterUrl: String, slaveId: String, hostname: String, cores: Int) { + def run(masterUrl: String, executorId: String, hostname: String, cores: Int) { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0) val actor = actorSystem.actorOf( - Props(new StandaloneExecutorBackend(new Executor, masterUrl, slaveId, hostname, cores)), + Props(new StandaloneExecutorBackend(new Executor, masterUrl, executorId, hostname, cores)), name = "Executor") actorSystem.awaitTermination() } def main(args: Array[String]) { if (args.length != 4) { - System.err.println("Usage: StandaloneExecutorBackend ") + System.err.println("Usage: StandaloneExecutorBackend ") System.exit(1) } run(args(0), args(1), args(2), args(3).toInt) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f599eb00bdff6..bd541d4207953 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -35,9 +35,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with eventQueue.put(CompletionEvent(task, reason, result, accumUpdates)) } - // Called by TaskScheduler when a host fails. - override def hostLost(host: String) { - eventQueue.put(HostLost(host)) + // Called by TaskScheduler when an executor fails. + override def executorLost(execId: String) { + eventQueue.put(ExecutorLost(execId)) } // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures. @@ -72,7 +72,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with // For tracking failed nodes, we use the MapOutputTracker's generation number, which is // sent with every task. When we detect a node failing, we note the current generation number - // and failed host, increment it for new tasks, and use this to ignore stray ShuffleMapTask + // and failed executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask // results. // TODO: Garbage collect information about failure generations when we know there are no more // stray messages to detect. @@ -108,7 +108,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } def clearCacheLocs() { - cacheLocs.clear + cacheLocs.clear() } /** @@ -271,8 +271,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with submitStage(finalStage) } - case HostLost(host) => - handleHostLost(host) + case ExecutorLost(execId) => + handleExecutorLost(execId) case completion: CompletionEvent => handleTaskCompletion(completion) @@ -436,10 +436,10 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with case smt: ShuffleMapTask => val stage = idToStage(smt.stageId) val status = event.result.asInstanceOf[MapStatus] - val host = status.address.ip - logInfo("ShuffleMapTask finished with host " + host) - if (failedGeneration.contains(host) && smt.generation <= failedGeneration(host)) { - logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + host) + val execId = status.location.executorId + logDebug("ShuffleMapTask finished on " + execId) + if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) { + logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { stage.addOutputLoc(smt.partition, status) } @@ -511,9 +511,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with // Remember that a fetch failed now; this is used to resubmit the broken // stages later, after a small wait (to give other tasks the chance to fail) lastFetchFailureTime = System.currentTimeMillis() // TODO: Use pluggable clock - // TODO: mark the host as failed only if there were lots of fetch failures on it + // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { - handleHostLost(bmAddress.ip, Some(task.generation)) + handleExecutorLost(bmAddress.executorId, Some(task.generation)) } case other => @@ -523,21 +523,21 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } /** - * Responds to a host being lost. This is called inside the event loop so it assumes that it can - * modify the scheduler's internal state. Use hostLost() to post a host lost event from outside. + * Responds to an executor being lost. This is called inside the event loop, so it assumes it can + * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * * Optionally the generation during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - def handleHostLost(host: String, maybeGeneration: Option[Long] = None) { + def handleExecutorLost(execId: String, maybeGeneration: Option[Long] = None) { val currentGeneration = maybeGeneration.getOrElse(mapOutputTracker.getGeneration) - if (!failedGeneration.contains(host) || failedGeneration(host) < currentGeneration) { - failedGeneration(host) = currentGeneration - logInfo("Host lost: " + host + " (generation " + currentGeneration + ")") - env.blockManager.master.notifyADeadHost(host) + if (!failedGeneration.contains(execId) || failedGeneration(execId) < currentGeneration) { + failedGeneration(execId) = currentGeneration + logInfo("Executor lost: %s (generation %d)".format(execId, currentGeneration)) + env.blockManager.master.removeExecutor(execId) // TODO: This will be really slow if we keep accumulating shuffle map stages for ((shuffleId, stage) <- shuffleToMapStage) { - stage.removeOutputsOnHost(host) + stage.removeOutputsOnExecutor(execId) val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray mapOutputTracker.registerMapOutputs(shuffleId, locs, true) } @@ -546,7 +546,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } clearCacheLocs() } else { - logDebug("Additional host lost message for " + host + + logDebug("Additional executor lost message for " + execId + "(generation " + currentGeneration + ")") } } diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 3422a21d9d6a9..b34fa78c072c0 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -28,7 +28,7 @@ private[spark] case class CompletionEvent( accumUpdates: Map[Long, Any]) extends DAGSchedulerEvent -private[spark] case class HostLost(host: String) extends DAGSchedulerEvent +private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent private[spark] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala index fae643f3a8247..203abb917be1c 100644 --- a/core/src/main/scala/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/spark/scheduler/MapStatus.scala @@ -8,19 +8,19 @@ import java.io.{ObjectOutput, ObjectInput, Externalizable} * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. * The map output sizes are compressed using MapOutputTracker.compressSize. */ -private[spark] class MapStatus(var address: BlockManagerId, var compressedSizes: Array[Byte]) +private[spark] class MapStatus(var location: BlockManagerId, var compressedSizes: Array[Byte]) extends Externalizable { def this() = this(null, null) // For deserialization only def writeExternal(out: ObjectOutput) { - address.writeExternal(out) + location.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) } def readExternal(in: ObjectInput) { - address = BlockManagerId(in) + location = BlockManagerId(in) compressedSizes = new Array[Byte](in.readInt()) in.readFully(compressedSizes) } diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 4846b66729303..e9419728e3f34 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -51,18 +51,18 @@ private[spark] class Stage( def removeOutputLoc(partition: Int, bmAddress: BlockManagerId) { val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.address == bmAddress) + val newList = prevList.filterNot(_.location == bmAddress) outputLocs(partition) = newList if (prevList != Nil && newList == Nil) { numAvailableOutputs -= 1 } } - def removeOutputsOnHost(host: String) { + def removeOutputsOnExecutor(execId: String) { var becameUnavailable = false for (partition <- 0 until numPartitions) { val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.address.ip == host) + val newList = prevList.filterNot(_.location.executorId == execId) outputLocs(partition) = newList if (prevList != Nil && newList == Nil) { becameUnavailable = true @@ -70,7 +70,8 @@ private[spark] class Stage( } } if (becameUnavailable) { - logInfo("%s is now unavailable on %s (%d/%d, %s)".format(this, host, numAvailableOutputs, numPartitions, isAvailable)) + logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format( + this, execId, numAvailableOutputs, numPartitions, isAvailable)) } } @@ -82,7 +83,7 @@ private[spark] class Stage( def origin: String = rdd.origin - override def toString = "Stage " + id // + ": [RDD = " + rdd.id + ", isShuffle = " + isShuffleMap + "]" + override def toString = "Stage " + id override def hashCode(): Int = id } diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index fa4de15d0da60..9fcef86e46a29 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -12,7 +12,7 @@ private[spark] trait TaskSchedulerListener { def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any]): Unit // A node was lost from the cluster. - def hostLost(host: String): Unit + def executorLost(execId: String): Unit // The TaskScheduler wants to abort an entire task set. def taskSetFailed(taskSet: TaskSet, reason: String): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index a639b72795fe6..0b4177805b374 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -27,19 +27,20 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] - val taskIdToSlaveId = new HashMap[Long, String] + val taskIdToExecutorId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] // Incrementing Mesos task IDs val nextTaskId = new AtomicLong(0) - // Which hosts in the cluster are alive (contains hostnames) - val hostsAlive = new HashSet[String] + // Which executor IDs we have executors on + val activeExecutorIds = new HashSet[String] - // Which slave IDs we have executors on - val slaveIdsWithExecutors = new HashSet[String] + // The set of executors we have on each host; this is used to compute hostsAlive, which + // in turn is used to decide when we can attain data locality on a given host + val executorsByHost = new HashMap[String, HashSet[String]] - val slaveIdToHost = new HashMap[String, String] + val executorIdToHost = new HashMap[String, String] // JAR server, if any JARs were added by the user to the SparkContext var jarServer: HttpServer = null @@ -102,7 +103,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) activeTaskSets -= manager.taskSet.id activeTaskSetsQueue -= manager taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskIdToSlaveId --= taskSetTaskIds(manager.taskSet.id) + taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) taskSetTaskIds.remove(manager.taskSet.id) } } @@ -117,8 +118,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname for (o <- offers) { - slaveIdToHost(o.slaveId) = o.hostname - hostsAlive += o.hostname + executorIdToHost(o.executorId) = o.hostname } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) @@ -128,16 +128,20 @@ private[spark] class ClusterScheduler(val sc: SparkContext) do { launchedTask = false for (i <- 0 until offers.size) { - val sid = offers(i).slaveId + val execId = offers(i).executorId val host = offers(i).hostname - manager.slaveOffer(sid, host, availableCpus(i)) match { + manager.slaveOffer(execId, host, availableCpus(i)) match { case Some(task) => tasks(i) += task val tid = task.taskId taskIdToTaskSetId(tid) = manager.taskSet.id taskSetTaskIds(manager.taskSet.id) += tid - taskIdToSlaveId(tid) = sid - slaveIdsWithExecutors += sid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + if (!executorsByHost.contains(host)) { + executorsByHost(host) = new HashSet() + } + executorsByHost(host) += execId availableCpus(i) -= 1 launchedTask = true @@ -152,25 +156,21 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { var taskSetToUpdate: Option[TaskSetManager] = None - var failedHost: Option[String] = None + var failedExecutor: Option[String] = None var taskFailed = false synchronized { try { - if (state == TaskState.LOST && taskIdToSlaveId.contains(tid)) { - // We lost the executor on this slave, so remember that it's gone - val slaveId = taskIdToSlaveId(tid) - val host = slaveIdToHost(slaveId) - if (hostsAlive.contains(host)) { - slaveIdsWithExecutors -= slaveId - hostsAlive -= host - activeTaskSetsQueue.foreach(_.hostLost(host)) - failedHost = Some(host) + if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { + // We lost this entire executor, so remember that it's gone + val execId = taskIdToExecutorId(tid) + if (activeExecutorIds.contains(execId)) { + removeExecutor(execId) + failedExecutor = Some(execId) } } taskIdToTaskSetId.get(tid) match { case Some(taskSetId) => if (activeTaskSets.contains(taskSetId)) { - //activeTaskSets(taskSetId).statusUpdate(status) taskSetToUpdate = Some(activeTaskSets(taskSetId)) } if (TaskState.isFinished(state)) { @@ -178,7 +178,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) if (taskSetTaskIds.contains(taskSetId)) { taskSetTaskIds(taskSetId) -= tid } - taskIdToSlaveId.remove(tid) + taskIdToExecutorId.remove(tid) } if (state == TaskState.FAILED) { taskFailed = true @@ -190,12 +190,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext) case e: Exception => logError("Exception in statusUpdate", e) } } - // Update the task set and DAGScheduler without holding a lock on this, because that can deadlock + // Update the task set and DAGScheduler without holding a lock on this, since that can deadlock if (taskSetToUpdate != None) { taskSetToUpdate.get.statusUpdate(tid, state, serializedData) } - if (failedHost != None) { - listener.hostLost(failedHost.get) + if (failedExecutor != None) { + listener.executorLost(failedExecutor.get) backend.reviveOffers() } if (taskFailed) { @@ -249,32 +249,42 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - def slaveLost(slaveId: String, reason: ExecutorLossReason) { - var failedHost: Option[String] = None + def executorLost(executorId: String, reason: ExecutorLossReason) { + var failedExecutor: Option[String] = None synchronized { - slaveIdToHost.get(slaveId) match { - case Some(host) => - if (hostsAlive.contains(host)) { - logError("Lost an executor on " + host + ": " + reason) - slaveIdsWithExecutors -= slaveId - hostsAlive -= host - activeTaskSetsQueue.foreach(_.hostLost(host)) - failedHost = Some(host) - } else { - // We may get multiple slaveLost() calls with different loss reasons. For example, one - // may be triggered by a dropped connection from the slave while another may be a report - // of executor termination from Mesos. We produce log messages for both so we eventually - // report the termination reason. - logError("Lost an executor on " + host + " (already removed): " + reason) - } - case None => - // We were told about a slave being lost before we could even allocate work to it - logError("Lost slave " + slaveId + " (no work assigned yet)") + if (activeExecutorIds.contains(executorId)) { + val host = executorIdToHost(executorId) + logError("Lost executor %s on %s: %s".format(executorId, host, reason)) + removeExecutor(executorId) + failedExecutor = Some(executorId) + } else { + // We may get multiple executorLost() calls with different loss reasons. For example, one + // may be triggered by a dropped connection from the slave while another may be a report + // of executor termination from Mesos. We produce log messages for both so we eventually + // report the termination reason. + logError("Lost an executor " + executorId + " (already removed): " + reason) } } - if (failedHost != None) { - listener.hostLost(failedHost.get) + // Call listener.executorLost without holding the lock on this to prevent deadlock + if (failedExecutor != None) { + listener.executorLost(failedExecutor.get) backend.reviveOffers() } } + + /** Get a list of hosts that currently have executors */ + def hostsAlive: scala.collection.Set[String] = executorsByHost.keySet + + /** Remove an executor from all our data structures and mark it as lost */ + private def removeExecutor(executorId: String) { + activeExecutorIds -= executorId + val host = executorIdToHost(executorId) + val execs = executorsByHost.getOrElse(host, new HashSet) + execs -= executorId + if (execs.isEmpty) { + executorsByHost -= host + } + executorIdToHost -= executorId + activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 4f82cd96dd66a..f0792c1b76939 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -37,7 +37,7 @@ private[spark] class SparkDeploySchedulerBackend( val masterUrl = "akka://spark@%s:%s/user/%s".format( System.getProperty("spark.master.host"), System.getProperty("spark.master.port"), StandaloneSchedulerBackend.ACTOR_NAME) - val args = Seq(masterUrl, "{{SLAVEID}}", "{{HOSTNAME}}", "{{CORES}}") + val args = Seq(masterUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(throw new IllegalArgumentException("must supply spark home for spark standalone")) val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command, sparkHome) @@ -81,7 +81,7 @@ private[spark] class SparkDeploySchedulerBackend( executorIdToSlaveId.get(id) match { case Some(slaveId) => executorIdToSlaveId.remove(id) - scheduler.slaveLost(slaveId, reason) + scheduler.executorLost(slaveId, reason) case None => logInfo("No slave ID known for executor %s".format(id)) } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index eeaae23dc8697..32be1e7a26577 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -28,8 +28,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor val slaveAddress = new HashMap[String, Address] val slaveHost = new HashMap[String, String] val freeCores = new HashMap[String, Int] - val actorToSlaveId = new HashMap[ActorRef, String] - val addressToSlaveId = new HashMap[Address, String] + val actorToExecutorId = new HashMap[ActorRef, String] + val addressToExecutorId = new HashMap[Address, String] override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() @@ -37,28 +37,28 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } def receive = { - case RegisterSlave(slaveId, host, cores) => - if (slaveActor.contains(slaveId)) { - sender ! RegisterSlaveFailed("Duplicate slave ID: " + slaveId) + case RegisterSlave(executorId, host, cores) => + if (slaveActor.contains(executorId)) { + sender ! RegisterSlaveFailed("Duplicate executor ID: " + executorId) } else { - logInfo("Registered slave: " + sender + " with ID " + slaveId) + logInfo("Registered executor: " + sender + " with ID " + executorId) sender ! RegisteredSlave(sparkProperties) context.watch(sender) - slaveActor(slaveId) = sender - slaveHost(slaveId) = host - freeCores(slaveId) = cores - slaveAddress(slaveId) = sender.path.address - actorToSlaveId(sender) = slaveId - addressToSlaveId(sender.path.address) = slaveId + slaveActor(executorId) = sender + slaveHost(executorId) = host + freeCores(executorId) = cores + slaveAddress(executorId) = sender.path.address + actorToExecutorId(sender) = executorId + addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) makeOffers() } - case StatusUpdate(slaveId, taskId, state, data) => + case StatusUpdate(executorId, taskId, state, data) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { - freeCores(slaveId) += 1 - makeOffers(slaveId) + freeCores(executorId) += 1 + makeOffers(executorId) } case ReviveOffers => @@ -69,13 +69,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor context.stop(self) case Terminated(actor) => - actorToSlaveId.get(actor).foreach(removeSlave(_, "Akka actor terminated")) + actorToExecutorId.get(actor).foreach(removeSlave(_, "Akka actor terminated")) case RemoteClientDisconnected(transport, address) => - addressToSlaveId.get(address).foreach(removeSlave(_, "remote Akka client disconnected")) + addressToExecutorId.get(address).foreach(removeSlave(_, "remote Akka client disconnected")) case RemoteClientShutdown(transport, address) => - addressToSlaveId.get(address).foreach(removeSlave(_, "remote Akka client shutdown")) + addressToExecutorId.get(address).foreach(removeSlave(_, "remote Akka client shutdown")) } // Make fake resource offers on all slaves @@ -85,31 +85,31 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } // Make fake resource offers on just one slave - def makeOffers(slaveId: String) { + def makeOffers(executorId: String) { launchTasks(scheduler.resourceOffers( - Seq(new WorkerOffer(slaveId, slaveHost(slaveId), freeCores(slaveId))))) + Seq(new WorkerOffer(executorId, slaveHost(executorId), freeCores(executorId))))) } // Launch tasks returned by a set of resource offers def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { - freeCores(task.slaveId) -= 1 - slaveActor(task.slaveId) ! LaunchTask(task) + freeCores(task.executorId) -= 1 + slaveActor(task.executorId) ! LaunchTask(task) } } // Remove a disconnected slave from the cluster - def removeSlave(slaveId: String, reason: String) { - logInfo("Slave " + slaveId + " disconnected, so removing it") - val numCores = freeCores(slaveId) - actorToSlaveId -= slaveActor(slaveId) - addressToSlaveId -= slaveAddress(slaveId) - slaveActor -= slaveId - slaveHost -= slaveId - freeCores -= slaveId - slaveHost -= slaveId + def removeSlave(executorId: String, reason: String) { + logInfo("Slave " + executorId + " disconnected, so removing it") + val numCores = freeCores(executorId) + actorToExecutorId -= slaveActor(executorId) + addressToExecutorId -= slaveAddress(executorId) + slaveActor -= executorId + slaveHost -= executorId + freeCores -= executorId + slaveHost -= executorId totalCoreCount.addAndGet(-numCores) - scheduler.slaveLost(slaveId, SlaveLost(reason)) + scheduler.executorLost(executorId, SlaveLost(reason)) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index aa097fd3a25c1..b41e951be99f8 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -5,7 +5,7 @@ import spark.util.SerializableBuffer private[spark] class TaskDescription( val taskId: Long, - val slaveId: String, + val executorId: String, val name: String, _serializedTask: ByteBuffer) extends Serializable { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index ca84503780161..0f975ce1eb419 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -4,7 +4,12 @@ package spark.scheduler.cluster * Information about a running task attempt inside a TaskSet. */ private[spark] -class TaskInfo(val taskId: Long, val index: Int, val launchTime: Long, val host: String) { +class TaskInfo( + val taskId: Long, + val index: Int, + val launchTime: Long, + val executorId: String, + val host: String) { var finishTime: Long = 0 var failed = false diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index a089b71644d3b..26201ad0dd1a0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -138,10 +138,11 @@ private[spark] class TaskSetManager( // attempt running on this host, in case the host is slow. In addition, if localOnly is set, the // task must have a preference for this host (or no preferred locations at all). def findSpeculativeTask(host: String, localOnly: Boolean): Option[Int] = { + val hostsAlive = sched.hostsAlive speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set val localTask = speculatableTasks.find { index => - val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive + val locations = tasks(index).preferredLocations.toSet & hostsAlive val attemptLocs = taskAttempts(index).map(_.host) (locations.size == 0 || locations.contains(host)) && !attemptLocs.contains(host) } @@ -189,7 +190,7 @@ private[spark] class TaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(slaveId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -206,11 +207,11 @@ private[spark] class TaskSetManager( } else { "non-preferred, not one of " + task.preferredLocations.mkString(", ") } - logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( - taskSet.id, index, taskId, slaveId, host, prefStr)) + logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format( + taskSet.id, index, taskId, execId, host, prefStr)) // Do various bookkeeping copiesRunning(index) += 1 - val info = new TaskInfo(taskId, index, time, host) + val info = new TaskInfo(taskId, index, time, execId, host) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) if (preferred) { @@ -224,7 +225,7 @@ private[spark] class TaskSetManager( logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId, slaveId, taskName, serializedTask)) + return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) } case _ => } @@ -356,19 +357,22 @@ private[spark] class TaskSetManager( sched.taskSetFinished(this) } - def hostLost(hostname: String) { - logInfo("Re-queueing tasks for " + hostname + " from TaskSet " + taskSet.id) - // If some task has preferred locations only on hostname, put it in the no-prefs list - // to avoid the wait from delay scheduling - for (index <- getPendingTasksForHost(hostname)) { - val newLocs = tasks(index).preferredLocations.toSet & sched.hostsAlive - if (newLocs.isEmpty) { - pendingTasksWithNoPrefs += index + def executorLost(execId: String, hostname: String) { + logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) + val newHostsAlive = sched.hostsAlive + // If some task has preferred locations only on hostname, and there are no more executors there, + // put it in the no-prefs list to avoid the wait from delay scheduling + if (!newHostsAlive.contains(hostname)) { + for (index <- getPendingTasksForHost(hostname)) { + val newLocs = tasks(index).preferredLocations.toSet & newHostsAlive + if (newLocs.isEmpty) { + pendingTasksWithNoPrefs += index + } } } - // Re-enqueue any tasks that ran on the failed host if this is a shuffle map stage + // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage if (tasks(0).isInstanceOf[ShuffleMapTask]) { - for ((tid, info) <- taskInfos if info.host == hostname) { + for ((tid, info) <- taskInfos if info.executorId == execId) { val index = taskInfos(tid).index if (finished(index)) { finished(index) = false @@ -382,7 +386,7 @@ private[spark] class TaskSetManager( } } // Also re-enqueue any tasks that were running on the node - for ((tid, info) <- taskInfos if info.running && info.host == hostname) { + for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { taskLost(tid, TaskState.KILLED, null) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala index 6b919d68b26e6..3c3afcbb14d3f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala @@ -1,8 +1,8 @@ package spark.scheduler.cluster /** - * Represents free resources available on a worker node. + * Represents free resources available on an executor. */ private[spark] -class WorkerOffer(val slaveId: String, val hostname: String, val cores: Int) { +class WorkerOffer(val executorId: String, val hostname: String, val cores: Int) { } diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 2989e31f5e9d8..f3467db86b6ee 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -268,7 +268,7 @@ private[spark] class MesosSchedulerBackend( synchronized { slaveIdsWithExecutors -= slaveId.getValue } - scheduler.slaveLost(slaveId.getValue, reason) + scheduler.executorLost(slaveId.getValue, reason) } override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 19d35b8667883..1215d5f5c8a3d 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -30,6 +30,7 @@ extends Exception(message) private[spark] class BlockManager( + executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, val serializer: Serializer, @@ -68,8 +69,8 @@ class BlockManager( val connectionManager = new ConnectionManager(0) implicit val futureExecContext = connectionManager.futureExecContext - val connectionManagerId = connectionManager.id - val blockManagerId = BlockManagerId(connectionManagerId.host, connectionManagerId.port) + val blockManagerId = BlockManagerId( + executorId, connectionManager.id.host, connectionManager.id.port) // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) @@ -109,8 +110,9 @@ class BlockManager( /** * Construct a BlockManager with a memory limit set based on system properties. */ - def this(actorSystem: ActorSystem, master: BlockManagerMaster, serializer: Serializer) = { - this(actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) + def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, + serializer: Serializer) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) } /** diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index abb8b45a1f48b..f2f1e77d41a65 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -7,27 +7,32 @@ import java.util.concurrent.ConcurrentHashMap * This class represent an unique identifier for a BlockManager. * The first 2 constructors of this class is made private to ensure that * BlockManagerId objects can be created only using the factory method in - * [[spark.storage.BlockManager$]]. This allows de-duplication of id objects. + * [[spark.storage.BlockManager$]]. This allows de-duplication of ID objects. * Also, constructor parameters are private to ensure that parameters cannot * be modified from outside this class. */ private[spark] class BlockManagerId private ( + private var executorId_ : String, private var ip_ : String, private var port_ : Int ) extends Externalizable { - private def this() = this(null, 0) // For deserialization only + private def this() = this(null, null, 0) // For deserialization only - def ip = ip_ + def executorId: String = executorId_ - def port = port_ + def ip: String = ip_ + + def port: Int = port_ override def writeExternal(out: ObjectOutput) { + out.writeUTF(executorId_) out.writeUTF(ip_) out.writeInt(port_) } override def readExternal(in: ObjectInput) { + executorId_ = in.readUTF() ip_ = in.readUTF() port_ = in.readInt() } @@ -35,21 +40,23 @@ private[spark] class BlockManagerId private ( @throws(classOf[IOException]) private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) - override def toString = "BlockManagerId(" + ip + ", " + port + ")" + override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, ip, port) - override def hashCode = ip.hashCode * 41 + port + override def hashCode: Int = (executorId.hashCode * 41 + ip.hashCode) * 41 + port override def equals(that: Any) = that match { - case id: BlockManagerId => port == id.port && ip == id.ip - case _ => false + case id: BlockManagerId => + executorId == id.executorId && port == id.port && ip == id.ip + case _ => + false } } private[spark] object BlockManagerId { - def apply(ip: String, port: Int) = - getCachedBlockManagerId(new BlockManagerId(ip, port)) + def apply(execId: String, ip: String, port: Int) = + getCachedBlockManagerId(new BlockManagerId(execId, ip, port)) def apply(in: ObjectInput) = { val obj = new BlockManagerId() diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 937115e92cf03..55ff1dde9cc35 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -24,7 +24,7 @@ private[spark] class BlockManagerMaster( masterPort: Int) extends Logging { - val AKKA_RETRY_ATTEMPS: Int = System.getProperty("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt val MASTER_AKKA_ACTOR_NAME = "BlockMasterManager" @@ -45,10 +45,10 @@ private[spark] class BlockManagerMaster( } } - /** Remove a dead host from the master actor. This is only called on the master side. */ - def notifyADeadHost(host: String) { - tell(RemoveHost(host)) - logInfo("Removed " + host + " successfully in notifyADeadHost") + /** Remove a dead executor from the master actor. This is only called on the master side. */ + def removeExecutor(execId: String) { + tell(RemoveExecutor(execId)) + logInfo("Removed " + execId + " successfully in removeExecutor") } /** @@ -146,7 +146,7 @@ private[spark] class BlockManagerMaster( } var attempts = 0 var lastException: Exception = null - while (attempts < AKKA_RETRY_ATTEMPS) { + while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { val future = masterActor.ask(message)(timeout) diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index b31b6286d3685..f88517f1a39ec 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -23,9 +23,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { private val blockManagerInfo = new HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] - // Mapping from host name to block manager id. We allow multiple block managers - // on the same host name (ip). - private val blockManagerIdByHost = new HashMap[String, ArrayBuffer[BlockManagerId]] + // Mapping from executor ID to block manager ID. + private val blockManagerIdByExecutor = new HashMap[String, BlockManagerId] // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[String, Pair[Int, HashSet[BlockManagerId]]] @@ -74,8 +73,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { case RemoveBlock(blockId) => removeBlock(blockId) - case RemoveHost(host) => - removeHost(host) + case RemoveExecutor(execId) => + removeExecutor(execId) sender ! true case StopBlockManagerMaster => @@ -99,16 +98,12 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { def removeBlockManager(blockManagerId: BlockManagerId) { val info = blockManagerInfo(blockManagerId) - // Remove the block manager from blockManagerIdByHost. If the list of block - // managers belonging to the IP is empty, remove the entry from the hash map. - blockManagerIdByHost.get(blockManagerId.ip).foreach { managers: ArrayBuffer[BlockManagerId] => - managers -= blockManagerId - if (managers.size == 0) blockManagerIdByHost.remove(blockManagerId.ip) - } + // Remove the block manager from blockManagerIdByExecutor. + blockManagerIdByExecutor -= blockManagerId.executorId // Remove it from blockManagerInfo and remove all the blocks. blockManagerInfo.remove(blockManagerId) - var iterator = info.blocks.keySet.iterator + val iterator = info.blocks.keySet.iterator while (iterator.hasNext) { val blockId = iterator.next val locations = blockLocations.get(blockId)._2 @@ -133,17 +128,15 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { toRemove.foreach(removeBlockManager) } - def removeHost(host: String) { - logInfo("Trying to remove the host: " + host + " from BlockManagerMaster.") - logInfo("Previous hosts: " + blockManagerInfo.keySet.toSeq) - blockManagerIdByHost.get(host).foreach(_.foreach(removeBlockManager)) - logInfo("Current hosts: " + blockManagerInfo.keySet.toSeq) + def removeExecutor(execId: String) { + logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.") + blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) sender ! true } def heartBeat(blockManagerId: BlockManagerId) { if (!blockManagerInfo.contains(blockManagerId)) { - if (blockManagerId.ip == Utils.localHostName() && !isLocal) { + if (blockManagerId.executorId == "" && !isLocal) { sender ! true } else { sender ! false @@ -188,24 +181,20 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { sender ! res } - private def register(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { - val startTimeMs = System.currentTimeMillis() - val tmp = " " + blockManagerId + " " - - if (blockManagerId.ip == Utils.localHostName() && !isLocal) { - logInfo("Got Register Msg from master node, don't register it") - } else if (!blockManagerInfo.contains(blockManagerId)) { - blockManagerIdByHost.get(blockManagerId.ip) match { - case Some(managers) => - // A block manager of the same host name already exists. - logInfo("Got another registration for host " + blockManagerId) - managers += blockManagerId + private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + if (id.executorId == "" && !isLocal) { + // Got a register message from the master node; don't register it + } else if (!blockManagerInfo.contains(id)) { + blockManagerIdByExecutor.get(id.executorId) match { + case Some(manager) => + // A block manager of the same host name already exists + logError("Got two different block manager registrations on " + id.executorId) + System.exit(1) case None => - blockManagerIdByHost += (blockManagerId.ip -> ArrayBuffer(blockManagerId)) + blockManagerIdByExecutor(id.executorId) = id } - - blockManagerInfo += (blockManagerId -> new BlockManagerMasterActor.BlockManagerInfo( - blockManagerId, System.currentTimeMillis(), maxMemSize, slaveActor)) + blockManagerInfo(id) = new BlockManagerMasterActor.BlockManagerInfo( + id, System.currentTimeMillis(), maxMemSize, slaveActor) } sender ! true } @@ -217,11 +206,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { memSize: Long, diskSize: Long) { - val startTimeMs = System.currentTimeMillis() - val tmp = " " + blockManagerId + " " + blockId + " " - if (!blockManagerInfo.contains(blockManagerId)) { - if (blockManagerId.ip == Utils.localHostName() && !isLocal) { + if (blockManagerId.executorId == "" && !isLocal) { // We intentionally do not register the master (except in local mode), // so we should not indicate failure. sender ! true @@ -353,8 +339,8 @@ object BlockManagerMasterActor { _lastSeenMs = System.currentTimeMillis() } - def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long, diskSize: Long) - : Unit = synchronized { + def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long, + diskSize: Long) { updateLastSeenMs() diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 3d03ff3a93874..1494f901037c9 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -88,7 +88,7 @@ private[spark] case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster private[spark] -case class RemoveHost(host: String) extends ToBlockManagerMaster +case class RemoveExecutor(execId: String) extends ToBlockManagerMaster private[spark] case object StopBlockManagerMaster extends ToBlockManagerMaster diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 1003cc7a61160..b7423c7234c44 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -11,6 +11,7 @@ import cc.spray.typeconversion.TwirlSupport._ import scala.collection.mutable.ArrayBuffer import spark.{Logging, SparkContext, SparkEnv} import spark.util.AkkaUtils +import spark.Utils private[spark] @@ -20,10 +21,10 @@ object BlockManagerUI extends Logging { def start(actorSystem : ActorSystem, masterActor: ActorRef, sc: SparkContext) { val webUIDirectives = new BlockManagerUIDirectives(actorSystem, masterActor, sc) try { - logInfo("Starting BlockManager WebUI.") - val port = Option(System.getenv("BLOCKMANAGER_UI_PORT")).getOrElse("9080").toInt - AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, + val boundPort = AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", + Option(System.getenv("BLOCKMANAGER_UI_PORT")).getOrElse("9080").toInt, webUIDirectives.handler, "BlockManagerHTTPServer") + logInfo("Started BlockManager web UI at %s:%d".format(Utils.localHostName(), boundPort)) } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 689f07b9692fb..f04c046c31d79 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -78,7 +78,8 @@ private[spark] object ThreadingTest { val masterIp: String = System.getProperty("spark.master.host", "localhost") val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true, masterIp, masterPort) - val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer, 1024 * 1024) + val blockManager = new BlockManager( + "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index ff2c3079be07d..775ff8f1aafc1 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -52,10 +52,10 @@ private[spark] object AkkaUtils { /** * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to - * handle requests. Throws a SparkException if this fails. + * handle requests. Returns the bound port or throws a SparkException on failure. */ def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, - name: String = "HttpServer") { + name: String = "HttpServer"): Int = { val ioWorker = new IoWorker(actorSystem).start() val httpService = actorSystem.actorOf(Props(new HttpService(route))) val rootService = actorSystem.actorOf(Props(new SprayCanRootService(httpService))) @@ -67,7 +67,7 @@ private[spark] object AkkaUtils { try { Await.result(future, timeout) match { case bound: HttpServer.Bound => - return + return bound.endpoint.getPort case other: Any => throw new SparkException("Failed to bind web UI to port " + port + ": " + other) } diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index bb7c5c01c80d7..188f8910da8d5 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -63,9 +63,9 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging { override def empty: Map[A, B] = new TimeStampedHashMap[A, B]() - override def size(): Int = internalMap.size() + override def size: Int = internalMap.size - override def foreach[U](f: ((A, B)) => U): Unit = { + override def foreach[U](f: ((A, B)) => U) { val iterator = internalMap.entrySet().iterator() while(iterator.hasNext) { val entry = iterator.next() diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 70a7c8bc2f528..342610e1dd93b 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -13,7 +13,8 @@ class DriverSuite extends FunSuite with Timeouts { val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(10 seconds) { - Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) + Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master), + new File(System.getenv("SPARK_HOME"))) } } } @@ -28,4 +29,4 @@ object DriverWithoutCleanup { val sc = new SparkContext(args(0), "DriverWithoutCleanup") sc.parallelize(1 to 100, 4).count() } -} \ No newline at end of file +} diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 7d5305f1e0369..e8fe7ecabc2b7 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -43,13 +43,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000))) val statuses = tracker.getServerStatuses(10, 0) - assert(statuses.toSeq === Seq((BlockManagerId("hostA", 1000), size1000), - (BlockManagerId("hostB", 1000), size10000))) + assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), + (BlockManagerId("b", "hostB", 1000), size10000))) tracker.stop() } @@ -61,47 +61,52 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), Array(compressedSize1000, compressedSize1000, compressedSize1000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000, compressedSize1000))) // As if we had two simulatenous fetch failures - tracker.unregisterMapOutput(10, 0, BlockManagerId("hostA", 1000)) - tracker.unregisterMapOutput(10, 0, BlockManagerId("hostA", 1000)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) - // The remaining reduce task might try to grab the output dispite the shuffle failure; + // The remaining reduce task might try to grab the output despite the shuffle failure; // this should cause it to fail, and the scheduler will ignore the failure due to the // stage already being aborted. intercept[FetchFailedException] { tracker.getServerStatuses(10, 1) } } test("remote fetch") { - val (actorSystem, boundPort) = - AkkaUtils.createActorSystem("test", "localhost", 0) - System.setProperty("spark.master.port", boundPort.toString) - val masterTracker = new MapOutputTracker(actorSystem, true) - val slaveTracker = new MapOutputTracker(actorSystem, false) - masterTracker.registerShuffle(10, 1) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + try { + System.clearProperty("spark.master.host") // In case some previous test had set it + val (actorSystem, boundPort) = + AkkaUtils.createActorSystem("test", "localhost", 0) + System.setProperty("spark.master.port", boundPort.toString) + val masterTracker = new MapOutputTracker(actorSystem, true) + val slaveTracker = new MapOutputTracker(actorSystem, false) + masterTracker.registerShuffle(10, 1) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("hostA", 1000), Array(compressedSize1000))) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("hostA", 1000), size1000))) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val size1000 = MapOutputTracker.decompressSize(compressedSize1000) + masterTracker.registerMapOutput(10, 0, new MapStatus( + BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) - masterTracker.unregisterMapOutput(10, 0, BlockManagerId("hostA", 1000)) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - // failure should be cached - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + // failure should be cached + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + } finally { + System.clearProperty("spark.master.port") + } } } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 216574468951a..2d177bbf67458 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -86,9 +86,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("BlockManagerId object caching") { - val id1 = BlockManagerId("XXX", 1) - val id2 = BlockManagerId("XXX", 1) // this should return the same object as id1 - val id3 = BlockManagerId("XXX", 2) // this should return a different object + val id1 = BlockManagerId("e1", "XXX", 1) + val id2 = BlockManagerId("e1", "XXX", 1) // this should return the same object as id1 + val id3 = BlockManagerId("e1", "XXX", 2) // this should return a different object assert(id2 === id1, "id2 is not same as id1") assert(id2.eq(id1), "id2 is not the same object as id1") assert(id3 != id1, "id3 is same as id1") @@ -103,7 +103,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -133,8 +133,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager(actorSystem, master, serializer, 2000) - store2 = new BlockManager(actorSystem, master, new KryoSerializer, 2000) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000) + store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -149,7 +149,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -198,7 +198,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -206,7 +206,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.getSingle("a1") != None, "a1 was not in store") assert(master.getLocations("a1").size > 0, "master was not told about a1") - master.notifyADeadHost(store.blockManagerId.ip) + master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") store invokePrivate heartBeat() @@ -214,14 +214,14 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) assert(master.getLocations("a1").size > 0, "master was not told about a1") - master.notifyADeadHost(store.blockManagerId.ip) + master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") store.putSingle("a2", a1, StorageLevel.MEMORY_ONLY) @@ -233,35 +233,35 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) // try many times to trigger any deadlocks for (i <- 1 to 100) { - master.notifyADeadHost(store.blockManagerId.ip) + master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { - override def run = { + override def run() { store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, true) } } val t2 = new Thread { - override def run = { + override def run() { store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) } } val t3 = new Thread { - override def run = { + override def run() { store invokePrivate heartBeat() } } - t1.start - t2.start - t3.start - t1.join - t2.join - t3.join + t1.start() + t2.start() + t3.start() + t1.join() + t2.join() + t3.join() store.dropFromMemory("a1", null) store.dropFromMemory("a2", null) @@ -270,7 +270,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -289,7 +289,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -308,14 +308,14 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) store.putSingle("rdd_0_1", a1, StorageLevel.MEMORY_ONLY) store.putSingle("rdd_0_2", a2, StorageLevel.MEMORY_ONLY) store.putSingle("rdd_0_3", a3, StorageLevel.MEMORY_ONLY) - // Even though we accessed rdd_0_3 last, it should not have replaced partitiosn 1 and 2 + // Even though we accessed rdd_0_3 last, it should not have replaced partitions 1 and 2 // from the same RDD assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store") assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store") @@ -327,7 +327,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) store.putSingle("rdd_0_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle("rdd_0_2", new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle("rdd_1_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -350,7 +350,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -363,7 +363,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -378,7 +378,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -393,7 +393,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -408,7 +408,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -423,7 +423,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -448,7 +448,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -472,7 +472,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager(actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -518,7 +518,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager(actorSystem, master, serializer, 500) + store = new BlockManager("", actorSystem, master, serializer, 500) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -529,49 +529,49 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { System.setProperty("spark.shuffle.compress", "true") - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000) store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("shuffle_0_0_0") <= 100, "shuffle_0_0_0 was not compressed") store.stop() store = null System.setProperty("spark.shuffle.compress", "false") - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("exec2", actorSystem, master, serializer, 2000) store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("shuffle_0_0_0") >= 1000, "shuffle_0_0_0 was compressed") store.stop() store = null System.setProperty("spark.broadcast.compress", "true") - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("exec3", actorSystem, master, serializer, 2000) store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("broadcast_0") <= 100, "broadcast_0 was not compressed") store.stop() store = null System.setProperty("spark.broadcast.compress", "false") - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("exec4", actorSystem, master, serializer, 2000) store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("broadcast_0") >= 1000, "broadcast_0 was compressed") store.stop() store = null System.setProperty("spark.rdd.compress", "true") - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("exec5", actorSystem, master, serializer, 2000) store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("rdd_0_0") <= 100, "rdd_0_0 was not compressed") store.stop() store = null System.setProperty("spark.rdd.compress", "false") - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("exec6", actorSystem, master, serializer, 2000) store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("rdd_0_0") >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager(actorSystem, master, serializer, 2000) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() diff --git a/sbt/sbt b/sbt/sbt index a3055c13c1b0f..8f426d18e892f 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then fi export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath -java -Xmx1200M -XX:MaxPermSize=200m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200M -XX:MaxPermSize=250m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From 909850729ec59b788645575fdc03df7cc51fe42b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 27 Jan 2013 23:17:20 -0800 Subject: [PATCH 080/696] Rename more things from slave to executor --- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- .../executor/StandaloneExecutorBackend.scala | 12 ++--- .../scheduler/cluster/SlaveResources.scala | 4 -- .../cluster/SparkDeploySchedulerBackend.scala | 16 ++----- .../cluster/StandaloneClusterMessage.scala | 16 ++++--- .../cluster/StandaloneSchedulerBackend.scala | 48 +++++++++---------- .../scala/spark/storage/BlockManagerUI.scala | 2 + .../scala/spark/util/MetadataCleaner.scala | 10 ++-- 8 files changed, 50 insertions(+), 60 deletions(-) delete mode 100644 core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index af3acfecb6270..f5ff267d44fd8 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -65,7 +65,7 @@ private[spark] class ExecutorRunner( } } - /** Replace variables such as {{SLAVEID}} and {{CORES}} in a command argument passed to us */ + /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */ def substituteVariables(argument: String): String = argument match { case "{{EXECUTOR_ID}}" => execId.toString case "{{HOSTNAME}}" => hostname diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 435ee5743e7cf..50871802ea130 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -8,10 +8,10 @@ import akka.actor.{ActorRef, Actor, Props} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} import akka.remote.RemoteClientLifeCycleEvent import spark.scheduler.cluster._ -import spark.scheduler.cluster.RegisteredSlave +import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask -import spark.scheduler.cluster.RegisterSlaveFailed -import spark.scheduler.cluster.RegisterSlave +import spark.scheduler.cluster.RegisterExecutorFailed +import spark.scheduler.cluster.RegisterExecutor private[spark] class StandaloneExecutorBackend( @@ -30,7 +30,7 @@ private[spark] class StandaloneExecutorBackend( try { logInfo("Connecting to master: " + masterUrl) master = context.actorFor(masterUrl) - master ! RegisterSlave(executorId, hostname, cores) + master ! RegisterExecutor(executorId, hostname, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { @@ -41,11 +41,11 @@ private[spark] class StandaloneExecutorBackend( } override def receive = { - case RegisteredSlave(sparkProperties) => + case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with master") executor.initialize(executorId, hostname, sparkProperties) - case RegisterSlaveFailed(message) => + case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) System.exit(1) diff --git a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala b/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala deleted file mode 100644 index 96ebaa460118e..0000000000000 --- a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala +++ /dev/null @@ -1,4 +0,0 @@ -package spark.scheduler.cluster - -private[spark] -class SlaveResources(val slaveId: String, val hostname: String, val coresFree: Int) {} diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index f0792c1b76939..6dd3ae003d90a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -19,7 +19,6 @@ private[spark] class SparkDeploySchedulerBackend( var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt - val executorIdToSlaveId = new HashMap[String, String] // Memory used by each executor (in megabytes) val executorMemory = { @@ -47,7 +46,7 @@ private[spark] class SparkDeploySchedulerBackend( } override def stop() { - stopping = true; + stopping = true super.stop() client.stop() if (shutdownCallback != null) { @@ -67,23 +66,16 @@ private[spark] class SparkDeploySchedulerBackend( } def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int) { - executorIdToSlaveId += id -> workerId logInfo("Granted executor ID %s on host %s with %d cores, %s RAM".format( id, host, cores, Utils.memoryMegabytesToString(memory))) } - def executorRemoved(id: String, message: String, exitStatus: Option[Int]) { + def executorRemoved(executorId: String, message: String, exitStatus: Option[Int]) { val reason: ExecutorLossReason = exitStatus match { case Some(code) => ExecutorExited(code) case None => SlaveLost(message) } - logInfo("Executor %s removed: %s".format(id, message)) - executorIdToSlaveId.get(id) match { - case Some(slaveId) => - executorIdToSlaveId.remove(id) - scheduler.executorLost(slaveId, reason) - case None => - logInfo("No slave ID known for executor %s".format(id)) - } + logInfo("Executor %s removed: %s".format(executorId, message)) + scheduler.executorLost(executorId, reason) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index 1386cd9d4402e..c68f15bdfa2c6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -11,24 +11,26 @@ private[spark] case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage private[spark] -case class RegisteredSlave(sparkProperties: Seq[(String, String)]) extends StandaloneClusterMessage +case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) + extends StandaloneClusterMessage private[spark] -case class RegisterSlaveFailed(message: String) extends StandaloneClusterMessage +case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage -// Slaves to master +// Executors to master private[spark] -case class RegisterSlave(slaveId: String, host: String, cores: Int) extends StandaloneClusterMessage +case class RegisterExecutor(executorId: String, host: String, cores: Int) + extends StandaloneClusterMessage private[spark] -case class StatusUpdate(slaveId: String, taskId: Long, state: TaskState, data: SerializableBuffer) +case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) extends StandaloneClusterMessage private[spark] object StatusUpdate { /** Alternate factory method that takes a ByteBuffer directly for the data field */ - def apply(slaveId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = { - StatusUpdate(slaveId, taskId, state, new SerializableBuffer(data)) + def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 32be1e7a26577..69822f568c78a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -24,9 +24,9 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor var totalCoreCount = new AtomicInteger(0) class MasterActor(sparkProperties: Seq[(String, String)]) extends Actor { - val slaveActor = new HashMap[String, ActorRef] - val slaveAddress = new HashMap[String, Address] - val slaveHost = new HashMap[String, String] + val executorActor = new HashMap[String, ActorRef] + val executorAddress = new HashMap[String, Address] + val executorHost = new HashMap[String, String] val freeCores = new HashMap[String, Int] val actorToExecutorId = new HashMap[ActorRef, String] val addressToExecutorId = new HashMap[Address, String] @@ -37,17 +37,17 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } def receive = { - case RegisterSlave(executorId, host, cores) => - if (slaveActor.contains(executorId)) { - sender ! RegisterSlaveFailed("Duplicate executor ID: " + executorId) + case RegisterExecutor(executorId, host, cores) => + if (executorActor.contains(executorId)) { + sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) } else { logInfo("Registered executor: " + sender + " with ID " + executorId) - sender ! RegisteredSlave(sparkProperties) + sender ! RegisteredExecutor(sparkProperties) context.watch(sender) - slaveActor(executorId) = sender - slaveHost(executorId) = host + executorActor(executorId) = sender + executorHost(executorId) = host freeCores(executorId) = cores - slaveAddress(executorId) = sender.path.address + executorAddress(executorId) = sender.path.address actorToExecutorId(sender) = executorId addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) @@ -69,45 +69,45 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor context.stop(self) case Terminated(actor) => - actorToExecutorId.get(actor).foreach(removeSlave(_, "Akka actor terminated")) + actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) case RemoteClientDisconnected(transport, address) => - addressToExecutorId.get(address).foreach(removeSlave(_, "remote Akka client disconnected")) + addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected")) case RemoteClientShutdown(transport, address) => - addressToExecutorId.get(address).foreach(removeSlave(_, "remote Akka client shutdown")) + addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown")) } - // Make fake resource offers on all slaves + // Make fake resource offers on all executors def makeOffers() { launchTasks(scheduler.resourceOffers( - slaveHost.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 slave + // Make fake resource offers on just one executor def makeOffers(executorId: String) { launchTasks(scheduler.resourceOffers( - Seq(new WorkerOffer(executorId, slaveHost(executorId), freeCores(executorId))))) + Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId))))) } // Launch tasks returned by a set of resource offers def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { freeCores(task.executorId) -= 1 - slaveActor(task.executorId) ! LaunchTask(task) + executorActor(task.executorId) ! LaunchTask(task) } } // Remove a disconnected slave from the cluster - def removeSlave(executorId: String, reason: String) { + def removeExecutor(executorId: String, reason: String) { logInfo("Slave " + executorId + " disconnected, so removing it") val numCores = freeCores(executorId) - actorToExecutorId -= slaveActor(executorId) - addressToExecutorId -= slaveAddress(executorId) - slaveActor -= executorId - slaveHost -= executorId + actorToExecutorId -= executorActor(executorId) + addressToExecutorId -= executorAddress(executorId) + executorActor -= executorId + executorHost -= executorId freeCores -= executorId - slaveHost -= executorId + executorHost -= executorId totalCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) } diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index b7423c7234c44..956ede201ea2d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -21,6 +21,8 @@ object BlockManagerUI extends Logging { def start(actorSystem : ActorSystem, masterActor: ActorRef, sc: SparkContext) { val webUIDirectives = new BlockManagerUIDirectives(actorSystem, masterActor, sc) try { + // TODO: This needs to find a random free port to bind to. Unfortunately, there's no way + // in spray to do that, so we'll have to rely on something like new ServerSocket() val boundPort = AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", Option(System.getenv("BLOCKMANAGER_UI_PORT")).getOrElse("9080").toInt, webUIDirectives.handler, "BlockManagerHTTPServer") diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index 139e21d09e00c..721c4c6029a50 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -14,18 +14,16 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging val task = new TimerTask { def run() { try { - if (delaySeconds > 0) { - cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000)) - logInfo("Ran metadata cleaner for " + name) - } + cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000)) + logInfo("Ran metadata cleaner for " + name) } catch { case e: Exception => logError("Error running cleanup task for " + name, e) } } } - if (periodSeconds > 0) { - logInfo( + if (delaySeconds > 0) { + logDebug( "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds and " + "period of " + periodSeconds + " secs") timer.schedule(task, periodSeconds * 1000, periodSeconds * 1000) From f03d9760fd8ac67fd0865cb355ba75d2eff507fe Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 27 Jan 2013 23:56:14 -0800 Subject: [PATCH 081/696] Clean up BlockManagerUI a little (make it not be an object, merge with Directives, and bind to a random port) --- core/src/main/scala/spark/SparkContext.scala | 7 +- core/src/main/scala/spark/Utils.scala | 17 ++- .../spark/deploy/master/MasterWebUI.scala | 6 +- .../spark/deploy/worker/WorkerWebUI.scala | 6 +- .../scala/spark/storage/BlockManagerUI.scala | 120 +++++++++--------- .../src/main/scala/spark/util/AkkaUtils.scala | 6 +- .../scala/spark/util/MetadataCleaner.scala | 3 + 7 files changed, 91 insertions(+), 74 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 39721b47aee6b..77036c1275fa5 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -44,6 +44,7 @@ import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import storage.BlockManagerUI import util.{MetadataCleaner, TimeStampedHashMap} /** @@ -88,8 +89,9 @@ class SparkContext( SparkEnv.set(env) // Start the BlockManager UI - spark.storage.BlockManagerUI.start(SparkEnv.get.actorSystem, - SparkEnv.get.blockManager.master.masterActor, this) + private[spark] val ui = new BlockManagerUI( + env.actorSystem, env.blockManager.master.masterActor, this) + ui.start() // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() @@ -97,7 +99,6 @@ class SparkContext( // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]() - private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index ae7726437250a..1e58d012731b5 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -1,7 +1,7 @@ package spark import java.io._ -import java.net.{NetworkInterface, InetAddress, Inet4Address, URL, URI} +import java.net._ import java.util.{Locale, Random, UUID} import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} import org.apache.hadoop.conf.Configuration @@ -11,6 +11,7 @@ import scala.collection.JavaConversions._ import scala.io.Source import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder +import scala.Some /** * Various utility methods used by Spark. @@ -431,4 +432,18 @@ private object Utils extends Logging { } "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine) } + + /** + * Try to find a free port to bind to on the local host. This should ideally never be needed, + * except that, unfortunately, some of the networking libraries we currently rely on (e.g. Spray) + * don't let users bind to port 0 and then figure out which free port they actually bound to. + * We work around this by binding a ServerSocket and immediately unbinding it. This is *not* + * necessarily guaranteed to work, but it's the best we can do. + */ + def findFreePort(): Int = { + val socket = new ServerSocket(0) + val portBound = socket.getLocalPort + socket.close() + portBound + } } diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 458ee2d66589c..a01774f511007 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -14,12 +14,15 @@ import cc.spray.typeconversion.SprayJsonSupport._ import spark.deploy._ import spark.deploy.JsonProtocol._ +/** + * Web UI server for the standalone master. + */ private[spark] class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/master/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Timeout(1 seconds) + implicit val timeout = Timeout(10 seconds) val handler = { get { @@ -76,5 +79,4 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct getFromResourceDirectory(RESOURCE_DIR) } } - } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index f9489d99fc13f..ef81f072a3082 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -13,12 +13,15 @@ import cc.spray.typeconversion.SprayJsonSupport._ import spark.deploy.{WorkerState, RequestWorkerState} import spark.deploy.JsonProtocol._ +/** + * Web UI server for the standalone worker. + */ private[spark] class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/worker/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Timeout(1 seconds) + implicit val timeout = Timeout(10 seconds) val handler = { get { @@ -50,5 +53,4 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct getFromResourceDirectory(RESOURCE_DIR) } } - } diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 956ede201ea2d..eda320fa47972 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -1,32 +1,41 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Await import akka.pattern.ask import akka.util.Timeout import akka.util.duration._ -import cc.spray.Directives import cc.spray.directives._ import cc.spray.typeconversion.TwirlSupport._ +import cc.spray.Directives import scala.collection.mutable.ArrayBuffer -import spark.{Logging, SparkContext, SparkEnv} +import spark.{Logging, SparkContext} import spark.util.AkkaUtils import spark.Utils +/** + * Web UI server for the BlockManager inside each SparkContext. + */ private[spark] -object BlockManagerUI extends Logging { +class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) + extends Directives with Logging { + + val STATIC_RESOURCE_DIR = "spark/deploy/static" + + implicit val timeout = Timeout(10 seconds) - /* Starts the Web interface for the BlockManager */ - def start(actorSystem : ActorSystem, masterActor: ActorRef, sc: SparkContext) { - val webUIDirectives = new BlockManagerUIDirectives(actorSystem, masterActor, sc) + /** Start a HTTP server to run the Web interface */ + def start() { try { - // TODO: This needs to find a random free port to bind to. Unfortunately, there's no way - // in spray to do that, so we'll have to rely on something like new ServerSocket() - val boundPort = AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", - Option(System.getenv("BLOCKMANAGER_UI_PORT")).getOrElse("9080").toInt, - webUIDirectives.handler, "BlockManagerHTTPServer") - logInfo("Started BlockManager web UI at %s:%d".format(Utils.localHostName(), boundPort)) + val port = if (System.getProperty("spark.ui.port") != null) { + System.getProperty("spark.ui.port").toInt + } else { + // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which + // random port it bound to, so we have to try to find a local one by creating a socket. + Utils.findFreePort() + } + AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer") + logInfo("Started BlockManager web UI at http://%s:%d".format(Utils.localHostName(), port)) } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) @@ -34,58 +43,43 @@ object BlockManagerUI extends Logging { } } -} - - -private[spark] -class BlockManagerUIDirectives(val actorSystem: ActorSystem, master: ActorRef, - sc: SparkContext) extends Directives { - - val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Timeout(1 seconds) - val handler = { - - get { path("") { completeWith { - // Request the current storage status from the Master - val future = master ? GetStorageStatus - future.map { status => - val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray - - // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) - - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - - spark.storage.html.index. - render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) - } - }}} ~ - get { path("rdd") { parameter("id") { id => { completeWith { - val future = master ? GetStorageStatus - future.map { status => - val prefix = "rdd_" + id.toString - - - val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray - val filteredStorageStatusList = StorageUtils. - filterStorageStatusByPrefix(storageStatusList, prefix) - - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - - spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) - + get { + path("") { + completeWith { + // Request the current storage status from the Master + val future = blockManagerMaster ? GetStorageStatus + future.map { status => + // Calculate macro-level statistics + val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + spark.storage.html.index. + render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) + } + } + } ~ + path("rdd") { + parameter("id") { id => + completeWith { + val future = blockManagerMaster ? GetStorageStatus + future.map { status => + val prefix = "rdd_" + id.toString + val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray + val filteredStorageStatusList = StorageUtils. + filterStorageStatusByPrefix(storageStatusList, prefix) + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) + } + } + } + } ~ + pathPrefix("static") { + getFromResourceDirectory(STATIC_RESOURCE_DIR) } - }}}}} ~ - pathPrefix("static") { - getFromResourceDirectory(STATIC_RESOURCE_DIR) } - } - - - } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 775ff8f1aafc1..e0fdeffbc44d8 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,6 +1,6 @@ package spark.util -import akka.actor.{Props, ActorSystemImpl, ActorSystem} +import akka.actor.{ActorRef, Props, ActorSystemImpl, ActorSystem} import com.typesafe.config.ConfigFactory import akka.util.duration._ import akka.pattern.ask @@ -55,7 +55,7 @@ private[spark] object AkkaUtils { * handle requests. Returns the bound port or throws a SparkException on failure. */ def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, - name: String = "HttpServer"): Int = { + name: String = "HttpServer"): ActorRef = { val ioWorker = new IoWorker(actorSystem).start() val httpService = actorSystem.actorOf(Props(new HttpService(route))) val rootService = actorSystem.actorOf(Props(new SprayCanRootService(httpService))) @@ -67,7 +67,7 @@ private[spark] object AkkaUtils { try { Await.result(future, timeout) match { case bound: HttpServer.Bound => - return bound.endpoint.getPort + return server case other: Any => throw new SparkException("Failed to bind web UI to port " + port + ": " + other) } diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index 721c4c6029a50..51fb440108c65 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -5,6 +5,9 @@ import java.util.{TimerTask, Timer} import spark.Logging +/** + * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) + */ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging { val delaySeconds = MetadataCleaner.getDelaySeconds From 286f8f876ff495df33a7966e77ca90d69f338450 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 28 Jan 2013 01:29:27 -0800 Subject: [PATCH 082/696] Change time unit in MetadataCleaner to seconds --- core/src/main/scala/spark/util/MetadataCleaner.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index 51fb440108c65..6cf93a9b17055 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -9,7 +9,6 @@ import spark.Logging * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) */ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging { - val delaySeconds = MetadataCleaner.getDelaySeconds val periodSeconds = math.max(10, delaySeconds / 10) val timer = new Timer(name + " cleanup timer", true) @@ -39,7 +38,7 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging object MetadataCleaner { - def getDelaySeconds = (System.getProperty("spark.cleaner.delay", "-100").toDouble * 60).toInt - def setDelaySeconds(delay: Long) { System.setProperty("spark.cleaner.delay", delay.toString) } + def getDelaySeconds = System.getProperty("spark.cleaner.delay", "-1").toInt + def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.delay", delay.toString) } } From 07f568e1bfc67eead88e2c5dbfb9cac23e1ac8bc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 24 Jan 2013 15:27:29 -0800 Subject: [PATCH 083/696] SPARK-658: Adding logging of stage duration --- .../scala/spark/scheduler/DAGScheduler.scala | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bd541d4207953..8aad6671824bb 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -86,6 +86,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val activeJobs = new HashSet[ActiveJob] val resultStageToJob = new HashMap[Stage, ActiveJob] + val stageSubmissionTimes = new HashMap[Stage, Long] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) @@ -393,6 +394,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with logDebug("New pending tasks: " + myPending) taskSched.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority)) + if (!stageSubmissionTimes.contains(stage)) { + stageSubmissionTimes.put(stage, System.currentTimeMillis()) + } } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) @@ -407,6 +411,15 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with def handleTaskCompletion(event: CompletionEvent) { val task = event.task val stage = idToStage(task.stageId) + + def stageFinished(stage: Stage) = { + val serviceTime = stageSubmissionTimes.remove(stage) match { + case Some(t) => (System.currentTimeMillis() - t).toString + case _ => "Unkown" + } + logInfo("%s (%s) finished in %s ms".format(stage, stage.origin, serviceTime)) + running -= stage + } event.reason match { case Success => logInfo("Completed " + task) @@ -421,13 +434,13 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with if (!job.finished(rt.outputId)) { job.finished(rt.outputId) = true job.numFinished += 1 - job.listener.taskSucceeded(rt.outputId, event.result) // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { activeJobs -= job resultStageToJob -= stage - running -= stage + stageFinished(stage) } + job.listener.taskSucceeded(rt.outputId, event.result) } case None => logInfo("Ignoring result from " + rt + " because its job has finished") @@ -444,8 +457,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with stage.addOutputLoc(smt.partition, status) } if (running.contains(stage) && pendingTasks(stage).isEmpty) { - logInfo(stage + " (" + stage.origin + ") finished; looking for newly runnable stages") - running -= stage + stageFinished(stage) + logInfo("looking for newly runnable stages") logInfo("running: " + running) logInfo("waiting: " + waiting) logInfo("failed: " + failed) From c423be7d8e1349fc00431328b76b52f4eee8a975 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 24 Jan 2013 18:25:57 -0800 Subject: [PATCH 084/696] Renaming stage finished function --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 8aad6671824bb..bce7418e87485 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -412,7 +412,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val task = event.task val stage = idToStage(task.stageId) - def stageFinished(stage: Stage) = { + def markStageAsFinished(stage: Stage) = { val serviceTime = stageSubmissionTimes.remove(stage) match { case Some(t) => (System.currentTimeMillis() - t).toString case _ => "Unkown" @@ -438,7 +438,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with if (job.numFinished == job.numPartitions) { activeJobs -= job resultStageToJob -= stage - stageFinished(stage) + markStageAsFinished(stage) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -457,7 +457,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with stage.addOutputLoc(smt.partition, status) } if (running.contains(stage) && pendingTasks(stage).isEmpty) { - stageFinished(stage) + markStageAsFinished(stage) logInfo("looking for newly runnable stages") logInfo("running: " + running) logInfo("waiting: " + waiting) From 501433f1d59b1b326c0a7169fa1fd6136f7628e3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 28 Jan 2013 10:17:35 -0800 Subject: [PATCH 085/696] Making submission time a field --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 7 +++---- core/src/main/scala/spark/scheduler/Stage.scala | 3 +++ 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bce7418e87485..7ba1f3430a15a 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -86,7 +86,6 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val activeJobs = new HashSet[ActiveJob] val resultStageToJob = new HashMap[Stage, ActiveJob] - val stageSubmissionTimes = new HashMap[Stage, Long] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) @@ -394,8 +393,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with logDebug("New pending tasks: " + myPending) taskSched.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority)) - if (!stageSubmissionTimes.contains(stage)) { - stageSubmissionTimes.put(stage, System.currentTimeMillis()) + if (!stage.submissionTime.isDefined) { + stage.submissionTime = Some(System.currentTimeMillis()) } } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( @@ -413,7 +412,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val stage = idToStage(task.stageId) def markStageAsFinished(stage: Stage) = { - val serviceTime = stageSubmissionTimes.remove(stage) match { + val serviceTime = stage.submissionTime match { case Some(t) => (System.currentTimeMillis() - t).toString case _ => "Unkown" } diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index e9419728e3f34..374114d87034c 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -32,6 +32,9 @@ private[spark] class Stage( val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) var numAvailableOutputs = 0 + /** When first task was submitted to scheduler. */ + var submissionTime: Option[Long] = None + private var nextAttemptId = 0 def isAvailable: Boolean = { From da44a391fb74fc008211d5ec122cdcfc1726bc32 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 28 Jan 2013 11:16:14 -0800 Subject: [PATCH 086/696] Add an option to use the old scripts --- ec2/spark_ec2.py | 43 ++++++++++++++++++++++++++++++------------- 1 file changed, 30 insertions(+), 13 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index dfccb6c238411..cafb7bf011064 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -87,6 +87,9 @@ def parse_args(): parser.add_option("-g", "--ganglia", action="store_true", default=False, help="Setup ganglia monitoring for the cluster. NOTE: The ganglia " + "monitoring page will be publicly accessible") + parser.add_option("--mesos-scripts", action="store_true", default=False, + help="Use older mesos-ec2 scripts to setup the cluster. NOTE: Ganglia " + + "will not be setup with this option") parser.add_option("-u", "--user", default="root", help="The ssh user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, @@ -362,6 +365,13 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_key): + master = master_nodes[0].public_dns_name + if deploy_ssh_key: + print "Copying SSH key %s to master..." % opts.identity_file + ssh(master, opts, 'mkdir -p ~/.ssh') + scp(master, opts, opts.identity_file, '~/.ssh/id_rsa') + ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa') + if opts.cluster_type == "mesos": modules = ['ephemeral-hdfs', 'persistent-hdfs', 'mesos'] elif opts.cluster_type == "standalone": @@ -370,32 +380,39 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k if opts.ganglia: modules.append('ganglia') - master = master_nodes[0].public_dns_name - if deploy_ssh_key: - print "Copying SSH key %s to master..." % opts.identity_file - ssh(master, opts, 'mkdir -p ~/.ssh') - scp(master, opts, opts.identity_file, '~/.ssh/id_rsa') - ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa') + if not opts.mesos_scripts: + # NOTE: We should clone the repository before running deploy_files to + # prevent ec2-variables.sh from being overwritten + ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/shivaram/spark-ec2.git") - # NOTE: We should clone the repository before running deploy_files to prevent - # ec2-variables.sh from being overwritten - ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/shivaram/spark-ec2.git") print "Deploying files to master..." deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, zoo_nodes, modules) + print "Running setup on master..." - setup_spark_cluster(master, opts) + if opts.mesos_scripts: + if opts.cluster_type == "mesos": + setup_mesos_cluster(master, opts) + elif opts.cluster_type == "standalone": + setup_standalone_cluster(master, slave_nodes, opts) + else: + setup_spark_cluster(master, opts) print "Done!" -def setup_spark_cluster(master, opts): - ssh(master, opts, "chmod u+x spark-ec2/setup.sh") - ssh(master, opts, "spark-ec2/setup.sh") +def setup_mesos_cluster(master, opts): + ssh(master, opts, "chmod u+x mesos-ec2/setup") + ssh(master, opts, "mesos-ec2/setup %s %s %s %s" % + ("generic", "none", "master", opts.swap)) def setup_standalone_cluster(master, slave_nodes, opts): slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) ssh(master, opts, "/root/spark/bin/start-all.sh") +def setup_spark_cluster(master, opts): + ssh(master, opts, "chmod u+x spark-ec2/setup.sh") + ssh(master, opts, "spark-ec2/setup.sh") + # Wait for a whole cluster (masters, slaves and ZooKeeper) to start up def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes, zoo_nodes): From a423ee546c389b5ce0d2117299456712370d7ad1 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 22 Jan 2013 18:48:43 -0800 Subject: [PATCH 087/696] expose RDD & storage info directly via SparkContext --- core/src/main/scala/spark/SparkContext.scala | 16 ++++++++ .../spark/storage/BlockManagerMaster.scala | 4 ++ .../scala/spark/storage/BlockManagerUI.scala | 39 +++++++------------ .../scala/spark/storage/StorageUtils.scala | 10 +++-- 4 files changed, 41 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 77036c1275fa5..be992250a9b58 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -46,6 +46,7 @@ import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, C import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import storage.BlockManagerUI import util.{MetadataCleaner, TimeStampedHashMap} +import storage.{StorageStatus, StorageUtils, RDDInfo} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -473,6 +474,21 @@ class SparkContext( } } + /** + * Return information about what RDDs are cached, if they are in mem or on disk, how much space + * they take, etc. + */ + def getRDDStorageInfo : Array[RDDInfo] = { + StorageUtils.rddInfoFromStorageStatus(getSlavesStorageStatus, this) + } + + /** + * Return information about blocks stored in all of the slaves + */ + def getSlavesStorageStatus : Array[StorageStatus] = { + env.blockManager.master.getStorageStatus + } + /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 55ff1dde9cc35..c7ee76f0b7b7c 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -118,6 +118,10 @@ private[spark] class BlockManagerMaster( askMasterWithRetry[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) } + def getStorageStatus: Array[StorageStatus] = { + askMasterWithRetry[ArrayBuffer[StorageStatus]](GetStorageStatus).toArray + } + /** Stop the master actor, called only on the Spark master node */ def stop() { if (masterActor != null) { diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index eda320fa47972..52f6d1b657fda 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -1,13 +1,10 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} -import akka.pattern.ask import akka.util.Timeout import akka.util.duration._ -import cc.spray.directives._ import cc.spray.typeconversion.TwirlSupport._ import cc.spray.Directives -import scala.collection.mutable.ArrayBuffer import spark.{Logging, SparkContext} import spark.util.AkkaUtils import spark.Utils @@ -48,32 +45,26 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, path("") { completeWith { // Request the current storage status from the Master - val future = blockManagerMaster ? GetStorageStatus - future.map { status => - // Calculate macro-level statistics - val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - spark.storage.html.index. - render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) - } + val storageStatusList = sc.getSlavesStorageStatus + // Calculate macro-level statistics + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + spark.storage.html.index. + render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) } } ~ path("rdd") { parameter("id") { id => completeWith { - val future = blockManagerMaster ? GetStorageStatus - future.map { status => - val prefix = "rdd_" + id.toString - val storageStatusList = status.asInstanceOf[ArrayBuffer[StorageStatus]].toArray - val filteredStorageStatusList = StorageUtils. - filterStorageStatusByPrefix(storageStatusList, prefix) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) - } + val prefix = "rdd_" + id.toString + val storageStatusList = sc.getSlavesStorageStatus + val filteredStorageStatusList = StorageUtils. + filterStorageStatusByPrefix(storageStatusList, prefix) + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) } } } ~ diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index a10e3a95c60d3..d6e33c8619a67 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -56,9 +56,11 @@ object StorageUtils { // Find the id of the RDD, e.g. rdd_1 => 1 val rddId = rddKey.split("_").last.toInt // Get the friendly name for the rdd, if available. - val rddName = Option(sc.persistentRdds(rddId).name).getOrElse(rddKey) - val rddStorageLevel = sc.persistentRdds(rddId).getStorageLevel - + val rdd = sc.persistentRdds(rddId) + val rddName = Option(rdd.name).getOrElse(rddKey) + val rddStorageLevel = rdd.getStorageLevel + //TODO get total number of partitions in rdd + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, memSize, diskSize) }.toArray } @@ -75,4 +77,4 @@ object StorageUtils { } -} \ No newline at end of file +} From 0f22c4207f27bc8d1675af82f873141dda754f5c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 10:08:59 -0800 Subject: [PATCH 088/696] better formatting for RDDInfo --- core/src/main/scala/spark/storage/StorageUtils.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index d6e33c8619a67..ce7c067eea57e 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -1,6 +1,6 @@ package spark.storage -import spark.SparkContext +import spark.{Utils, SparkContext} import BlockManagerMasterActor.BlockStatus private[spark] @@ -22,8 +22,14 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numPartitions: Int, memSize: Long, diskSize: Long) - + numPartitions: Int, memSize: Long, diskSize: Long) { + override def toString = { + import Utils.memoryBytesToString + import java.lang.{Integer => JInt} + String.format("RDD \"%s\" (%d) Storage: %s; Partitions: %d; MemorySize: %s; DiskSize: %s", name, id.asInstanceOf[JInt], + storageLevel.toString, numPartitions.asInstanceOf[JInt], memoryBytesToString(memSize), memoryBytesToString(diskSize)) + } +} /* Helper methods for storage-related objects */ private[spark] From b14841455ca44b6826d09585b6cc40b0a2aabce0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 09:24:41 -0800 Subject: [PATCH 089/696] track task completion in DAGScheduler, and send a stageCompleted event with taskInfo to SparkListeners --- .../scala/spark/scheduler/DAGScheduler.scala | 16 ++++++++++++++-- .../spark/scheduler/DAGSchedulerEvent.scala | 4 +++- .../scala/spark/scheduler/SparkListener.scala | 9 +++++++++ .../main/scala/spark/scheduler/StageInfo.scala | 6 ++++++ .../spark/scheduler/TaskSchedulerListener.scala | 3 ++- .../spark/scheduler/cluster/TaskSetManager.scala | 6 +++--- .../spark/scheduler/local/LocalScheduler.scala | 7 +++++-- 7 files changed, 42 insertions(+), 9 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/SparkListener.scala create mode 100644 core/src/main/scala/spark/scheduler/StageInfo.scala diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bd541d4207953..63f359181b12d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -1,5 +1,6 @@ package spark.scheduler +import cluster.TaskInfo import java.net.URI import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.Future @@ -31,8 +32,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any]) { - eventQueue.put(CompletionEvent(task, reason, result, accumUpdates)) + accumUpdates: Map[Long, Any], + taskInfo: TaskInfo) { + eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo)) } // Called by TaskScheduler when an executor fails. @@ -64,6 +66,10 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] + private val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + + private val sparkListeners = Traversable[SparkListener]() + var cacheLocs = new HashMap[Int, Array[List[String]]] val env = SparkEnv.get @@ -141,6 +147,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val id = nextStageId.getAndIncrement() val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) idToStage(id) = stage + stageToInfos(stage) = StageInfo(stage) stage } @@ -414,6 +421,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task + stageToInfos(stage).taskInfos += event.taskInfo task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { @@ -427,6 +435,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with activeJobs -= job resultStageToJob -= stage running -= stage + val stageComp = StageCompleted(stageToInfos(stage)) + sparkListeners.foreach{_.onStageCompleted(stageComp)} } } case None => @@ -446,6 +456,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with if (running.contains(stage) && pendingTasks(stage).isEmpty) { logInfo(stage + " (" + stage.origin + ") finished; looking for newly runnable stages") running -= stage + val stageComp = StageCompleted(stageToInfos(stage)) + sparkListeners.foreach{_.onStageCompleted(stageComp)} logInfo("running: " + running) logInfo("waiting: " + waiting) logInfo("failed: " + failed) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index b34fa78c072c0..9f19e137e4131 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,5 +1,6 @@ package spark.scheduler +import cluster.TaskInfo import scala.collection.mutable.Map import spark._ @@ -25,7 +26,8 @@ private[spark] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any]) + accumUpdates: Map[Long, Any], + taskInfo: TaskInfo) extends DAGSchedulerEvent private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala new file mode 100644 index 0000000000000..54afc714b3b57 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -0,0 +1,9 @@ +package spark.scheduler + +trait SparkListener { + def onStageCompleted(stageCompleted: StageCompleted) +} + +sealed trait SparkListenerEvents + +case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala new file mode 100644 index 0000000000000..473bdb1c14a0d --- /dev/null +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -0,0 +1,6 @@ +package spark.scheduler + +import cluster.TaskInfo +import collection._ + +case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 9fcef86e46a29..4ede03cc2a1f0 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -1,5 +1,6 @@ package spark.scheduler +import cluster.TaskInfo import scala.collection.mutable.Map import spark.TaskEndReason @@ -9,7 +10,7 @@ import spark.TaskEndReason */ private[spark] trait TaskSchedulerListener { // A task has finished or failed. - def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any]): Unit + def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo): Unit // A node was lost from the cluster. def executorLost(execId: String): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 26201ad0dd1a0..0a57aeda8bcb6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -262,7 +262,7 @@ private[spark] class TaskSetManager( tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates) + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info) // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { @@ -293,7 +293,7 @@ private[spark] class TaskSetManager( reason match { case fetchFailed: FetchFailed => logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.listener.taskEnded(tasks(index), fetchFailed, null, null) + sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info) finished(index) = true tasksFinished += 1 sched.taskSetFinished(this) @@ -381,7 +381,7 @@ private[spark] class TaskSetManager( addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. - sched.listener.taskEnded(tasks(index), Resubmitted, null, null) + sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info) } } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 9ff7c02097b2e..995ab79b5ff95 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -9,6 +9,7 @@ import scala.collection.mutable.HashMap import spark._ import executor.ExecutorURLClassLoader import spark.scheduler._ +import cluster.TaskInfo /** * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally @@ -54,6 +55,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon def runTask(task: Task[_], idInJob: Int, attemptId: Int) { logInfo("Running task " + idInJob) + val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local") // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { @@ -81,10 +83,11 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( ser.serialize(Accumulators.values)) logInfo("Finished task " + idInJob) + info.markSuccessful() // If the threadpool has not already been shutdown, notify DAGScheduler if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, Success, resultToReturn, accumUpdates) + listener.taskEnded(task, Success, resultToReturn, accumUpdates, info) } catch { case t: Throwable => { logError("Exception in task " + idInJob, t) @@ -95,7 +98,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } else { // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, new ExceptionFailure(t), null, null) + listener.taskEnded(task, new ExceptionFailure(t), null, null, info) } } } From b88daee91687f35766a4daa523ba780bc1f412d2 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 18:20:37 -0800 Subject: [PATCH 090/696] simple util to summarize distributions --- .../main/scala/spark/util/Distribution.scala | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 core/src/main/scala/spark/util/Distribution.scala diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala new file mode 100644 index 0000000000000..99bbb43ff1de0 --- /dev/null +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -0,0 +1,53 @@ +package spark.util + +import java.io.PrintStream + +/** + * util for getting some stats from a small sample of numeric values, with some handy summary functions + * + * Entirely in memory, not intended as a good way to compute stats over large data sets. + */ +class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { + def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) + java.util.Arrays.sort(data, startIdx, endIdx) + val length = endIdx - startIdx + + val defaultProbabilities = Array(0,0.25,0.5,0.75,1.0) + + /** + * Get the value of the distribution at the given probabilities. Probabilities should be + * given from 0 to 1 + * @param probabilities + */ + def getQuantiles(probabilities: Traversable[Double]) = { + probabilities.map{q =>data((q * length).toInt + startIdx)} + } + + def showQuantiles(out: PrintStream = System.out, probabilities: Traversable[Double] = defaultProbabilities) = { + out.println("min\t25%\t50%\t75%max") + probabilities.foreach{q => out.print(q + "\t")} + out.println + } + + def summary : (StatCounter, Traversable[Double]) = { + (StatCounter(data), getQuantiles(defaultProbabilities)) + } + + /** + * print a summary of this distribution to the given PrintStream. + * @param out + */ + def summary(out: PrintStream = System.out) { + val (statCounter, quantiles) = summary + out.println(statCounter) + Distribution.showQuantiles(out, quantiles) + } +} + +object Distribution { + def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) { + out.println("min\t25%\t50%\t75%max") + quantiles.foreach{q => out.print(q + "\t")} + out.println + } +} \ No newline at end of file From 38b83bc66bd440810778b6bc543766e25c084112 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 18:20:57 -0800 Subject: [PATCH 091/696] can get task runtime summary from task info --- core/src/main/scala/spark/scheduler/StageInfo.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 473bdb1c14a0d..3b9c6d87f0fd1 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -2,5 +2,10 @@ package spark.scheduler import cluster.TaskInfo import collection._ +import spark.util.Distribution -case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) +case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) { + def getTaskRuntimeDistribution = { + new Distribution(taskInfos.map{_.duration.toDouble}) + } +} From 01d77f329f5878b7c8672bbdc1859f3ca95d759d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 18:32:52 -0800 Subject: [PATCH 092/696] expose stageInfo in SparkContext --- core/src/main/scala/spark/SparkContext.scala | 9 ++++++++- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index be992250a9b58..38717a2c1728a 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -40,13 +40,16 @@ import spark.deploy.LocalSparkCluster import spark.partial.ApproximateEvaluator import spark.partial.PartialResult import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD} -import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler} +import scheduler._ import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import storage.BlockManagerUI +import storage.RDDInfo +import storage.StorageStatus import util.{MetadataCleaner, TimeStampedHashMap} import storage.{StorageStatus, StorageUtils, RDDInfo} +import scala.Some /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -482,6 +485,10 @@ class SparkContext( StorageUtils.rddInfoFromStorageStatus(getSlavesStorageStatus, this) } + def getStageInfo: Map[Stage,StageInfo] = { + dagScheduler.stageToInfos + } + /** * Return information about blocks stored in all of the slaves */ diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 63f359181b12d..3265d158fb1a8 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -66,7 +66,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] - private val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] private val sparkListeners = Traversable[SparkListener]() From cec9c768c2eed8d4eefaae0003eeb3971317cef9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 20:02:53 -0800 Subject: [PATCH 093/696] convenient name available in StageInfo --- core/src/main/scala/spark/scheduler/StageInfo.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 3b9c6d87f0fd1..21c6d1aee0064 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -5,6 +5,9 @@ import collection._ import spark.util.Distribution case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) { + + def name = stage.rdd.name + "(" + stage.origin + ")" + def getTaskRuntimeDistribution = { new Distribution(taskInfos.map{_.duration.toDouble}) } From efff7bfb3382f4e07f9fad0e6e647c0ec629355e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Jan 2013 20:23:11 -0800 Subject: [PATCH 094/696] add long and float accumulatorparams --- core/src/main/scala/spark/SparkContext.scala | 10 ++++++++++ core/src/test/scala/spark/AccumulatorSuite.scala | 6 ++++++ 2 files changed, 16 insertions(+) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 77036c1275fa5..dc9b8688b3c8a 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -673,6 +673,16 @@ object SparkContext { def zero(initialValue: Int) = 0 } + implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + def addInPlace(t1: Long, t2: Long) = t1 + t2 + def zero(initialValue: Long) = 0l + } + + implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + def addInPlace(t1: Float, t2: Float) = t1 + t2 + def zero(initialValue: Float) = 0f + } + // TODO: Add AccumulatorParams for other types, e.g. lists and strings implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index 78d64a44aed0c..ac8ae7d308de8 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -17,6 +17,12 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte val d = sc.parallelize(1 to 20) d.foreach{x => acc += x} acc.value should be (210) + + + val longAcc = sc.accumulator(0l) + val maxInt = Integer.MAX_VALUE.toLong + d.foreach{x => longAcc += maxInt + x} + longAcc.value should be (210l + maxInt * 20) } test ("value not assignable from tasks") { From 1f9b486a8be49ef547ac1532cafd63c4c9d4ddda Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 28 Jan 2013 20:24:54 -0800 Subject: [PATCH 095/696] Some DEBUG-level log cleanup. A few changes to make the DEBUG-level logs less noisy and more readable. - Moved a few very frequent messages to Trace - Changed some BlockManger log messages to make them more understandable SPARK-666 #resolve --- .../main/scala/spark/scheduler/DAGScheduler.scala | 8 ++++---- .../main/scala/spark/storage/BlockManager.scala | 14 +++++++------- .../spark/storage/BlockManagerMasterActor.scala | 2 +- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bd541d4207953..f10d7cc84ebcb 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -308,10 +308,10 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } else { // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. - logDebug("Checking for newly runnable parent stages") - logDebug("running: " + running) - logDebug("waiting: " + waiting) - logDebug("failed: " + failed) + logTrace("Checking for newly runnable parent stages") + logTrace("running: " + running) + logTrace("waiting: " + waiting) + logTrace("failed: " + failed) val waiting2 = waiting.toArray waiting.clear() for (stage <- waiting2.sortBy(_.priority)) { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 1215d5f5c8a3d..c61fd75c2bc11 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -243,7 +243,7 @@ class BlockManager( val startTimeMs = System.currentTimeMillis var managers = master.getLocations(blockId) val locations = managers.map(_.ip) - logDebug("Get block locations in " + Utils.getUsedTimeMs(startTimeMs)) + logDebug("Got block locations in " + Utils.getUsedTimeMs(startTimeMs)) return locations } @@ -253,7 +253,7 @@ class BlockManager( def getLocations(blockIds: Array[String]): Array[Seq[String]] = { val startTimeMs = System.currentTimeMillis val locations = master.getLocations(blockIds).map(_.map(_.ip).toSeq).toArray - logDebug("Get multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) + logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) return locations } @@ -645,7 +645,7 @@ class BlockManager( var size = 0L myInfo.synchronized { - logDebug("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") if (level.useMemory) { @@ -677,8 +677,10 @@ class BlockManager( } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) + // Replicate block if required if (level.replication > 1) { + val remoteStartTime = System.currentTimeMillis // Serialize the block if not already done if (bytesAfterPut == null) { if (valuesAfterPut == null) { @@ -688,12 +690,10 @@ class BlockManager( bytesAfterPut = dataSerialize(blockId, valuesAfterPut) } replicate(blockId, bytesAfterPut, level) + logDebug("Put block " + blockId + " remotely took " + Utils.getUsedTimeMs(remoteStartTime)) } - BlockManager.dispose(bytesAfterPut) - logDebug("Put block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)) - return size } @@ -978,7 +978,7 @@ object BlockManager extends Logging { */ def dispose(buffer: ByteBuffer) { if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) { - logDebug("Unmapping " + buffer) + logTrace("Unmapping " + buffer) if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) { buffer.asInstanceOf[DirectBuffer].cleaner().clean() } diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index f88517f1a39ec..2830bc62975ae 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -115,7 +115,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } def expireDeadHosts() { - logDebug("Checking for hosts with no recent heart beats in BlockManagerMaster.") + logTrace("Checking for hosts with no recent heart beats in BlockManagerMaster.") val now = System.currentTimeMillis() val minSeenTime = now - slaveTimeout val toRemove = new HashSet[BlockManagerId] From 7ee824e42ebaa1fc0b0248e0a35021108625ed14 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 28 Jan 2013 21:48:32 -0800 Subject: [PATCH 096/696] Units from ms -> s --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 7ba1f3430a15a..b8336d9d0620c 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -413,10 +413,10 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with def markStageAsFinished(stage: Stage) = { val serviceTime = stage.submissionTime match { - case Some(t) => (System.currentTimeMillis() - t).toString + case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0) case _ => "Unkown" } - logInfo("%s (%s) finished in %s ms".format(stage, stage.origin, serviceTime)) + logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime)) running -= stage } event.reason match { From b45857c965219e2d26f35adb2ea3a2b831fdb77f Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Mon, 28 Jan 2013 23:56:56 -0600 Subject: [PATCH 097/696] Add RDD.toDebugString. Original idea by Nathan Kronenfeld. --- core/src/main/scala/spark/RDD.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 0d3857f9dd488..172431c31a2b8 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -638,4 +638,14 @@ abstract class RDD[T: ClassManifest]( protected[spark] def clearDependencies() { dependencies_ = null } + + /** A description of this RDD and its recursive dependencies for debugging. */ + def toDebugString(): String = { + def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = { + Seq(prefix + rdd) ++ rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) + } + debugString(this).mkString("\n") + } + + override def toString() = "%s[%d] at %s".format(getClass.getSimpleName, id, origin) } From 951cfd9ba2a9239a777f156f10af820e9df49606 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 29 Jan 2013 00:02:17 -0600 Subject: [PATCH 098/696] Add JavaRDDLike.toDebugString(). --- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 4c95c989b5367..44f778e5c2eb6 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -330,4 +330,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround case _ => Optional.absent() } } + + /** A description of this RDD and its recursive dependencies for debugging. */ + def toDebugString(): String = { + rdd.toDebugString() + } } From 3cda14af3fea97c2372c7335505e9dad7e0dd117 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 29 Jan 2013 00:12:31 -0600 Subject: [PATCH 099/696] Add number of splits. --- core/src/main/scala/spark/RDD.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 172431c31a2b8..39bacd2afb934 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -642,7 +642,8 @@ abstract class RDD[T: ClassManifest]( /** A description of this RDD and its recursive dependencies for debugging. */ def toDebugString(): String = { def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = { - Seq(prefix + rdd) ++ rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) + Seq(prefix + rdd + " (" + rdd.splits.size + " splits)") ++ + rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) } debugString(this).mkString("\n") } From cbf72bffa5874319c7ee7117a073e9d01fa51585 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 29 Jan 2013 00:20:36 -0600 Subject: [PATCH 100/696] Include name, if set, in RDD.toString(). --- core/src/main/scala/spark/RDD.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 39bacd2afb934..a23441483e36e 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -648,5 +648,10 @@ abstract class RDD[T: ClassManifest]( debugString(this).mkString("\n") } - override def toString() = "%s[%d] at %s".format(getClass.getSimpleName, id, origin) + override def toString(): String = "%s%s[%d] at %s".format( + Option(name).map(_ + " ").getOrElse(""), + getClass.getSimpleName, + id, + origin) + } From b29599e5cf0272f0d0e3ceceebb473a8163eab8c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 28 Jan 2013 22:24:47 -0800 Subject: [PATCH 101/696] Fix code that depended on metadata cleaner interval being in minutes --- streaming/src/main/scala/spark/streaming/DStream.scala | 8 ++++---- .../src/main/scala/spark/streaming/StreamingContext.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index b11ef443dc5e2..352f83fe0c33b 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -198,10 +198,10 @@ abstract class DStream[T: ClassManifest] ( metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000, "It seems you are doing some DStream window operation or setting a checkpoint interval " + "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " + - "than " + rememberDuration.milliseconds + " milliseconds. But the Spark's metadata cleanup" + - "delay is set to " + (metadataCleanerDelay / 60.0) + " minutes, which is not sufficient. Please set " + - "the Java property 'spark.cleaner.delay' to more than " + - math.ceil(rememberDuration.milliseconds.toDouble / 60000.0).toInt + " minutes." + "than " + rememberDuration.milliseconds / 1000 + " seconds. But Spark's metadata cleanup" + + "delay is set to " + metadataCleanerDelay + " seconds, which is not sufficient. Please " + + "set the Java property 'spark.cleaner.delay' to more than " + + math.ceil(rememberDuration.milliseconds / 1000.0).toInt + " seconds." ) dependencies.foreach(_.validate()) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 14500bdcb17a0..37ba524b4897e 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -389,7 +389,7 @@ object StreamingContext { // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second interval. if (MetadataCleaner.getDelaySeconds < 0) { - MetadataCleaner.setDelaySeconds(60) + MetadataCleaner.setDelaySeconds(3600) } new SparkContext(master, frameworkName) } From 64ba6a8c2c5f46e6de6deb6a6fd576a55cb3b198 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 28 Jan 2013 22:30:12 -0800 Subject: [PATCH 102/696] Simplify checkpointing code and RDD class a little: - RDD's getDependencies and getSplits methods are now guaranteed to be called only once, so subclasses can safely do computation in there without worrying about caching the results. - The management of a "splits_" variable that is cleared out when we checkpoint an RDD is now done in the RDD class. - A few of the RDD subclasses are simpler. - CheckpointRDD's compute() method no longer assumes that it is given a CheckpointRDDSplit -- it can work just as well on a split from the original RDD, because it only looks at its index. This is important because things like UnionRDD and ZippedRDD remember the parent's splits as part of their own and wouldn't work on checkpointed parents. - RDD.iterator can now reuse cached data if an RDD is computed before it is checkpointed. It seems like it wouldn't do this before (it always called iterator() on the CheckpointRDD, which read from HDFS). --- core/src/main/scala/spark/CacheManager.scala | 6 +- .../main/scala/spark/PairRDDFunctions.scala | 4 +- core/src/main/scala/spark/RDD.scala | 130 ++++++++++-------- .../main/scala/spark/RDDCheckpointData.scala | 19 +-- .../scala/spark/api/java/JavaRDDLike.scala | 2 +- .../main/scala/spark/rdd/CartesianRDD.scala | 12 +- .../main/scala/spark/rdd/CheckpointRDD.scala | 61 ++++---- .../main/scala/spark/rdd/CoalescedRDD.scala | 14 +- core/src/main/scala/spark/rdd/MappedRDD.scala | 6 +- .../scala/spark/rdd/PartitionPruningRDD.scala | 13 +- .../main/scala/spark/rdd/ShuffledRDD.scala | 8 +- core/src/main/scala/spark/rdd/UnionRDD.scala | 14 +- core/src/main/scala/spark/rdd/ZippedRDD.scala | 7 +- .../scala/spark/util/MetadataCleaner.scala | 4 +- .../test/scala/spark/CheckpointSuite.scala | 21 +-- 15 files changed, 153 insertions(+), 168 deletions(-) diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala index a0b53fd9d67c5..711435c3335fa 100644 --- a/core/src/main/scala/spark/CacheManager.scala +++ b/core/src/main/scala/spark/CacheManager.scala @@ -10,9 +10,9 @@ import spark.storage.{BlockManager, StorageLevel} private[spark] class CacheManager(blockManager: BlockManager) extends Logging { private val loading = new HashSet[String] - /** Gets or computes an RDD split. Used by RDD.iterator() when a RDD is cached. */ + /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */ def getOrCompute[T](rdd: RDD[T], split: Split, context: TaskContext, storageLevel: StorageLevel) - : Iterator[T] = { + : Iterator[T] = { val key = "rdd_%d_%d".format(rdd.id, split.index) logInfo("Cache key is " + key) blockManager.get(key) match { @@ -50,7 +50,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // If we got here, we have to load the split val elements = new ArrayBuffer[Any] logInfo("Computing partition " + split) - elements ++= rdd.compute(split, context) + elements ++= rdd.computeOrReadCheckpoint(split, context) // Try to put this block in the blockManager blockManager.put(key, elements, storageLevel, true) return elements.iterator.asInstanceOf[Iterator[T]] diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 53b051f1c523a..231e23a7deacc 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -649,9 +649,7 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( } private[spark] -class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) - extends RDD[(K, U)](prev) { - +class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev) { override def getSplits = firstParent[(K, V)].splits override val partitioner = firstParent[(K, V)].partitioner override def compute(split: Split, context: TaskContext) = diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 0d3857f9dd488..dbad6d4c838eb 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -1,27 +1,17 @@ package spark -import java.io.{ObjectOutputStream, IOException, EOFException, ObjectInputStream} import java.net.URL import java.util.{Date, Random} import java.util.{HashMap => JHashMap} -import java.util.concurrent.atomic.AtomicLong import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import org.apache.hadoop.fs.Path import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred.FileOutputCommitter -import org.apache.hadoop.mapred.HadoopWriter -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.OutputCommitter -import org.apache.hadoop.mapred.OutputFormat -import org.apache.hadoop.mapred.SequenceFileOutputFormat import org.apache.hadoop.mapred.TextOutputFormat import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} @@ -30,7 +20,6 @@ import spark.partial.BoundedDouble import spark.partial.CountEvaluator import spark.partial.GroupedCountEvaluator import spark.partial.PartialResult -import spark.rdd.BlockRDD import spark.rdd.CartesianRDD import spark.rdd.FilteredRDD import spark.rdd.FlatMappedRDD @@ -73,11 +62,11 @@ import SparkContext._ * on RDD internals. */ abstract class RDD[T: ClassManifest]( - @transient var sc: SparkContext, - var dependencies_ : List[Dependency[_]] + @transient private var sc: SparkContext, + @transient private var deps: Seq[Dependency[_]] ) extends Serializable with Logging { - + /** Construct an RDD with just a one-to-one dependency on one parent */ def this(@transient oneParent: RDD[_]) = this(oneParent.context , List(new OneToOneDependency(oneParent))) @@ -85,25 +74,27 @@ abstract class RDD[T: ClassManifest]( // Methods that should be implemented by subclasses of RDD // ======================================================================= - /** Function for computing a given partition. */ + /** Implemented by subclasses to compute a given partition. */ def compute(split: Split, context: TaskContext): Iterator[T] - /** Set of partitions in this RDD. */ - protected def getSplits(): Array[Split] + /** + * Implemented by subclasses to return the set of partitions in this RDD. This method will only + * be called once, so it is safe to implement a time-consuming computation in it. + */ + protected def getSplits: Array[Split] - /** How this RDD depends on any parent RDDs. */ - protected def getDependencies(): List[Dependency[_]] = dependencies_ + /** + * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only + * be called once, so it is safe to implement a time-consuming computation in it. + */ + protected def getDependencies: Seq[Dependency[_]] = deps - /** A friendly name for this RDD */ - var name: String = null - /** Optionally overridden by subclasses to specify placement preferences. */ protected def getPreferredLocations(split: Split): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ val partitioner: Option[Partitioner] = None - // ======================================================================= // Methods and fields available on all RDDs // ======================================================================= @@ -111,13 +102,16 @@ abstract class RDD[T: ClassManifest]( /** A unique ID for this RDD (within its SparkContext). */ val id = sc.newRddId() + /** A friendly name for this RDD */ + var name: String = null + /** Assign a name to this RDD */ def setName(_name: String) = { name = _name this } - /** + /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. Can only be called once on each RDD. */ @@ -142,15 +136,24 @@ abstract class RDD[T: ClassManifest]( /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ def getStorageLevel = storageLevel + // Our dependencies and splits will be gotten by calling subclass's methods below, and will + // be overwritten when we're checkpointed + private var dependencies_ : Seq[Dependency[_]] = null + @transient private var splits_ : Array[Split] = null + + /** An Option holding our checkpoint RDD, if we are checkpointed */ + private def checkpointRDD: Option[RDD[T]] = checkpointData.flatMap(_.checkpointRDD) + /** - * Get the preferred location of a split, taking into account whether the + * Get the list of dependencies of this RDD, taking into account whether the * RDD is checkpointed or not. */ - final def preferredLocations(split: Split): Seq[String] = { - if (isCheckpointed) { - checkpointData.get.getPreferredLocations(split) - } else { - getPreferredLocations(split) + final def dependencies: Seq[Dependency[_]] = { + checkpointRDD.map(r => List(new OneToOneDependency(r))).getOrElse { + if (dependencies_ == null) { + dependencies_ = getDependencies + } + dependencies_ } } @@ -159,22 +162,21 @@ abstract class RDD[T: ClassManifest]( * RDD is checkpointed or not. */ final def splits: Array[Split] = { - if (isCheckpointed) { - checkpointData.get.getSplits - } else { - getSplits + checkpointRDD.map(_.splits).getOrElse { + if (splits_ == null) { + splits_ = getSplits + } + splits_ } } /** - * Get the list of dependencies of this RDD, taking into account whether the + * Get the preferred location of a split, taking into account whether the * RDD is checkpointed or not. */ - final def dependencies: List[Dependency[_]] = { - if (isCheckpointed) { - dependencies_ - } else { - getDependencies + final def preferredLocations(split: Split): Seq[String] = { + checkpointRDD.map(_.getPreferredLocations(split)).getOrElse { + getPreferredLocations(split) } } @@ -184,10 +186,19 @@ abstract class RDD[T: ClassManifest]( * subclasses of RDD. */ final def iterator(split: Split, context: TaskContext): Iterator[T] = { - if (isCheckpointed) { - checkpointData.get.iterator(split, context) - } else if (storageLevel != StorageLevel.NONE) { + if (storageLevel != StorageLevel.NONE) { SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel) + } else { + computeOrReadCheckpoint(split, context) + } + } + + /** + * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. + */ + private[spark] def computeOrReadCheckpoint(split: Split, context: TaskContext): Iterator[T] = { + if (isCheckpointed) { + firstParent[T].iterator(split, context) } else { compute(split, context) } @@ -578,15 +589,15 @@ abstract class RDD[T: ClassManifest]( /** * Return whether this RDD has been checkpointed or not */ - def isCheckpointed(): Boolean = { - if (checkpointData.isDefined) checkpointData.get.isCheckpointed() else false + def isCheckpointed: Boolean = { + checkpointData.map(_.isCheckpointed).getOrElse(false) } /** * Gets the name of the file to which this RDD was checkpointed */ - def getCheckpointFile(): Option[String] = { - if (checkpointData.isDefined) checkpointData.get.getCheckpointFile() else None + def getCheckpointFile: Option[String] = { + checkpointData.flatMap(_.getCheckpointFile) } // ======================================================================= @@ -611,31 +622,36 @@ abstract class RDD[T: ClassManifest]( def context = sc /** - * Performs the checkpointing of this RDD by saving this . It is called by the DAGScheduler + * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler * after a job using this RDD has completed (therefore the RDD has been materialized and * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. */ - protected[spark] def doCheckpoint() { - if (checkpointData.isDefined) checkpointData.get.doCheckpoint() - dependencies.foreach(_.rdd.doCheckpoint()) + private[spark] def doCheckpoint() { + if (checkpointData.isDefined) { + checkpointData.get.doCheckpoint() + } else { + dependencies.foreach(_.rdd.doCheckpoint()) + } } /** - * Changes the dependencies of this RDD from its original parents to the new RDD - * (`newRDD`) created from the checkpoint file. + * Changes the dependencies of this RDD from its original parents to a new RDD (`newRDD`) + * created from the checkpoint file, and forget its old dependencies and splits. */ - protected[spark] def changeDependencies(newRDD: RDD[_]) { + private[spark] def markCheckpointed(checkpointRDD: RDD[_]) { clearDependencies() - dependencies_ = List(new OneToOneDependency(newRDD)) + dependencies_ = null + splits_ = null + deps = null // Forget the constructor argument for dependencies too } /** * Clears the dependencies of this RDD. This method must ensure that all references * to the original parent RDDs is removed to enable the parent RDDs to be garbage * collected. Subclasses of RDD may override this method for implementing their own cleaning - * logic. See [[spark.rdd.UnionRDD]] and [[spark.rdd.ShuffledRDD]] to get a better idea. + * logic. See [[spark.rdd.UnionRDD]] for an example. */ - protected[spark] def clearDependencies() { + protected def clearDependencies() { dependencies_ = null } } diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index 18df530b7d526..a4a4ebaf53af8 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -20,7 +20,7 @@ private[spark] object CheckpointState extends Enumeration { * of the checkpointed RDD. */ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) -extends Logging with Serializable { + extends Logging with Serializable { import CheckpointState._ @@ -31,7 +31,7 @@ extends Logging with Serializable { @transient var cpFile: Option[String] = None // The CheckpointRDD created from the checkpoint file, that is, the new parent the associated RDD. - @transient var cpRDD: Option[RDD[T]] = None + var cpRDD: Option[RDD[T]] = None // Mark the RDD for checkpointing def markForCheckpoint() { @@ -41,12 +41,12 @@ extends Logging with Serializable { } // Is the RDD already checkpointed - def isCheckpointed(): Boolean = { + def isCheckpointed: Boolean = { RDDCheckpointData.synchronized { cpState == Checkpointed } } // Get the file to which this RDD was checkpointed to as an Option - def getCheckpointFile(): Option[String] = { + def getCheckpointFile: Option[String] = { RDDCheckpointData.synchronized { cpFile } } @@ -71,7 +71,7 @@ extends Logging with Serializable { RDDCheckpointData.synchronized { cpFile = Some(path) cpRDD = Some(newRDD) - rdd.changeDependencies(newRDD) + rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and splits cpState = Checkpointed RDDCheckpointData.clearTaskCaches() logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id) @@ -79,7 +79,7 @@ extends Logging with Serializable { } // Get preferred location of a split after checkpointing - def getPreferredLocations(split: Split) = { + def getPreferredLocations(split: Split): Seq[String] = { RDDCheckpointData.synchronized { cpRDD.get.preferredLocations(split) } @@ -91,9 +91,10 @@ extends Logging with Serializable { } } - // Get iterator. This is called at the worker nodes. - def iterator(split: Split, context: TaskContext): Iterator[T] = { - rdd.firstParent[T].iterator(split, context) + def checkpointRDD: Option[RDD[T]] = { + RDDCheckpointData.synchronized { + cpRDD + } } } diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 4c95c989b5367..46fd8fe85edcd 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -319,7 +319,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround /** * Return whether this RDD has been checkpointed or not */ - def isCheckpointed(): Boolean = rdd.isCheckpointed() + def isCheckpointed: Boolean = rdd.isCheckpointed /** * Gets the name of the file to which this RDD was checkpointed diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 453d410ad446b..0f9ca0653198f 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -1,7 +1,7 @@ package spark.rdd import java.io.{ObjectOutputStream, IOException} -import spark.{OneToOneDependency, NarrowDependency, RDD, SparkContext, Split, TaskContext} +import spark._ private[spark] @@ -35,7 +35,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( val numSplitsInRdd2 = rdd2.splits.size - @transient var splits_ = { + override def getSplits: Array[Split] = { // create the cross product split val array = new Array[Split](rdd1.splits.size * rdd2.splits.size) for (s1 <- rdd1.splits; s2 <- rdd2.splits) { @@ -45,8 +45,6 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( array } - override def getSplits = splits_ - override def getPreferredLocations(split: Split) = { val currSplit = split.asInstanceOf[CartesianSplit] rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2) @@ -58,7 +56,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) } - var deps_ = List( + override def getDependencies: Seq[Dependency[_]] = List( new NarrowDependency(rdd1) { def getParents(id: Int): Seq[Int] = List(id / numSplitsInRdd2) }, @@ -67,11 +65,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( } ) - override def getDependencies = deps_ - override def clearDependencies() { - deps_ = Nil - splits_ = null rdd1 = null rdd2 = null } diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 6f00f6ac73006..96b593ba7ca6d 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -9,23 +9,26 @@ import org.apache.hadoop.fs.Path import java.io.{File, IOException, EOFException} import java.text.NumberFormat -private[spark] class CheckpointRDDSplit(idx: Int, val splitFile: String) extends Split { - override val index: Int = idx -} +private[spark] class CheckpointRDDSplit(val index: Int) extends Split {} /** * This RDD represents a RDD checkpoint file (similar to HadoopRDD). */ private[spark] -class CheckpointRDD[T: ClassManifest](sc: SparkContext, checkpointPath: String) +class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: String) extends RDD[T](sc, Nil) { - @transient val path = new Path(checkpointPath) - @transient val fs = path.getFileSystem(new Configuration()) + @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) @transient val splits_ : Array[Split] = { - val splitFiles = fs.listStatus(path).map(_.getPath.toString).filter(_.contains("part-")).sorted - splitFiles.zipWithIndex.map(x => new CheckpointRDDSplit(x._2, x._1)).toArray + val dirContents = fs.listStatus(new Path(checkpointPath)) + val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted + val numSplits = splitFiles.size + if (!splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || + !splitFiles(numSplits-1).endsWith(CheckpointRDD.splitIdToFile(numSplits-1))) { + throw new SparkException("Invalid checkpoint directory: " + checkpointPath) + } + Array.tabulate(numSplits)(i => new CheckpointRDDSplit(i)) } checkpointData = Some(new RDDCheckpointData[T](this)) @@ -34,36 +37,34 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, checkpointPath: String) override def getSplits = splits_ override def getPreferredLocations(split: Split): Seq[String] = { - val status = fs.getFileStatus(path) + val status = fs.getFileStatus(new Path(checkpointPath)) val locations = fs.getFileBlockLocations(status, 0, status.getLen) - locations.firstOption.toList.flatMap(_.getHosts).filter(_ != "localhost") + locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost") } override def compute(split: Split, context: TaskContext): Iterator[T] = { - CheckpointRDD.readFromFile(split.asInstanceOf[CheckpointRDDSplit].splitFile, context) + val file = new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index)) + CheckpointRDD.readFromFile(file, context) } override def checkpoint() { - // Do nothing. Hadoop RDD should not be checkpointed. + // Do nothing. CheckpointRDD should not be checkpointed. } } private[spark] object CheckpointRDD extends Logging { - def splitIdToFileName(splitId: Int): String = { - val numfmt = NumberFormat.getInstance() - numfmt.setMinimumIntegerDigits(5) - numfmt.setGroupingUsed(false) - "part-" + numfmt.format(splitId) + def splitIdToFile(splitId: Int): String = { + "part-%05d".format(splitId) } - def writeToFile[T](path: String, blockSize: Int = -1)(context: TaskContext, iterator: Iterator[T]) { + def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) { val outputDir = new Path(path) val fs = outputDir.getFileSystem(new Configuration()) - val finalOutputName = splitIdToFileName(context.splitId) + val finalOutputName = splitIdToFile(ctx.splitId) val finalOutputPath = new Path(outputDir, finalOutputName) - val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + context.attemptId) + val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId) if (fs.exists(tempOutputPath)) { throw new IOException("Checkpoint failed: temporary path " + @@ -83,22 +84,22 @@ private[spark] object CheckpointRDD extends Logging { serializeStream.close() if (!fs.rename(tempOutputPath, finalOutputPath)) { - if (!fs.delete(finalOutputPath, true)) { - throw new IOException("Checkpoint failed: failed to delete earlier output of task " - + context.attemptId) - } - if (!fs.rename(tempOutputPath, finalOutputPath)) { + if (!fs.exists(finalOutputPath)) { + fs.delete(tempOutputPath, false) throw new IOException("Checkpoint failed: failed to save output of task: " - + context.attemptId) + + ctx.attemptId + " and final output path does not exist") + } else { + // Some other copy of this task must've finished before us and renamed it + logInfo("Final output path " + finalOutputPath + " already exists; not overwriting it") + fs.delete(tempOutputPath, false) } } } - def readFromFile[T](path: String, context: TaskContext): Iterator[T] = { - val inputPath = new Path(path) - val fs = inputPath.getFileSystem(new Configuration()) + def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { + val fs = path.getFileSystem(new Configuration()) val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - val fileInputStream = fs.open(inputPath, bufferSize) + val fileInputStream = fs.open(path, bufferSize) val serializer = SparkEnv.get.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 167755bbba267..4c57434b65f9a 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -27,11 +27,11 @@ private[spark] case class CoalescedRDDSplit( * or to avoid having a large number of small tasks when processing a directory with many files. */ class CoalescedRDD[T: ClassManifest]( - var prev: RDD[T], + @transient var prev: RDD[T], maxPartitions: Int) - extends RDD[T](prev.context, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs + extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies - @transient var splits_ : Array[Split] = { + override def getSplits: Array[Split] = { val prevSplits = prev.splits if (prevSplits.length < maxPartitions) { prevSplits.map(_.index).map{idx => new CoalescedRDDSplit(idx, prev, Array(idx)) } @@ -44,26 +44,20 @@ class CoalescedRDD[T: ClassManifest]( } } - override def getSplits = splits_ - override def compute(split: Split, context: TaskContext): Iterator[T] = { split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { parentSplit => firstParent[T].iterator(parentSplit, context) } } - var deps_ : List[Dependency[_]] = List( + override def getDependencies: Seq[Dependency[_]] = List( new NarrowDependency(prev) { def getParents(id: Int): Seq[Int] = splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices } ) - override def getDependencies() = deps_ - override def clearDependencies() { - deps_ = Nil - splits_ = null prev = null } } diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index c6ceb272cdc7b..5466c9c657fcb 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -3,13 +3,11 @@ package spark.rdd import spark.{RDD, Split, TaskContext} private[spark] -class MappedRDD[U: ClassManifest, T: ClassManifest]( - prev: RDD[T], - f: T => U) +class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) extends RDD[U](prev) { override def getSplits = firstParent[T].splits override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context).map(f) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 97dd37950e018..b8482338c6f8f 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -7,23 +7,18 @@ import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. + * + * TODO: This currently doesn't give partition IDs properly! */ class PartitionPruningRDD[T: ClassManifest]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { - @transient - var partitions_ : Array[Split] = dependencies_.head.asInstanceOf[PruneDependency[T]].partitions - override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context) - override protected def getSplits = partitions_ + override protected def getSplits = + getDependencies.head.asInstanceOf[PruneDependency[T]].partitions override val partitioner = firstParent[T].partitioner - - override def clearDependencies() { - super.clearDependencies() - partitions_ = null - } } diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 28ff19876dacb..d396478673615 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -22,16 +22,10 @@ class ShuffledRDD[K, V]( override val partitioner = Some(part) - @transient var splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) - - override def getSplits = splits_ + override def getSplits = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index) } - - override def clearDependencies() { - splits_ = null - } } diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 82f0a44ecd900..26a2d511f2670 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -26,9 +26,9 @@ private[spark] class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], splitIn class UnionRDD[T: ClassManifest]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) - extends RDD[T](sc, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs + extends RDD[T](sc, Nil) { // Nil since we implement getDependencies - @transient var splits_ : Array[Split] = { + override def getSplits: Array[Split] = { val array = new Array[Split](rdds.map(_.splits.size).sum) var pos = 0 for (rdd <- rdds; split <- rdd.splits) { @@ -38,20 +38,16 @@ class UnionRDD[T: ClassManifest]( array } - override def getSplits = splits_ - - @transient var deps_ = { + override def getDependencies: Seq[Dependency[_]] = { val deps = new ArrayBuffer[Dependency[_]] var pos = 0 for (rdd <- rdds) { deps += new RangeDependency(rdd, 0, pos, rdd.splits.size) pos += rdd.splits.size } - deps.toList + deps } - override def getDependencies = deps_ - override def compute(s: Split, context: TaskContext): Iterator[T] = s.asInstanceOf[UnionSplit[T]].iterator(context) @@ -59,8 +55,6 @@ class UnionRDD[T: ClassManifest]( s.asInstanceOf[UnionSplit[T]].preferredLocations() override def clearDependencies() { - deps_ = null - splits_ = null rdds = null } } diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index d950b06c85042..e5df6d8c7239b 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -32,9 +32,7 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) with Serializable { - // TODO: FIX THIS. - - @transient var splits_ : Array[Split] = { + override def getSplits: Array[Split] = { if (rdd1.splits.size != rdd2.splits.size) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } @@ -45,8 +43,6 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( array } - override def getSplits = splits_ - override def compute(s: Split, context: TaskContext): Iterator[(T, U)] = { val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context)) @@ -58,7 +54,6 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( } override def clearDependencies() { - splits_ = null rdd1 = null rdd2 = null } diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index 6cf93a9b17055..eaff7ae581ec5 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -26,8 +26,8 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging if (delaySeconds > 0) { logDebug( - "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds and " - + "period of " + periodSeconds + " secs") + "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds " + + "and period of " + periodSeconds + " secs") timer.schedule(task, periodSeconds * 1000, periodSeconds * 1000) } diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 33c317720c651..0b74607fb85a6 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -99,7 +99,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits. // Note that this test is very specific to the current implementation of CartesianRDD. val ones = sc.makeRDD(1 to 100, 10).map(x => x) - ones.checkpoint // checkpoint that MappedRDD + ones.checkpoint() // checkpoint that MappedRDD val cartesian = new CartesianRDD(sc, ones, ones) val splitBeforeCheckpoint = serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit]) @@ -125,7 +125,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits. // Note that this test is very specific to the current implementation of CoalescedRDDSplits val ones = sc.makeRDD(1 to 100, 10).map(x => x) - ones.checkpoint // checkpoint that MappedRDD + ones.checkpoint() // checkpoint that MappedRDD val coalesced = new CoalescedRDD(ones, 2) val splitBeforeCheckpoint = serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit]) @@ -160,7 +160,6 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { // so only the RDD will reduce in serialized size, not the splits. testParentCheckpointing( rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false) - } /** @@ -176,7 +175,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { testRDDSplitSize: Boolean = false ) { // Generate the final RDD using given RDD operation - val baseRDD = generateLongLineageRDD + val baseRDD = generateLongLineageRDD() val operatedRDD = op(baseRDD) val parentRDD = operatedRDD.dependencies.headOption.orNull val rddType = operatedRDD.getClass.getSimpleName @@ -245,12 +244,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { testRDDSplitSize: Boolean ) { // Generate the final RDD using given RDD operation - val baseRDD = generateLongLineageRDD + val baseRDD = generateLongLineageRDD() val operatedRDD = op(baseRDD) val parentRDD = operatedRDD.dependencies.head.rdd val rddType = operatedRDD.getClass.getSimpleName val parentRDDType = parentRDD.getClass.getSimpleName + // Get the splits and dependencies of the parent in case they're lazily computed + parentRDD.dependencies + parentRDD.splits + // Find serialized sizes before and after the checkpoint val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) parentRDD.checkpoint() // checkpoint the parent RDD, not the generated one @@ -267,7 +270,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { if (testRDDSize) { assert( rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint, - "Size of " + rddType + " did not reduce after parent checkpointing parent " + parentRDDType + + "Size of " + rddType + " did not reduce after checkpointing parent " + parentRDDType + "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]" ) } @@ -318,10 +321,12 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } /** - * Get serialized sizes of the RDD and its splits + * Get serialized sizes of the RDD and its splits, in order to test whether the size shrinks + * upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint. */ def getSerializedSizes(rdd: RDD[_]): (Int, Int) = { - (Utils.serialize(rdd).size, Utils.serialize(rdd.splits).size) + (Utils.serialize(rdd).length - Utils.serialize(rdd.checkpointData).length, + Utils.serialize(rdd.splits).length) } /** From a34096a76de9d07518ce33111ad43b88049c1ac2 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 28 Jan 2013 22:40:16 -0800 Subject: [PATCH 103/696] Add easymock to POMs --- core/pom.xml | 5 +++++ pom.xml | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 862d3ec37ac8f..a2b9b726a69d3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -98,6 +98,11 @@ scalacheck_${scala.version} test + + org.easymock + easymock + test + com.novocode junit-interface diff --git a/pom.xml b/pom.xml index 3ea989a082b2a..4a4ff560e77c8 100644 --- a/pom.xml +++ b/pom.xml @@ -273,6 +273,12 @@ 1.8 test + + org.easymock + easymock + 3.1 + test + org.scalacheck scalacheck_${scala.version} From 16a0789e10d2ac714e7c623b026c4a58ca9678d6 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 29 Jan 2013 17:09:53 -0800 Subject: [PATCH 104/696] Remember ConnectionManagerId used to initiate SendingConnections. This prevents ConnectionManager from getting confused if a machine has multiple host names and the one getHostName() finds happens not to be the one that was passed from, e.g., the BlockManagerMaster. --- .../src/main/scala/spark/network/Connection.scala | 15 +++++++++++---- .../scala/spark/network/ConnectionManager.scala | 3 ++- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index c193bf7c8d37e..cd5b7d57f32f5 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -12,7 +12,14 @@ import java.net._ private[spark] -abstract class Connection(val channel: SocketChannel, val selector: Selector) extends Logging { +abstract class Connection(val channel: SocketChannel, val selector: Selector, + val remoteConnectionManagerId: ConnectionManagerId) extends Logging { + def this(channel_ : SocketChannel, selector_ : Selector) = { + this(channel_, selector_, + ConnectionManagerId.fromSocketAddress( + channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] + )) + } channel.configureBlocking(false) channel.socket.setTcpNoDelay(true) @@ -25,7 +32,6 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex var onKeyInterestChangeCallback: (Connection, Int) => Unit = null val remoteAddress = getRemoteAddress() - val remoteConnectionManagerId = ConnectionManagerId.fromSocketAddress(remoteAddress) def key() = channel.keyFor(selector) @@ -103,8 +109,9 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex } -private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector) -extends Connection(SocketChannel.open, selector_) { +private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, + remoteId_ : ConnectionManagerId) +extends Connection(SocketChannel.open, selector_, remoteId_) { class Outbox(fair: Int = 0) { val messages = new Queue[Message]() diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 2ecd14f536747..c7f226044d1e9 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -299,7 +299,8 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private def sendMessage(connectionManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, connectionManagerId.port) - val newConnection = connectionRequests.getOrElseUpdate(connectionManagerId, new SendingConnection(inetSocketAddress, selector)) + val newConnection = connectionRequests.getOrElseUpdate(connectionManagerId, + new SendingConnection(inetSocketAddress, selector, connectionManagerId)) newConnection } val lookupKey = ConnectionManagerId.fromSocketAddress(connectionManagerId.toSocketAddress) From 0f81025ecadbfd21edb64602658ae8ba26e5bf66 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 29 Jan 2013 18:54:58 -0800 Subject: [PATCH 105/696] Add easymock to SBT configuration. --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 03b8094f7dce5..af8b5ba01745b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -92,7 +92,8 @@ object SparkBuild extends Build { "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", "org.scalatest" %% "scalatest" % "1.8" % "test", "org.scalacheck" %% "scalacheck" % "1.9" % "test", - "com.novocode" % "junit-interface" % "0.8" % "test" + "com.novocode" % "junit-interface" % "0.8" % "test", + "org.easymock" % "easymock" % "3.1" % "test" ), parallelExecution := false, /* Workaround for issue #206 (fixed after SBT 0.11.0) */ From a3d14c0404d6b28433784f84086a29ecc0045a12 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 28 Jan 2013 22:41:08 -0800 Subject: [PATCH 106/696] Refactoring to DAGScheduler to aid testing --- core/src/main/scala/spark/SparkContext.scala | 1 + .../scala/spark/scheduler/DAGScheduler.scala | 29 +++++++++++-------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index dc9b8688b3c8a..6ae04f4a44c01 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -187,6 +187,7 @@ class SparkContext( taskScheduler.start() private var dagScheduler = new DAGScheduler(taskScheduler) + dagScheduler.start() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b130be6a3863a..965596116256e 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -23,7 +23,14 @@ import util.{MetadataCleaner, TimeStampedHashMap} * and to report fetch failures (the submitTasks method, and code to add CompletionEvents). */ private[spark] -class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with Logging { +class DAGScheduler(taskSched: TaskScheduler, + mapOutputTracker: MapOutputTracker, + blockManagerMaster: BlockManagerMaster, + env: SparkEnv) + extends TaskSchedulerListener with Logging { + def this(taskSched: TaskScheduler) { + this(taskSched, SparkEnv.get.mapOutputTracker, SparkEnv.get.blockManager.master, SparkEnv.get) + } taskSched.setListener(this) // Called by TaskScheduler to report task completions or failures. @@ -66,10 +73,6 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with var cacheLocs = new HashMap[Int, Array[List[String]]] - val env = SparkEnv.get - val mapOutputTracker = env.mapOutputTracker - val blockManagerMaster = env.blockManager.master - // For tracking failed nodes, we use the MapOutputTracker's generation number, which is // sent with every task. When we detect a node failing, we note the current generation number // and failed executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask @@ -90,12 +93,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) // Start a thread to run the DAGScheduler event loop - new Thread("DAGScheduler") { - setDaemon(true) - override def run() { - DAGScheduler.this.run() - } - }.start() + def start() { + new Thread("DAGScheduler") { + setDaemon(true) + override def run() { + DAGScheduler.this.run() + } + }.start() + } def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { if (!cacheLocs.contains(rdd.id)) { @@ -546,7 +551,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with if (!failedGeneration.contains(execId) || failedGeneration(execId) < currentGeneration) { failedGeneration(execId) = currentGeneration logInfo("Executor lost: %s (generation %d)".format(execId, currentGeneration)) - env.blockManager.master.removeExecutor(execId) + blockManagerMaster.removeExecutor(execId) // TODO: This will be really slow if we keep accumulating shuffle map stages for ((shuffleId, stage) <- shuffleToMapStage) { stage.removeOutputsOnExecutor(execId) From 9eac7d01f0880d1d3d51e922ef2566c4ee92989f Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 28 Jan 2013 22:42:35 -0800 Subject: [PATCH 107/696] Add DAGScheduler tests. --- .../spark/scheduler/DAGSchedulerSuite.scala | 540 ++++++++++++++++++ 1 file changed, 540 insertions(+) create mode 100644 core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala new file mode 100644 index 0000000000000..53f5214d7ab51 --- /dev/null +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -0,0 +1,540 @@ +package spark.scheduler + +import scala.collection.mutable.{Map, HashMap} + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.AsyncAssertions +import org.scalatest.concurrent.TimeLimitedTests +import org.scalatest.mock.EasyMockSugar +import org.scalatest.time.{Span, Seconds} + +import org.easymock.EasyMock._ +import org.easymock.EasyMock +import org.easymock.{IAnswer, IArgumentMatcher} + +import akka.actor.ActorSystem + +import spark.storage.BlockManager +import spark.storage.BlockManagerId +import spark.storage.BlockManagerMaster +import spark.{Dependency, ShuffleDependency, OneToOneDependency} +import spark.FetchFailedException +import spark.MapOutputTracker +import spark.RDD +import spark.SparkContext +import spark.SparkException +import spark.Split +import spark.TaskContext +import spark.TaskEndReason + +import spark.{FetchFailed, Success} + +class DAGSchedulerSuite extends FunSuite + with BeforeAndAfter with EasyMockSugar with TimeLimitedTests + with AsyncAssertions with spark.Logging { + + // If we crash the DAGScheduler thread, our test will probably hang. + override val timeLimit = Span(5, Seconds) + + val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") + var scheduler: DAGScheduler = null + var w: Waiter = null + val taskScheduler = mock[TaskScheduler] + val blockManagerMaster = mock[BlockManagerMaster] + var mapOutputTracker: MapOutputTracker = null + var schedulerThread: Thread = null + var schedulerException: Throwable = null + val taskSetMatchers = new HashMap[MyRDD, IArgumentMatcher] + val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] + + implicit val mocks = MockObjects(taskScheduler, blockManagerMaster) + + def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) + + def resetExpecting(f: => Unit) { + reset(taskScheduler) + reset(blockManagerMaster) + expecting(f) + } + + before { + taskSetMatchers.clear() + cacheLocations.clear() + val actorSystem = ActorSystem("test") + mapOutputTracker = new MapOutputTracker(actorSystem, true) + resetExpecting { + taskScheduler.setListener(anyObject()) + } + whenExecuting { + scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) + } + w = new Waiter + schedulerException = null + schedulerThread = new Thread("DAGScheduler under test") { + override def run() { + try { + scheduler.run() + } catch { + case t: Throwable => + logError("Got exception in DAGScheduler: ", t) + schedulerException = t + } finally { + w.dismiss() + } + } + } + schedulerThread.start + logInfo("finished before") + } + + after { + logInfo("started after") + resetExpecting { + taskScheduler.stop() + } + whenExecuting { + scheduler.stop + schedulerThread.join + } + w.await() + if (schedulerException != null) { + throw new Exception("Exception caught from scheduler thread", schedulerException) + } + } + + // Type of RDD we use for testing. Note that we should never call the real RDD compute methods. + // This is a pair RDD type so it can always be used in ShuffleDependencies. + type MyRDD = RDD[(Int, Int)] + + def makeRdd( + numSplits: Int, + dependencies: List[Dependency[_]], + locations: Seq[Seq[String]] = Nil + ): MyRDD = { + val maxSplit = numSplits - 1 + return new MyRDD(sc, dependencies) { + override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getSplits() = (0 to maxSplit).map(i => new Split { + override def index = i + }).toArray + override def getPreferredLocations(split: Split): Seq[String] = + if (locations.isDefinedAt(split.index)) + locations(split.index) + else + Nil + override def toString: String = "DAGSchedulerSuiteRDD " + id + } + } + + def taskSetForRdd(rdd: MyRDD): TaskSet = { + val matcher = taskSetMatchers.getOrElseUpdate(rdd, + new IArgumentMatcher { + override def matches(actual: Any): Boolean = { + val taskSet = actual.asInstanceOf[TaskSet] + taskSet.tasks(0) match { + case rt: ResultTask[_, _] => rt.rdd.id == rdd.id + case smt: ShuffleMapTask => smt.rdd.id == rdd.id + case _ => false + } + } + override def appendTo(buf: StringBuffer) { + buf.append("taskSetForRdd(" + rdd + ")") + } + }) + EasyMock.reportMatcher(matcher) + return null + } + + def expectGetLocations(): Unit = { + EasyMock.expect(blockManagerMaster.getLocations(anyObject().asInstanceOf[Array[String]])). + andAnswer(new IAnswer[Seq[Seq[BlockManagerId]]] { + override def answer(): Seq[Seq[BlockManagerId]] = { + val blocks = getCurrentArguments()(0).asInstanceOf[Array[String]] + return blocks.map { name => + val pieces = name.split("_") + if (pieces(0) == "rdd") { + val key = pieces(1).toInt -> pieces(2).toInt + if (cacheLocations.contains(key)) { + cacheLocations(key) + } else { + Seq[BlockManagerId]() + } + } else { + Seq[BlockManagerId]() + } + }.toSeq + } + }).anyTimes() + } + + def expectStageAnd(rdd: MyRDD, results: Seq[(TaskEndReason, Any)], + preferredLocations: Option[Seq[Seq[String]]] = None)(afterSubmit: TaskSet => Unit) { + // TODO: Remember which submission + EasyMock.expect(taskScheduler.submitTasks(taskSetForRdd(rdd))).andAnswer(new IAnswer[Unit] { + override def answer(): Unit = { + val taskSet = getCurrentArguments()(0).asInstanceOf[TaskSet] + for (task <- taskSet.tasks) { + task.generation = mapOutputTracker.getGeneration + } + afterSubmit(taskSet) + preferredLocations match { + case None => + for (taskLocs <- taskSet.tasks.map(_.preferredLocations)) { + w { assert(taskLocs.size === 0) } + } + case Some(locations) => + w { assert(locations.size === taskSet.tasks.size) } + for ((expectLocs, taskLocs) <- + taskSet.tasks.map(_.preferredLocations).zip(locations)) { + w { assert(expectLocs === taskLocs) } + } + } + w { assert(taskSet.tasks.size >= results.size)} + for ((result, i) <- results.zipWithIndex) { + if (i < taskSet.tasks.size) { + scheduler.taskEnded(taskSet.tasks(i), result._1, result._2, Map[Long, Any]()) + } + } + } + }) + } + + def expectStage(rdd: MyRDD, results: Seq[(TaskEndReason, Any)], + preferredLocations: Option[Seq[Seq[String]]] = None) { + expectStageAnd(rdd, results, preferredLocations) { _ => } + } + + def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): Array[Int] = { + return scheduler.runJob[(Int, Int), Int]( + rdd, + (context: TaskContext, it: Iterator[(Int, Int)]) => it.next._1.asInstanceOf[Int], + (0 to (rdd.splits.size - 1)), + "test-site", + allowLocal + ) + } + + def makeMapStatus(host: String, reduces: Int): MapStatus = + new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) + + test("zero split job") { + val rdd = makeRdd(0, Nil) + resetExpecting { + expectGetLocations() + // deliberately expect no stages to be submitted + } + whenExecuting { + assert(submitRdd(rdd) === Array[Int]()) + } + } + + test("run trivial job") { + val rdd = makeRdd(1, Nil) + resetExpecting { + expectGetLocations() + expectStage(rdd, List( (Success, 42) )) + } + whenExecuting { + assert(submitRdd(rdd) === Array(42)) + } + } + + test("local job") { + val rdd = new MyRDD(sc, Nil) { + override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] = + Array(42 -> 0).iterator + override def getSplits() = Array( new Split { override def index = 0 } ) + override def getPreferredLocations(split: Split) = Nil + override def toString = "DAGSchedulerSuite Local RDD" + } + resetExpecting { + expectGetLocations() + // deliberately expect no stages to be submitted + } + whenExecuting { + assert(submitRdd(rdd, true) === Array(42)) + } + } + + test("run trivial job w/ dependency") { + val baseRdd = makeRdd(1, Nil) + val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + resetExpecting { + expectGetLocations() + expectStage(finalRdd, List( (Success, 42) )) + } + whenExecuting { + assert(submitRdd(finalRdd) === Array(42)) + } + } + + test("location preferences w/ dependency") { + val baseRdd = makeRdd(1, Nil) + val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + resetExpecting { + expectGetLocations() + cacheLocations(baseRdd.id -> 0) = + Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) + expectStage(finalRdd, List( (Success, 42) ), + Some(List(Seq("hostA", "hostB")))) + } + whenExecuting { + assert(submitRdd(finalRdd) === Array(42)) + } + } + + test("trivial job failure") { + val rdd = makeRdd(1, Nil) + resetExpecting { + expectGetLocations() + expectStageAnd(rdd, List()) { taskSet => scheduler.taskSetFailed(taskSet, "test failure") } + } + whenExecuting(taskScheduler, blockManagerMaster) { + intercept[SparkException] { submitRdd(rdd) } + } + } + + test("run trivial shuffle") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = makeRdd(1, List(shuffleDep)) + + resetExpecting { + expectGetLocations() + expectStage(shuffleMapRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)) + )) + expectStageAnd(reduceRdd, List( (Success, 42) )) { _ => + w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) } + } + } + whenExecuting { + assert(submitRdd(reduceRdd) === Array(42)) + } + } + + test("run trivial shuffle with fetch failure") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = makeRdd(2, List(shuffleDep)) + + resetExpecting { + expectGetLocations() + expectStage(shuffleMapRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)) + )) + blockManagerMaster.removeExecutor("exec-hostA") + expectStage(reduceRdd, List( + (Success, 42), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null) + )) + // partial recompute + expectStage(shuffleMapRdd, List( (Success, makeMapStatus("hostA", 1)) )) + expectStageAnd(reduceRdd, List( (Success, 43) )) { _ => + w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostA"), + makeBlockManagerId("hostB"))) } + } + } + whenExecuting { + assert(submitRdd(reduceRdd) === Array(42, 43)) + } + } + + test("ignore late map task completions") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = makeRdd(2, List(shuffleDep)) + + resetExpecting { + expectGetLocations() + expectStageAnd(shuffleMapRdd, List( + (Success, makeMapStatus("hostA", 1)) + )) { taskSet => + val newGeneration = mapOutputTracker.getGeneration + 1 + scheduler.executorLost("exec-hostA") + val noAccum = Map[Long, Any]() + // We rely on the event queue being ordered and increasing the generation number by 1 + // should be ignored for being too old + scheduler.taskEnded(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum) + // should work because it's a non-failed host + scheduler.taskEnded(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum) + // should be ignored for being too old + scheduler.taskEnded(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum) + // should be ignored (not end the stage) because it's too old + scheduler.taskEnded(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum) + taskSet.tasks(1).generation = newGeneration + scheduler.taskEnded(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum) + } + blockManagerMaster.removeExecutor("exec-hostA") + expectStageAnd(reduceRdd, List( + (Success, 42), (Success, 43) + )) { _ => + w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) } + } + } + whenExecuting { + assert(submitRdd(reduceRdd) === Array(42, 43)) + } + } + + test("run trivial shuffle with out-of-band failure") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = makeRdd(1, List(shuffleDep)) + resetExpecting { + expectGetLocations() + blockManagerMaster.removeExecutor("exec-hostA") + expectStageAnd(shuffleMapRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)) + )) { _ => scheduler.executorLost("exec-hostA") } + expectStage(shuffleMapRdd, List( + (Success, makeMapStatus("hostC", 1)) + )) + expectStageAnd(reduceRdd, List( (Success, 42) )) { _ => + w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostC"), + makeBlockManagerId("hostB"))) } + } + } + whenExecuting { + assert(submitRdd(reduceRdd) === Array(42)) + } + } + + test("recursive shuffle failures") { + val shuffleOneRdd = makeRdd(2, Nil) + val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) + val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) + val finalRdd = makeRdd(1, List(shuffleDepTwo)) + + resetExpecting { + expectGetLocations() + expectStage(shuffleOneRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)) + )) + expectStage(shuffleTwoRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostC", 1)) + )) + blockManagerMaster.removeExecutor("exec-hostA") + expectStage(finalRdd, List( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) + )) + // triggers a partial recompute of the first stage, then the second + expectStage(shuffleOneRdd, List( + (Success, makeMapStatus("hostA", 1)) + )) + expectStage(shuffleTwoRdd, List( + (Success, makeMapStatus("hostA", 1)) + )) + expectStage(finalRdd, List( + (Success, 42) + )) + } + whenExecuting { + assert(submitRdd(finalRdd) === Array(42)) + } + } + + test("cached post-shuffle") { + val shuffleOneRdd = makeRdd(2, Nil) + val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) + val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) + val finalRdd = makeRdd(1, List(shuffleDepTwo)) + + resetExpecting { + expectGetLocations() + expectStage(shuffleOneRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)) + )) + expectStageAnd(shuffleTwoRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostC", 1)) + )){ _ => + cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) + cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) + } + blockManagerMaster.removeExecutor("exec-hostA") + expectStage(finalRdd, List( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) + )) + // since we have a cached copy of the missing split of shuffleTwoRdd, we shouldn't + // immediately try to rerun shuffleOneRdd: + expectStage(shuffleTwoRdd, List( + (Success, makeMapStatus("hostD", 1)) + ), Some(Seq(List("hostD")))) + expectStage(finalRdd, List( + (Success, 42) + )) + } + whenExecuting { + assert(submitRdd(finalRdd) === Array(42)) + } + } + + test("cached post-shuffle but fails") { + val shuffleOneRdd = makeRdd(2, Nil) + val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) + val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) + val finalRdd = makeRdd(1, List(shuffleDepTwo)) + + resetExpecting { + expectGetLocations() + expectStage(shuffleOneRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)) + )) + expectStageAnd(shuffleTwoRdd, List( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostC", 1)) + )){ _ => + cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) + cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) + } + blockManagerMaster.removeExecutor("exec-hostA") + expectStage(finalRdd, List( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) + )) + // since we have a cached copy of the missing split of shuffleTwoRdd, we shouldn't + // immediately try to rerun shuffleOneRdd: + expectStageAnd(shuffleTwoRdd, List( + (FetchFailed(null, shuffleDepOne.shuffleId, 0, 0), null) + ), Some(Seq(List("hostD")))) { _ => + w { + intercept[FetchFailedException]{ + mapOutputTracker.getServerStatuses(shuffleDepOne.shuffleId, 0) + } + } + cacheLocations.remove(shuffleTwoRdd.id -> 0) + } + // after that fetch failure, we should refetch the cache locations and try to recompute + // the whole chain. Note that we will ignore that a fetch failure previously occured on + // this host. + expectStage(shuffleOneRdd, List( (Success, makeMapStatus("hostA", 1)) )) + expectStage(shuffleTwoRdd, List( (Success, makeMapStatus("hostA", 1)) )) + expectStage(finalRdd, List( (Success, 42) )) + } + whenExecuting { + assert(submitRdd(finalRdd) === Array(42)) + } + } +} + From 4bf3d7ea1252454ca584a3dabf26bdeab4069409 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 29 Jan 2013 19:05:45 -0800 Subject: [PATCH 108/696] Clear spark.master.port to cleanup for other tests --- core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 53f5214d7ab51..6c577c2685597 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -102,6 +102,7 @@ class DAGSchedulerSuite extends FunSuite if (schedulerException != null) { throw new Exception("Exception caught from scheduler thread", schedulerException) } + System.clearProperty("spark.master.port") } // Type of RDD we use for testing. Note that we should never call the real RDD compute methods. From 178b89204c9dbee36886e757ddaafbd079672f4a Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 30 Jan 2013 09:19:55 -0800 Subject: [PATCH 109/696] Refactor DAGScheduler more to allow testing without a separate thread. --- .../scala/spark/scheduler/DAGScheduler.scala | 176 +++++++++++------- 1 file changed, 111 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 965596116256e..6892509ed165b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -23,11 +23,13 @@ import util.{MetadataCleaner, TimeStampedHashMap} * and to report fetch failures (the submitTasks method, and code to add CompletionEvents). */ private[spark] -class DAGScheduler(taskSched: TaskScheduler, - mapOutputTracker: MapOutputTracker, - blockManagerMaster: BlockManagerMaster, - env: SparkEnv) - extends TaskSchedulerListener with Logging { +class DAGScheduler( + taskSched: TaskScheduler, + mapOutputTracker: MapOutputTracker, + blockManagerMaster: BlockManagerMaster, + env: SparkEnv) + extends TaskSchedulerListener with Logging { + def this(taskSched: TaskScheduler) { this(taskSched, SparkEnv.get.mapOutputTracker, SparkEnv.get.blockManager.master, SparkEnv.get) } @@ -203,6 +205,27 @@ class DAGScheduler(taskSched: TaskScheduler, missing.toList } + /** Returns (and does not) submit a JobSubmitted event suitable to run a given job, and + * a JobWaiter whose getResult() method will return the result of the job when it is complete. + * + * The job is assumed to have at least one partition; zero partition jobs should be handled + * without a JobSubmitted event. + */ + private[scheduler] def prepareJob[T, U: ClassManifest]( + finalRdd: RDD[T], + func: (TaskContext, Iterator[T]) => U, + partitions: Seq[Int], + callSite: String, + allowLocal: Boolean) + : (JobSubmitted, JobWaiter) = + { + assert(partitions.size > 0) + val waiter = new JobWaiter(partitions.size) + val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter) + return (toSubmit, waiter) + } + def runJob[T, U: ClassManifest]( finalRdd: RDD[T], func: (TaskContext, Iterator[T]) => U, @@ -214,9 +237,8 @@ class DAGScheduler(taskSched: TaskScheduler, if (partitions.size == 0) { return new Array[U](0) } - val waiter = new JobWaiter(partitions.size) - val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - eventQueue.put(JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter)) + val (toSubmit, waiter) = prepareJob(finalRdd, func, partitions, callSite, allowLocal) + eventQueue.put(toSubmit) waiter.getResult() match { case JobSucceeded(results: Seq[_]) => return results.asInstanceOf[Seq[U]].toArray @@ -241,6 +263,81 @@ class DAGScheduler(taskSched: TaskScheduler, return listener.getResult() // Will throw an exception if the job fails } + /** Process one event retrieved from the event queue. + * Returns true if we should stop the event loop. + */ + private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { + event match { + case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener) => + val runId = nextRunId.getAndIncrement() + val finalStage = newStage(finalRDD, None, runId) + val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener) + clearCacheLocs() + logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + + " output partitions (allowLocal=" + allowLocal + ")") + logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") + logInfo("Parents of final stage: " + finalStage.parents) + logInfo("Missing parents: " + getMissingParentStages(finalStage)) + if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { + // Compute very short actions like first() or take() with no parent stages locally. + runLocally(job) + } else { + activeJobs += job + resultStageToJob(finalStage) = job + submitStage(finalStage) + } + + case ExecutorLost(execId) => + handleExecutorLost(execId) + + case completion: CompletionEvent => + handleTaskCompletion(completion) + + case TaskSetFailed(taskSet, reason) => + abortStage(idToStage(taskSet.stageId), reason) + + case StopDAGScheduler => + // Cancel any active jobs + for (job <- activeJobs) { + val error = new SparkException("Job cancelled because SparkContext was shut down") + job.listener.jobFailed(error) + } + return true + } + return false + } + + /** Resubmit any failed stages. Ordinarily called after a small amount of time has passed since + * the last fetch failure. + */ + private[scheduler] def resubmitFailedStages() { + logInfo("Resubmitting failed stages") + clearCacheLocs() + val failed2 = failed.toArray + failed.clear() + for (stage <- failed2.sortBy(_.priority)) { + submitStage(stage) + } + } + + /** Check for waiting or failed stages which are now eligible for resubmission. + * Ordinarily run on every iteration of the event loop. + */ + private[scheduler] def submitWaitingStages() { + // TODO: We might want to run this less often, when we are sure that something has become + // runnable that wasn't before. + logTrace("Checking for newly runnable parent stages") + logTrace("running: " + running) + logTrace("waiting: " + waiting) + logTrace("failed: " + failed) + val waiting2 = waiting.toArray + waiting.clear() + for (stage <- waiting2.sortBy(_.priority)) { + submitStage(stage) + } + } + + /** * The main event loop of the DAG scheduler, which waits for new-job / task-finished / failure * events and responds by launching tasks. This runs in a dedicated thread and receives events @@ -251,77 +348,26 @@ class DAGScheduler(taskSched: TaskScheduler, while (true) { val event = eventQueue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS) - val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability if (event != null) { logDebug("Got event of type " + event.getClass.getName) } - event match { - case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener) => - val runId = nextRunId.getAndIncrement() - val finalStage = newStage(finalRDD, None, runId) - val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener) - clearCacheLocs() - logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + - " output partitions") - logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") - logInfo("Parents of final stage: " + finalStage.parents) - logInfo("Missing parents: " + getMissingParentStages(finalStage)) - if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { - // Compute very short actions like first() or take() with no parent stages locally. - runLocally(job) - } else { - activeJobs += job - resultStageToJob(finalStage) = job - submitStage(finalStage) - } - - case ExecutorLost(execId) => - handleExecutorLost(execId) - - case completion: CompletionEvent => - handleTaskCompletion(completion) - - case TaskSetFailed(taskSet, reason) => - abortStage(idToStage(taskSet.stageId), reason) - - case StopDAGScheduler => - // Cancel any active jobs - for (job <- activeJobs) { - val error = new SparkException("Job cancelled because SparkContext was shut down") - job.listener.jobFailed(error) - } + if (event != null) { + if (processEvent(event)) { return - - case null => - // queue.poll() timed out, ignore it + } } + val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability // Periodically resubmit failed stages if some map output fetches have failed and we have // waited at least RESUBMIT_TIMEOUT. We wait for this short time because when a node fails, // tasks on many other nodes are bound to get a fetch failure, and they won't all get it at // the same time, so we want to make sure we've identified all the reduce tasks that depend // on the failed node. if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) { - logInfo("Resubmitting failed stages") - clearCacheLocs() - val failed2 = failed.toArray - failed.clear() - for (stage <- failed2.sortBy(_.priority)) { - submitStage(stage) - } + resubmitFailedStages } else { - // TODO: We might want to run this less often, when we are sure that something has become - // runnable that wasn't before. - logTrace("Checking for newly runnable parent stages") - logTrace("running: " + running) - logTrace("waiting: " + waiting) - logTrace("failed: " + failed) - val waiting2 = waiting.toArray - waiting.clear() - for (stage <- waiting2.sortBy(_.priority)) { - submitStage(stage) - } + submitWaitingStages } } } From 9c0bae75ade9e5b5a69077a5719adf4ee96e2c2e Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 30 Jan 2013 09:22:07 -0800 Subject: [PATCH 110/696] Change DAGSchedulerSuite to run DAGScheduler in the same Thread. --- .../spark/scheduler/DAGSchedulerSuite.scala | 568 ++++++++++-------- 1 file changed, 319 insertions(+), 249 deletions(-) diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 6c577c2685597..89173540d4009 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -4,12 +4,12 @@ import scala.collection.mutable.{Map, HashMap} import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.AsyncAssertions import org.scalatest.concurrent.TimeLimitedTests import org.scalatest.mock.EasyMockSugar import org.scalatest.time.{Span, Seconds} import org.easymock.EasyMock._ +import org.easymock.Capture import org.easymock.EasyMock import org.easymock.{IAnswer, IArgumentMatcher} @@ -30,33 +30,55 @@ import spark.TaskEndReason import spark.{FetchFailed, Success} -class DAGSchedulerSuite extends FunSuite - with BeforeAndAfter with EasyMockSugar with TimeLimitedTests - with AsyncAssertions with spark.Logging { +class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar with TimeLimitedTests { - // If we crash the DAGScheduler thread, our test will probably hang. + // impose a time limit on this test in case we don't let the job finish. override val timeLimit = Span(5, Seconds) val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") var scheduler: DAGScheduler = null - var w: Waiter = null val taskScheduler = mock[TaskScheduler] val blockManagerMaster = mock[BlockManagerMaster] var mapOutputTracker: MapOutputTracker = null var schedulerThread: Thread = null var schedulerException: Throwable = null + + /** Set of EasyMock argument matchers that match a TaskSet for a given RDD. + * We cache these so we do not create duplicate matchers for the same RDD. + * This allows us to easily setup a sequence of expectations for task sets for + * that RDD. + */ val taskSetMatchers = new HashMap[MyRDD, IArgumentMatcher] + + /** Set of cache locations to return from our mock BlockManagerMaster. + * Keys are (rdd ID, partition ID). Anything not present will return an empty + * list of cache locations silently. + */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] - implicit val mocks = MockObjects(taskScheduler, blockManagerMaster) + /** JobWaiter for the last JobSubmitted event we pushed. To keep tests (most of which + * will only submit one job) from needing to explicitly track it. + */ + var lastJobWaiter: JobWaiter = null - def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) + /** Tell EasyMockSugar what mock objects we want to be configured by expecting {...} + * and whenExecuting {...} */ + implicit val mocks = MockObjects(taskScheduler, blockManagerMaster) + /** Utility function to reset mocks and set expectations on them. EasyMock wants mock objects + * to be reset after each time their expectations are set, and we tend to check mock object + * calls over a single call to DAGScheduler. + * + * We also set a default expectation here that blockManagerMaster.getLocations can be called + * and will return values from cacheLocations. + */ def resetExpecting(f: => Unit) { reset(taskScheduler) reset(blockManagerMaster) - expecting(f) + expecting { + expectGetLocations() + f + } } before { @@ -70,45 +92,30 @@ class DAGSchedulerSuite extends FunSuite whenExecuting { scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) } - w = new Waiter - schedulerException = null - schedulerThread = new Thread("DAGScheduler under test") { - override def run() { - try { - scheduler.run() - } catch { - case t: Throwable => - logError("Got exception in DAGScheduler: ", t) - schedulerException = t - } finally { - w.dismiss() - } - } - } - schedulerThread.start - logInfo("finished before") } after { - logInfo("started after") + assert(scheduler.processEvent(StopDAGScheduler)) resetExpecting { taskScheduler.stop() } whenExecuting { - scheduler.stop - schedulerThread.join - } - w.await() - if (schedulerException != null) { - throw new Exception("Exception caught from scheduler thread", schedulerException) + scheduler.stop() } System.clearProperty("spark.master.port") } - // Type of RDD we use for testing. Note that we should never call the real RDD compute methods. - // This is a pair RDD type so it can always be used in ShuffleDependencies. + def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) + + /** Type of RDD we use for testing. Note that we should never call the real RDD compute methods. + * This is a pair RDD type so it can always be used in ShuffleDependencies. */ type MyRDD = RDD[(Int, Int)] + /** Create an RDD for passing to DAGScheduler. These RDDs will use the dependencies and + * preferredLocations (if any) that are passed to them. They are deliberately not executable + * so we can test that DAGScheduler does not try to execute RDDs locally. + */ def makeRdd( numSplits: Int, dependencies: List[Dependency[_]], @@ -130,6 +137,9 @@ class DAGSchedulerSuite extends FunSuite } } + /** EasyMock matcher method. For use as an argument matcher for a TaskSet whose first task + * is from a particular RDD. + */ def taskSetForRdd(rdd: MyRDD): TaskSet = { val matcher = taskSetMatchers.getOrElseUpdate(rdd, new IArgumentMatcher { @@ -149,6 +159,9 @@ class DAGSchedulerSuite extends FunSuite return null } + /** Setup an EasyMock expectation to repsond to blockManagerMaster.getLocations() called from + * cacheLocations. + */ def expectGetLocations(): Unit = { EasyMock.expect(blockManagerMaster.getLocations(anyObject().asInstanceOf[Array[String]])). andAnswer(new IAnswer[Seq[Seq[BlockManagerId]]] { @@ -171,51 +184,106 @@ class DAGSchedulerSuite extends FunSuite }).anyTimes() } - def expectStageAnd(rdd: MyRDD, results: Seq[(TaskEndReason, Any)], - preferredLocations: Option[Seq[Seq[String]]] = None)(afterSubmit: TaskSet => Unit) { - // TODO: Remember which submission - EasyMock.expect(taskScheduler.submitTasks(taskSetForRdd(rdd))).andAnswer(new IAnswer[Unit] { - override def answer(): Unit = { - val taskSet = getCurrentArguments()(0).asInstanceOf[TaskSet] - for (task <- taskSet.tasks) { - task.generation = mapOutputTracker.getGeneration - } - afterSubmit(taskSet) - preferredLocations match { - case None => - for (taskLocs <- taskSet.tasks.map(_.preferredLocations)) { - w { assert(taskLocs.size === 0) } - } - case Some(locations) => - w { assert(locations.size === taskSet.tasks.size) } - for ((expectLocs, taskLocs) <- - taskSet.tasks.map(_.preferredLocations).zip(locations)) { - w { assert(expectLocs === taskLocs) } - } - } - w { assert(taskSet.tasks.size >= results.size)} - for ((result, i) <- results.zipWithIndex) { - if (i < taskSet.tasks.size) { - scheduler.taskEnded(taskSet.tasks(i), result._1, result._2, Map[Long, Any]()) - } - } + /** Process the supplied event as if it were the top of the DAGScheduler event queue, expecting + * the scheduler not to exit. + * + * After processing the event, submit waiting stages as is done on most iterations of the + * DAGScheduler event loop. + */ + def runEvent(event: DAGSchedulerEvent) { + assert(!scheduler.processEvent(event)) + scheduler.submitWaitingStages() + } + + /** Expect a TaskSet for the specified RDD to be submitted to the TaskScheduler. Should be + * called from a resetExpecting { ... } block. + * + * Returns a easymock Capture that will contain the task set after the stage is submitted. + * Most tests should use interceptStage() instead of this directly. + */ + def expectStage(rdd: MyRDD): Capture[TaskSet] = { + val taskSetCapture = new Capture[TaskSet] + taskScheduler.submitTasks(and(capture(taskSetCapture), taskSetForRdd(rdd))) + return taskSetCapture + } + + /** Expect the supplied code snippet to submit a stage for the specified RDD. + * Return the resulting TaskSet. First marks all the tasks are belonging to the + * current MapOutputTracker generation. + */ + def interceptStage(rdd: MyRDD)(f: => Unit): TaskSet = { + var capture: Capture[TaskSet] = null + resetExpecting { + capture = expectStage(rdd) + } + whenExecuting { + f + } + val taskSet = capture.getValue + for (task <- taskSet.tasks) { + task.generation = mapOutputTracker.getGeneration + } + return taskSet + } + + /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ + def respondToTaskSet(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { + assert(taskSet.tasks.size >= results.size) + for ((result, i) <- results.zipWithIndex) { + if (i < taskSet.tasks.size) { + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any]())) } - }) + } } - def expectStage(rdd: MyRDD, results: Seq[(TaskEndReason, Any)], - preferredLocations: Option[Seq[Seq[String]]] = None) { - expectStageAnd(rdd, results, preferredLocations) { _ => } + /** Assert that the supplied TaskSet has exactly the given preferredLocations. */ + def expectTaskSetLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { + assert(locations.size === taskSet.tasks.size) + for ((expectLocs, taskLocs) <- + taskSet.tasks.map(_.preferredLocations).zip(locations)) { + assert(expectLocs === taskLocs) + } } - def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): Array[Int] = { - return scheduler.runJob[(Int, Int), Int]( + /** When we submit dummy Jobs, this is the compute function we supply. Except in a local test + * below, we do not expect this function to ever be executed; instead, we will return results + * directly through CompletionEvents. + */ + def jobComputeFunc(context: TaskContext, it: Iterator[(Int, Int)]): Int = + it.next._1.asInstanceOf[Int] + + + /** Start a job to compute the given RDD. Returns the JobWaiter that will + * collect the result of the job via callbacks from DAGScheduler. */ + def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): JobWaiter = { + val (toSubmit, waiter) = scheduler.prepareJob[(Int, Int), Int]( rdd, - (context: TaskContext, it: Iterator[(Int, Int)]) => it.next._1.asInstanceOf[Int], + jobComputeFunc, (0 to (rdd.splits.size - 1)), "test-site", allowLocal ) + lastJobWaiter = waiter + runEvent(toSubmit) + return waiter + } + + /** Assert that a job we started has failed. */ + def expectJobException(waiter: JobWaiter = lastJobWaiter) { + waiter.getResult match { + case JobSucceeded(_) => fail() + case JobFailed(_) => return + } + } + + /** Assert that a job we started has succeeded and has the given result. */ + def expectJobResult(expected: Array[Int], waiter: JobWaiter = lastJobWaiter) { + waiter.getResult match { + case JobSucceeded(answer) => + assert(expected === answer.asInstanceOf[Seq[Int]].toArray ) + case JobFailed(_) => + fail() + } } def makeMapStatus(host: String, reduces: Int): MapStatus = @@ -223,24 +291,14 @@ class DAGSchedulerSuite extends FunSuite test("zero split job") { val rdd = makeRdd(0, Nil) - resetExpecting { - expectGetLocations() - // deliberately expect no stages to be submitted - } - whenExecuting { - assert(submitRdd(rdd) === Array[Int]()) - } + assert(scheduler.runJob(rdd, jobComputeFunc, Seq(), "test-site", false) === Array[Int]()) } test("run trivial job") { val rdd = makeRdd(1, Nil) - resetExpecting { - expectGetLocations() - expectStage(rdd, List( (Success, 42) )) - } - whenExecuting { - assert(submitRdd(rdd) === Array(42)) - } + val taskSet = interceptStage(rdd) { submitRdd(rdd) } + respondToTaskSet(taskSet, List( (Success, 42) )) + expectJobResult(Array(42)) } test("local job") { @@ -251,51 +309,34 @@ class DAGSchedulerSuite extends FunSuite override def getPreferredLocations(split: Split) = Nil override def toString = "DAGSchedulerSuite Local RDD" } - resetExpecting { - expectGetLocations() - // deliberately expect no stages to be submitted - } - whenExecuting { - assert(submitRdd(rdd, true) === Array(42)) - } + submitRdd(rdd, true) + expectJobResult(Array(42)) } test("run trivial job w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) - resetExpecting { - expectGetLocations() - expectStage(finalRdd, List( (Success, 42) )) - } - whenExecuting { - assert(submitRdd(finalRdd) === Array(42)) - } + val taskSet = interceptStage(finalRdd) { submitRdd(finalRdd) } + respondToTaskSet(taskSet, List( (Success, 42) )) + expectJobResult(Array(42)) } - test("location preferences w/ dependency") { + test("cache location preferences w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) - resetExpecting { - expectGetLocations() - cacheLocations(baseRdd.id -> 0) = - Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) - expectStage(finalRdd, List( (Success, 42) ), - Some(List(Seq("hostA", "hostB")))) - } - whenExecuting { - assert(submitRdd(finalRdd) === Array(42)) - } + cacheLocations(baseRdd.id -> 0) = + Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) + val taskSet = interceptStage(finalRdd) { submitRdd(finalRdd) } + expectTaskSetLocations(taskSet, List(Seq("hostA", "hostB"))) + respondToTaskSet(taskSet, List( (Success, 42) )) + expectJobResult(Array(42)) } test("trivial job failure") { val rdd = makeRdd(1, Nil) - resetExpecting { - expectGetLocations() - expectStageAnd(rdd, List()) { taskSet => scheduler.taskSetFailed(taskSet, "test failure") } - } - whenExecuting(taskScheduler, blockManagerMaster) { - intercept[SparkException] { submitRdd(rdd) } - } + val taskSet = interceptStage(rdd) { submitRdd(rdd) } + runEvent(TaskSetFailed(taskSet, "test failure")) + expectJobException() } test("run trivial shuffle") { @@ -304,20 +345,17 @@ class DAGSchedulerSuite extends FunSuite val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) - resetExpecting { - expectGetLocations() - expectStage(shuffleMapRdd, List( + val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } + val secondStage = interceptStage(reduceRdd) { + respondToTaskSet(firstStage, List( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)) )) - expectStageAnd(reduceRdd, List( (Success, 42) )) { _ => - w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) } - } - } - whenExecuting { - assert(submitRdd(reduceRdd) === Array(42)) } + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + respondToTaskSet(secondStage, List( (Success, 42) )) + expectJobResult(Array(42)) } test("run trivial shuffle with fetch failure") { @@ -326,28 +364,32 @@ class DAGSchedulerSuite extends FunSuite val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) - resetExpecting { - expectGetLocations() - expectStage(shuffleMapRdd, List( + val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } + val secondStage = interceptStage(reduceRdd) { + respondToTaskSet(firstStage, List( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)) )) + } + resetExpecting { blockManagerMaster.removeExecutor("exec-hostA") - expectStage(reduceRdd, List( + } + whenExecuting { + respondToTaskSet(secondStage, List( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null) )) - // partial recompute - expectStage(shuffleMapRdd, List( (Success, makeMapStatus("hostA", 1)) )) - expectStageAnd(reduceRdd, List( (Success, 43) )) { _ => - w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostA"), - makeBlockManagerId("hostB"))) } - } } - whenExecuting { - assert(submitRdd(reduceRdd) === Array(42, 43)) + val thirdStage = interceptStage(shuffleMapRdd) { + scheduler.resubmitFailedStages() + } + val fourthStage = interceptStage(reduceRdd) { + respondToTaskSet(thirdStage, List( (Success, makeMapStatus("hostA", 1)) )) } + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + respondToTaskSet(fourthStage, List( (Success, 43) )) + expectJobResult(Array(42, 43)) } test("ignore late map task completions") { @@ -356,63 +398,64 @@ class DAGSchedulerSuite extends FunSuite val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) + val taskSet = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } + val oldGeneration = mapOutputTracker.getGeneration resetExpecting { - expectGetLocations() - expectStageAnd(shuffleMapRdd, List( - (Success, makeMapStatus("hostA", 1)) - )) { taskSet => - val newGeneration = mapOutputTracker.getGeneration + 1 - scheduler.executorLost("exec-hostA") - val noAccum = Map[Long, Any]() - // We rely on the event queue being ordered and increasing the generation number by 1 - // should be ignored for being too old - scheduler.taskEnded(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum) - // should work because it's a non-failed host - scheduler.taskEnded(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum) - // should be ignored for being too old - scheduler.taskEnded(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum) - // should be ignored (not end the stage) because it's too old - scheduler.taskEnded(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum) - taskSet.tasks(1).generation = newGeneration - scheduler.taskEnded(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum) - } blockManagerMaster.removeExecutor("exec-hostA") - expectStageAnd(reduceRdd, List( - (Success, 42), (Success, 43) - )) { _ => - w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) } - } } whenExecuting { - assert(submitRdd(reduceRdd) === Array(42, 43)) - } + runEvent(ExecutorLost("exec-hostA")) + } + val newGeneration = mapOutputTracker.getGeneration + assert(newGeneration > oldGeneration) + val noAccum = Map[Long, Any]() + // We rely on the event queue being ordered and increasing the generation number by 1 + // should be ignored for being too old + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + // should work because it's a non-failed host + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum)) + // should be ignored for being too old + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + taskSet.tasks(1).generation = newGeneration + val secondStage = interceptStage(reduceRdd) { + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum)) + } + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) + respondToTaskSet(secondStage, List( (Success, 42), (Success, 43) )) + expectJobResult(Array(42, 43)) } - test("run trivial shuffle with out-of-band failure") { + test("run trivial shuffle with out-of-band failure and retry") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) + + val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } resetExpecting { - expectGetLocations() blockManagerMaster.removeExecutor("exec-hostA") - expectStageAnd(shuffleMapRdd, List( + } + whenExecuting { + runEvent(ExecutorLost("exec-hostA")) + } + // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks + // rather than marking it is as failed and waiting. + val secondStage = interceptStage(shuffleMapRdd) { + respondToTaskSet(firstStage, List( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)) - )) { _ => scheduler.executorLost("exec-hostA") } - expectStage(shuffleMapRdd, List( - (Success, makeMapStatus("hostC", 1)) )) - expectStageAnd(reduceRdd, List( (Success, 42) )) { _ => - w { assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostC"), - makeBlockManagerId("hostB"))) } - } } - whenExecuting { - assert(submitRdd(reduceRdd) === Array(42)) + val thirdStage = interceptStage(reduceRdd) { + respondToTaskSet(secondStage, List( + (Success, makeMapStatus("hostC", 1)) + )) } + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + respondToTaskSet(thirdStage, List( (Success, 42) )) + expectJobResult(Array(42)) } test("recursive shuffle failures") { @@ -422,34 +465,42 @@ class DAGSchedulerSuite extends FunSuite val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) val finalRdd = makeRdd(1, List(shuffleDepTwo)) - resetExpecting { - expectGetLocations() - expectStage(shuffleOneRdd, List( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) + val firstStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } + val secondStage = interceptStage(shuffleTwoRdd) { + respondToTaskSet(firstStage, List( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)) )) - expectStage(shuffleTwoRdd, List( + } + val thirdStage = interceptStage(finalRdd) { + respondToTaskSet(secondStage, List( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostC", 1)) )) + } + resetExpecting { blockManagerMaster.removeExecutor("exec-hostA") - expectStage(finalRdd, List( + } + whenExecuting { + respondToTaskSet(thirdStage, List( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) )) - // triggers a partial recompute of the first stage, then the second - expectStage(shuffleOneRdd, List( - (Success, makeMapStatus("hostA", 1)) + } + val recomputeOne = interceptStage(shuffleOneRdd) { + scheduler.resubmitFailedStages + } + val recomputeTwo = interceptStage(shuffleTwoRdd) { + respondToTaskSet(recomputeOne, List( + (Success, makeMapStatus("hostA", 2)) )) - expectStage(shuffleTwoRdd, List( + } + val finalStage = interceptStage(finalRdd) { + respondToTaskSet(recomputeTwo, List( (Success, makeMapStatus("hostA", 1)) )) - expectStage(finalRdd, List( - (Success, 42) - )) - } - whenExecuting { - assert(submitRdd(finalRdd) === Array(42)) } + respondToTaskSet(finalStage, List( (Success, 42) )) + expectJobResult(Array(42)) } test("cached post-shuffle") { @@ -459,35 +510,41 @@ class DAGSchedulerSuite extends FunSuite val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) val finalRdd = makeRdd(1, List(shuffleDepTwo)) - resetExpecting { - expectGetLocations() - expectStage(shuffleOneRdd, List( + val firstShuffleStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } + cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) + cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) + val secondShuffleStage = interceptStage(shuffleTwoRdd) { + respondToTaskSet(firstShuffleStage, List( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)) + )) + } + val reduceStage = interceptStage(finalRdd) { + respondToTaskSet(secondShuffleStage, List( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)) )) - expectStageAnd(shuffleTwoRdd, List( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostC", 1)) - )){ _ => - cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) - cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) - } + } + resetExpecting { blockManagerMaster.removeExecutor("exec-hostA") - expectStage(finalRdd, List( + } + whenExecuting { + respondToTaskSet(reduceStage, List( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) )) - // since we have a cached copy of the missing split of shuffleTwoRdd, we shouldn't - // immediately try to rerun shuffleOneRdd: - expectStage(shuffleTwoRdd, List( + } + // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. + val recomputeTwo = interceptStage(shuffleTwoRdd) { + scheduler.resubmitFailedStages() + } + expectTaskSetLocations(recomputeTwo, Seq(Seq("hostD"))) + val finalRetry = interceptStage(finalRdd) { + respondToTaskSet(recomputeTwo, List( (Success, makeMapStatus("hostD", 1)) - ), Some(Seq(List("hostD")))) - expectStage(finalRdd, List( - (Success, 42) )) } - whenExecuting { - assert(submitRdd(finalRdd) === Array(42)) - } + respondToTaskSet(finalRetry, List( (Success, 42) )) + expectJobResult(Array(42)) } test("cached post-shuffle but fails") { @@ -497,45 +554,58 @@ class DAGSchedulerSuite extends FunSuite val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) val finalRdd = makeRdd(1, List(shuffleDepTwo)) - resetExpecting { - expectGetLocations() - expectStage(shuffleOneRdd, List( + val firstShuffleStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } + cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) + cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) + val secondShuffleStage = interceptStage(shuffleTwoRdd) { + respondToTaskSet(firstShuffleStage, List( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)) + )) + } + val reduceStage = interceptStage(finalRdd) { + respondToTaskSet(secondShuffleStage, List( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)) )) - expectStageAnd(shuffleTwoRdd, List( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostC", 1)) - )){ _ => - cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) - cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) - } + } + resetExpecting { blockManagerMaster.removeExecutor("exec-hostA") - expectStage(finalRdd, List( + } + whenExecuting { + respondToTaskSet(reduceStage, List( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) )) - // since we have a cached copy of the missing split of shuffleTwoRdd, we shouldn't - // immediately try to rerun shuffleOneRdd: - expectStageAnd(shuffleTwoRdd, List( - (FetchFailed(null, shuffleDepOne.shuffleId, 0, 0), null) - ), Some(Seq(List("hostD")))) { _ => - w { - intercept[FetchFailedException]{ - mapOutputTracker.getServerStatuses(shuffleDepOne.shuffleId, 0) - } - } - cacheLocations.remove(shuffleTwoRdd.id -> 0) - } - // after that fetch failure, we should refetch the cache locations and try to recompute - // the whole chain. Note that we will ignore that a fetch failure previously occured on - // this host. - expectStage(shuffleOneRdd, List( (Success, makeMapStatus("hostA", 1)) )) - expectStage(shuffleTwoRdd, List( (Success, makeMapStatus("hostA", 1)) )) - expectStage(finalRdd, List( (Success, 42) )) } - whenExecuting { - assert(submitRdd(finalRdd) === Array(42)) + val recomputeTwoCached = interceptStage(shuffleTwoRdd) { + scheduler.resubmitFailedStages() + } + expectTaskSetLocations(recomputeTwoCached, Seq(Seq("hostD"))) + intercept[FetchFailedException]{ + mapOutputTracker.getServerStatuses(shuffleDepOne.shuffleId, 0) + } + + // Simulate the shuffle input data failing to be cached. + cacheLocations.remove(shuffleTwoRdd.id -> 0) + respondToTaskSet(recomputeTwoCached, List( + (FetchFailed(null, shuffleDepOne.shuffleId, 0, 0), null) + )) + + // After the fetch failure, DAGScheduler should recheck the cache and decide to resubmit + // everything. + val recomputeOne = interceptStage(shuffleOneRdd) { + scheduler.resubmitFailedStages() } + // We use hostA here to make sure DAGScheduler doesn't think it's still dead. + val recomputeTwoUncached = interceptStage(shuffleTwoRdd) { + respondToTaskSet(recomputeOne, List( (Success, makeMapStatus("hostA", 1)) )) + } + expectTaskSetLocations(recomputeTwoUncached, Seq(Seq[String]())) + val finalRetry = interceptStage(finalRdd) { + respondToTaskSet(recomputeTwoUncached, List( (Success, makeMapStatus("hostA", 1)) )) + + } + respondToTaskSet(finalRetry, List( (Success, 42) )) + expectJobResult(Array(42)) } } - From 7f51458774ce4561f1df3ba9b68704c3f63852f3 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 30 Jan 2013 09:34:53 -0800 Subject: [PATCH 111/696] Comment at top of DAGSchedulerSuite --- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 89173540d4009..c31e2e7064715 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -30,9 +30,22 @@ import spark.TaskEndReason import spark.{FetchFailed, Success} +/** + * Tests for DAGScheduler. These tests directly call the event processing functinos in DAGScheduler + * rather than spawning an event loop thread as happens in the real code. They use EasyMock + * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are + * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead + * host notifications are sent). In addition, tests may check for side effects on a non-mocked + * MapOutputTracker instance. + * + * Tests primarily consist of running DAGScheduler#processEvent and + * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) + * and capturing the resulting TaskSets from the mock TaskScheduler. + */ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar with TimeLimitedTests { - // impose a time limit on this test in case we don't let the job finish. + // impose a time limit on this test in case we don't let the job finish, in which case + // JobWaiter#getResult will hang. override val timeLimit = Span(5, Seconds) val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") From f7de6978c14a331683e4a341fccd6e4c5e9fa523 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 29 Jan 2013 14:03:05 -0800 Subject: [PATCH 112/696] Use Mesos ExecutorIDs to hold SlaveIDs. Then we can safely use the Mesos ExecutorID as a Spark ExecutorID. --- .../spark/executor/MesosExecutorBackend.scala | 6 +++- .../mesos/MesosSchedulerBackend.scala | 30 ++++++++++--------- 2 files changed, 21 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 1ef88075ad1e2..b981b26916661 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -32,7 +32,11 @@ private[spark] class MesosExecutorBackend(executor: Executor) logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) this.driver = driver val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor.initialize(executorInfo.getExecutorId.getValue, slaveInfo.getHostname, properties) + executor.initialize( + slaveInfo.getId.getValue + "-" + executorInfo.getExecutorId.getValue, + slaveInfo.getHostname, + properties + ) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index f3467db86b6ee..eab1c60e0b2f9 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -51,7 +51,7 @@ private[spark] class MesosSchedulerBackend( val taskIdToSlaveId = new HashMap[Long, String] // An ExecutorInfo for our tasks - var executorInfo: ExecutorInfo = null + var execArgs: Array[Byte] = null override def start() { synchronized { @@ -70,12 +70,11 @@ private[spark] class MesosSchedulerBackend( } }.start() - executorInfo = createExecutorInfo() waitForRegister() } } - def createExecutorInfo(): ExecutorInfo = { + def createExecutorInfo(execId: String): ExecutorInfo = { val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) @@ -97,7 +96,7 @@ private[spark] class MesosSchedulerBackend( .setEnvironment(environment) .build() ExecutorInfo.newBuilder() - .setExecutorId(ExecutorID.newBuilder().setValue("default").build()) + .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) .setData(ByteString.copyFrom(createExecArg())) .addResources(memory) @@ -109,17 +108,20 @@ private[spark] class MesosSchedulerBackend( * containing all the spark.* system properties in the form of (String, String) pairs. */ private def createExecArg(): Array[Byte] = { - val props = new HashMap[String, String] - val iterator = System.getProperties.entrySet.iterator - while (iterator.hasNext) { - val entry = iterator.next - val (key, value) = (entry.getKey.toString, entry.getValue.toString) - if (key.startsWith("spark.")) { - props(key) = value + if (execArgs == null) { + val props = new HashMap[String, String] + val iterator = System.getProperties.entrySet.iterator + while (iterator.hasNext) { + val entry = iterator.next + val (key, value) = (entry.getKey.toString, entry.getValue.toString) + if (key.startsWith("spark.")) { + props(key) = value + } } + // Serialize the map as an array of (String, String) pairs + execArgs = Utils.serialize(props.toArray) } - // Serialize the map as an array of (String, String) pairs - return Utils.serialize(props.toArray) + return execArgs } override def offerRescinded(d: SchedulerDriver, o: OfferID) {} @@ -216,7 +218,7 @@ private[spark] class MesosSchedulerBackend( return MesosTaskInfo.newBuilder() .setTaskId(taskId) .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) - .setExecutor(executorInfo) + .setExecutor(createExecutorInfo(slaveId)) .setName(task.name) .addResources(cpuResource) .setData(ByteString.copyFrom(task.serializedTask)) From 252845d3046034d6e779bd7245d2f876debba8fd Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 30 Jan 2013 10:38:06 -0800 Subject: [PATCH 113/696] Remove remants of attempt to use slaveId-executorId in MesosExecutorBackend --- core/src/main/scala/spark/executor/MesosExecutorBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index b981b26916661..818d6d1dda7f9 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -33,7 +33,7 @@ private[spark] class MesosExecutorBackend(executor: Executor) this.driver = driver val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) executor.initialize( - slaveInfo.getId.getValue + "-" + executorInfo.getExecutorId.getValue, + executorInfo.getExecutorId.getValue, slaveInfo.getHostname, properties ) From 871476d506a2d543482defb923a42a2a01f206ab Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 30 Jan 2013 16:56:46 -0600 Subject: [PATCH 114/696] Include message and exitStatus if availalbe. --- core/src/main/scala/spark/deploy/worker/Worker.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 5a83a42daff17..8b41620d98232 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -134,7 +134,9 @@ private[spark] class Worker( val fullId = jobId + "/" + execId if (ExecutorState.isFinished(state)) { val executor = executors(fullId) - logInfo("Executor " + fullId + " finished with state " + state) + logInfo("Executor " + fullId + " finished with state " + state + + message.map(" message " + _).getOrElse("") + + exitStatus.map(" exitStatus " + _).getOrElse("")) finishedExecutors(fullId) = executor executors -= fullId coresUsed -= executor.cores From 58a7d320d7287f3773976f0efdf2bc2c1474f7f9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 30 Jan 2013 14:49:18 -0800 Subject: [PATCH 115/696] Inclue packaging and launching pyspark in guide. It's nicer if all the commands you need are made explicit. --- docs/python-programming-guide.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index a840b9b34bb2f..94311bd1020a2 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -67,8 +67,14 @@ The script automatically adds the `pyspark` package to the `PYTHONPATH`. # Interactive Use -The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. -When run without any input files, `pyspark` launches a shell that can be used explore data interactively, which is a simple way to learn the API: +The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: + +{% highlight bash %} +$ sbt/sbt package +$ ./pyspark +{% endhighlight %} + +The Python shell can be used explore data interactively and is a simple way to learn the API: {% highlight python %} >>> words = sc.textFile("/usr/share/dict/words") From 3f945e3b830c5a7d50acd61c5aabf964f40f7f4b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 30 Jan 2013 15:04:06 -0800 Subject: [PATCH 116/696] Make module help available in python shell. Also, adds a line in doc explaining how to use. --- docs/python-programming-guide.md | 1 + python/pyspark/shell.py | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 94311bd1020a2..4e84d23edf56b 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -80,6 +80,7 @@ The Python shell can be used explore data interactively and is a simple way to l >>> words = sc.textFile("/usr/share/dict/words") >>> words.filter(lambda w: w.startswith("spar")).take(5) [u'spar', u'sparable', u'sparada', u'sparadrap', u'sparagrass'] +>>> help(pyspark) # Show all pyspark functions {% endhighlight %} By default, the `pyspark` shell creates SparkContext that runs jobs locally. diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index f6328c561f562..54ff1bf8e7c3e 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -4,6 +4,7 @@ This file is designed to be launched as a PYTHONSTARTUP script. """ import os +import pyspark from pyspark.context import SparkContext From c1df24d0850b0ac89f35f1a47ce6b2fb5b95df0a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 30 Jan 2013 18:51:14 -0800 Subject: [PATCH 117/696] rename Slaves --> Executor --- core/src/main/scala/spark/SparkContext.scala | 6 +++--- core/src/main/scala/spark/storage/BlockManagerUI.scala | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index a09eca1dd0bb4..39e3555de8aac 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -468,7 +468,7 @@ class SparkContext( * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. */ - def getSlavesMemoryStatus: Map[String, (Long, Long)] = { + def getExecutorMemoryStatus: Map[String, (Long, Long)] = { env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) => (blockManagerId.ip + ":" + blockManagerId.port, mem) } @@ -479,13 +479,13 @@ class SparkContext( * they take, etc. */ def getRDDStorageInfo : Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getSlavesStorageStatus, this) + StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } /** * Return information about blocks stored in all of the slaves */ - def getSlavesStorageStatus : Array[StorageStatus] = { + def getExecutorStorageStatus : Array[StorageStatus] = { env.blockManager.master.getStorageStatus } diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 52f6d1b657fda..9e6721ec17169 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -45,7 +45,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, path("") { completeWith { // Request the current storage status from the Master - val storageStatusList = sc.getSlavesStorageStatus + val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) @@ -60,7 +60,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, parameter("id") { id => completeWith { val prefix = "rdd_" + id.toString - val storageStatusList = sc.getSlavesStorageStatus + val storageStatusList = sc.getExecutorStorageStatus val filteredStorageStatusList = StorageUtils. filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head From 3190483b98a62b646311dff199417823efacbb47 Mon Sep 17 00:00:00 2001 From: "haitao.yao" Date: Thu, 31 Jan 2013 14:23:51 +0800 Subject: [PATCH 118/696] bug fix for javadoc --- core/src/main/scala/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b0d4b58240222..91c71ccac3e31 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -436,7 +436,7 @@ class SparkContext( } /** - * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for + * Broadcast a read-only variable to the cluster, returning a [[spark.broadcast.Broadcast]] object for * reading it in distributed functions. The variable will be sent to each cluster only once. */ def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) From fe3eceab5724bec0103471eb905bb9701120b04a Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Thu, 31 Jan 2013 13:30:41 -0800 Subject: [PATCH 119/696] Remove activation of profiles by default See the discussion at https://github.com/mesos/spark/pull/355 for why default profile activation is a problem. --- bagel/pom.xml | 11 ----------- core/pom.xml | 11 ----------- examples/pom.xml | 11 ----------- pom.xml | 11 ----------- repl-bin/pom.xml | 11 ----------- repl/pom.xml | 11 ----------- streaming/pom.xml | 11 ----------- 7 files changed, 77 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 5f58347204289..a8256a6e8b0ea 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -45,11 +45,6 @@ hadoop1 - - - !hadoopVersion - - org.spark-project @@ -77,12 +72,6 @@ hadoop2 - - - hadoopVersion - 2 - - org.spark-project diff --git a/core/pom.xml b/core/pom.xml index 862d3ec37ac8f..873e8a1d0fe33 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -163,11 +163,6 @@ hadoop1 - - - !hadoopVersion - - org.apache.hadoop @@ -220,12 +215,6 @@ hadoop2 - - - hadoopVersion - 2 - - org.apache.hadoop diff --git a/examples/pom.xml b/examples/pom.xml index 4d4310347548e..f43af670c613f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -50,11 +50,6 @@ hadoop1 - - - !hadoopVersion - - org.spark-project @@ -88,12 +83,6 @@ hadoop2 - - - hadoopVersion - 2 - - org.spark-project diff --git a/pom.xml b/pom.xml index 3ea989a082b2a..c6b9012dc67b2 100644 --- a/pom.xml +++ b/pom.xml @@ -499,11 +499,6 @@ hadoop1 - - - !hadoopVersion - - 1 @@ -521,12 +516,6 @@ hadoop2 - - - hadoopVersion - 2 - - 2 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index da91c0f3ab2b1..0667b71cc7bba 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -70,11 +70,6 @@ hadoop1 - - - !hadoopVersion - - hadoop1 @@ -115,12 +110,6 @@ hadoop2 - - - hadoopVersion - 2 - - hadoop2 diff --git a/repl/pom.xml b/repl/pom.xml index 2dc96beaf55fe..4a296fa630106 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -72,11 +72,6 @@ hadoop1 - - - !hadoopVersion - - hadoop1 @@ -128,12 +123,6 @@ hadoop2 - - - hadoopVersion - 2 - - hadoop2 diff --git a/streaming/pom.xml b/streaming/pom.xml index 3dae815e1a6fa..6ee7e59df39d1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -83,11 +83,6 @@ hadoop1 - - - !hadoopVersion - - org.spark-project @@ -115,12 +110,6 @@ hadoop2 - - - hadoopVersion - 2 - - org.spark-project From 418e36caa8fcd9a70026ab762ec709732fdebd6b Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Thu, 31 Jan 2013 17:18:33 -0600 Subject: [PATCH 120/696] Add more private declarations. --- .../main/scala/spark/MapOutputTracker.scala | 2 +- .../spark/deploy/master/MasterWebUI.scala | 22 +++------ .../scala/spark/scheduler/DAGScheduler.scala | 46 ++++++++++--------- .../spark/scheduler/ShuffleMapTask.scala | 3 +- .../scheduler/cluster/ClusterScheduler.scala | 2 +- .../scheduler/cluster/TaskSetManager.scala | 19 ++++---- .../scheduler/local/LocalScheduler.scala | 4 +- .../scala/spark/util/MetadataCleaner.scala | 10 ++-- 8 files changed, 49 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index aaf433b324fec..473520758557d 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -170,7 +170,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isDriver: Boolea } } - def cleanup(cleanupTime: Long) { + private def cleanup(cleanupTime: Long) { mapStatuses.clearOldValues(cleanupTime) cachedSerializedStatuses.clearOldValues(cleanupTime) } diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index a01774f511007..529f72e9da1c2 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -45,13 +45,9 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct case (jobId, Some(js)) if (js.equalsIgnoreCase("json")) => val future = master ? RequestMasterState val jobInfo = for (masterState <- future.mapTo[MasterState]) yield { - masterState.activeJobs.find(_.id == jobId) match { - case Some(job) => job - case _ => masterState.completedJobs.find(_.id == jobId) match { - case Some(job) => job - case _ => null - } - } + masterState.activeJobs.find(_.id == jobId).getOrElse({ + masterState.completedJobs.find(_.id == jobId).getOrElse(null) + }) } respondWithMediaType(MediaTypes.`application/json`) { ctx => ctx.complete(jobInfo.mapTo[JobInfo]) @@ -61,14 +57,10 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct val future = master ? RequestMasterState future.map { state => val masterState = state.asInstanceOf[MasterState] - - masterState.activeJobs.find(_.id == jobId) match { - case Some(job) => spark.deploy.master.html.job_details.render(job) - case _ => masterState.completedJobs.find(_.id == jobId) match { - case Some(job) => spark.deploy.master.html.job_details.render(job) - case _ => null - } - } + val job = masterState.activeJobs.find(_.id == jobId).getOrElse({ + masterState.completedJobs.find(_.id == jobId).getOrElse(null) + }) + spark.deploy.master.html.job_details.render(job) } } } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b130be6a3863a..14f61f7e87bc8 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -97,7 +97,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } }.start() - def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { + private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.splits.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map { @@ -107,7 +107,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with cacheLocs(rdd.id) } - def clearCacheLocs() { + private def clearCacheLocs() { cacheLocs.clear() } @@ -116,7 +116,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * The priority value passed in will be used if the stage doesn't already exist with * a lower priority (we assume that priorities always increase across jobs for now). */ - def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = { + private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = { shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => @@ -131,11 +131,11 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * as a result stage for the final RDD used directly in an action. The stage will also be given * the provided priority. */ - def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { - // Kind of ugly: need to register RDDs with the cache and map output tracker here - // since we can't do it in the RDD constructor because # of splits is unknown - logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") + private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { if (shuffleDep != None) { + // Kind of ugly: need to register RDDs with the cache and map output tracker here + // since we can't do it in the RDD constructor because # of splits is unknown + logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.splits.size) } val id = nextStageId.getAndIncrement() @@ -148,7 +148,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * Get or create the list of parent stages for a given RDD. The stages will be assigned the * provided priority if they haven't already been created with a lower priority. */ - def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = { + private def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = { val parents = new HashSet[Stage] val visited = new HashSet[RDD[_]] def visit(r: RDD[_]) { @@ -170,7 +170,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with parents.toList } - def getMissingParentStages(stage: Stage): List[Stage] = { + private def getMissingParentStages(stage: Stage): List[Stage] = { val missing = new HashSet[Stage] val visited = new HashSet[RDD[_]] def visit(rdd: RDD[_]) { @@ -241,7 +241,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * events and responds by launching tasks. This runs in a dedicated thread and receives events * via the eventQueue. */ - def run() { + private def run() { SparkEnv.set(env) while (true) { @@ -326,7 +326,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * We run the operation in a separate thread just in case it takes a bunch of time, so that we * don't block the DAGScheduler event loop or other concurrent jobs. */ - def runLocally(job: ActiveJob) { + private def runLocally(job: ActiveJob) { logInfo("Computing the requested partition locally") new Thread("Local computation of job " + job.runId) { override def run() { @@ -349,13 +349,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with }.start() } - def submitStage(stage: Stage) { + /** Submits stage, but first recursively submits any missing parents. */ + private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) if (missing == Nil) { - logInfo("Submitting " + stage + " (" + stage.origin + "), which has no missing parents") + logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage) running += stage } else { @@ -367,7 +368,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } } - def submitMissingTasks(stage: Stage) { + /** Called when stage's parents are available and we can now do its task. */ + private def submitMissingTasks(stage: Stage) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet) @@ -388,7 +390,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } } if (tasks.size > 0) { - logInfo("Submitting " + tasks.size + " missing tasks from " + stage) + logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) taskSched.submitTasks( @@ -407,7 +409,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. */ - def handleTaskCompletion(event: CompletionEvent) { + private def handleTaskCompletion(event: CompletionEvent) { val task = event.task val stage = idToStage(task.stageId) @@ -492,7 +494,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with waiting --= newlyRunnable running ++= newlyRunnable for (stage <- newlyRunnable.sortBy(_.id)) { - logInfo("Submitting " + stage + " (" + stage.origin + "), which is now runnable") + logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable") submitMissingTasks(stage) } } @@ -541,7 +543,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * Optionally the generation during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - def handleExecutorLost(execId: String, maybeGeneration: Option[Long] = None) { + private def handleExecutorLost(execId: String, maybeGeneration: Option[Long] = None) { val currentGeneration = maybeGeneration.getOrElse(mapOutputTracker.getGeneration) if (!failedGeneration.contains(execId) || failedGeneration(execId) < currentGeneration) { failedGeneration(execId) = currentGeneration @@ -567,7 +569,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being cancelled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. */ - def abortStage(failedStage: Stage, reason: String) { + private def abortStage(failedStage: Stage, reason: String) { val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) @@ -583,7 +585,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with /** * Return true if one of stage's ancestors is target. */ - def stageDependsOn(stage: Stage, target: Stage): Boolean = { + private def stageDependsOn(stage: Stage, target: Stage): Boolean = { if (stage == target) { return true } @@ -610,7 +612,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with visitedRdds.contains(target.rdd) } - def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = { + private def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = { // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) if (cached != Nil) { @@ -636,7 +638,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with return Nil } - def cleanup(cleanupTime: Long) { + private def cleanup(cleanupTime: Long) { var sizeBefore = idToStage.size idToStage.clearOldValues(cleanupTime) logInfo("idToStage " + sizeBefore + " --> " + idToStage.size) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 83641a2a8427b..b701b67c89fba 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -127,7 +127,6 @@ private[spark] class ShuffleMapTask( val bucketId = dep.partitioner.getPartition(pair._1) buckets(bucketId) += pair } - val bucketIterators = buckets.map(_.iterator) val compressedSizes = new Array[Byte](numOutputSplits) @@ -135,7 +134,7 @@ private[spark] class ShuffleMapTask( for (i <- 0 until numOutputSplits) { val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i // Get a Scala iterator from Java map - val iter: Iterator[(Any, Any)] = bucketIterators(i) + val iter: Iterator[(Any, Any)] = buckets(i).iterator val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) compressedSizes(i) = MapOutputTracker.compressSize(size) } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 0b4177805b374..1e4fbdb8742fd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -86,7 +86,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - def submitTasks(taskSet: TaskSet) { + override def submitTasks(taskSet: TaskSet) { val tasks = taskSet.tasks logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") this.synchronized { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 26201ad0dd1a0..3dabdd76b1aa5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,10 +17,7 @@ import java.nio.ByteBuffer /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class TaskSetManager( - sched: ClusterScheduler, - val taskSet: TaskSet) - extends Logging { +private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) extends Logging { // Maximum time to wait to run a task in a preferred location (in ms) val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong @@ -100,7 +97,7 @@ private[spark] class TaskSetManager( } // Add a task to all the pending-task lists that it should be on. - def addPendingTask(index: Int) { + private def addPendingTask(index: Int) { val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive if (locations.size == 0) { pendingTasksWithNoPrefs += index @@ -115,7 +112,7 @@ private[spark] class TaskSetManager( // Return the pending tasks list for a given host, or an empty list if // there is no map entry for that host - def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { + private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { pendingTasksForHost.getOrElse(host, ArrayBuffer()) } @@ -123,7 +120,7 @@ private[spark] class TaskSetManager( // Return None if the list is empty. // This method also cleans up any tasks in the list that have already // been launched, since we want that to happen lazily. - def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { + private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { while (!list.isEmpty) { val index = list.last list.trimEnd(1) @@ -137,7 +134,7 @@ private[spark] class TaskSetManager( // Return a speculative task for a given host if any are available. The task should not have an // attempt running on this host, in case the host is slow. In addition, if localOnly is set, the // task must have a preference for this host (or no preferred locations at all). - def findSpeculativeTask(host: String, localOnly: Boolean): Option[Int] = { + private def findSpeculativeTask(host: String, localOnly: Boolean): Option[Int] = { val hostsAlive = sched.hostsAlive speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set val localTask = speculatableTasks.find { @@ -162,7 +159,7 @@ private[spark] class TaskSetManager( // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. - def findTask(host: String, localOnly: Boolean): Option[Int] = { + private def findTask(host: String, localOnly: Boolean): Option[Int] = { val localTask = findTaskFromList(getPendingTasksForHost(host)) if (localTask != None) { return localTask @@ -184,7 +181,7 @@ private[spark] class TaskSetManager( // Does a host count as a preferred location for a task? This is true if // either the task has preferred locations and this host is one, or it has // no preferred locations (in which we still count the launch as preferred). - def isPreferredLocation(task: Task[_], host: String): Boolean = { + private def isPreferredLocation(task: Task[_], host: String): Boolean = { val locs = task.preferredLocations return (locs.contains(host) || locs.isEmpty) } @@ -335,7 +332,7 @@ private[spark] class TaskSetManager( if (numFailures(index) > MAX_TASK_FAILURES) { logError("Task %s:%d failed more than %d times; aborting job".format( taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %d failed more than %d times".format(index, MAX_TASK_FAILURES)) + abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) } } } else { diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 9ff7c02097b2e..482d1cc85343a 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -53,7 +53,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } def runTask(task: Task[_], idInJob: Int, attemptId: Int) { - logInfo("Running task " + idInJob) + logInfo("Running " + task) // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { @@ -80,7 +80,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon val resultToReturn = ser.deserialize[Any](ser.serialize(result)) val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( ser.serialize(Accumulators.values)) - logInfo("Finished task " + idInJob) + logInfo("Finished " + task) // If the threadpool has not already been shutdown, notify DAGScheduler if (!Thread.currentThread().isInterrupted) diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index eaff7ae581ec5..a342d378ffab2 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -9,12 +9,12 @@ import spark.Logging * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) */ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging { - val delaySeconds = MetadataCleaner.getDelaySeconds - val periodSeconds = math.max(10, delaySeconds / 10) - val timer = new Timer(name + " cleanup timer", true) + private val delaySeconds = MetadataCleaner.getDelaySeconds + private val periodSeconds = math.max(10, delaySeconds / 10) + private val timer = new Timer(name + " cleanup timer", true) - val task = new TimerTask { - def run() { + private val task = new TimerTask { + override def run() { try { cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000)) logInfo("Ran metadata cleaner for " + name) From 782187c21047ee31728bdb173a2b7ee708cef77b Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Thu, 31 Jan 2013 18:27:25 -0600 Subject: [PATCH 121/696] Once we find a split with no block, we don't have to look for more. --- .../scala/spark/scheduler/DAGScheduler.scala | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b130be6a3863a..b62b25f6880a7 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -177,18 +177,17 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with if (!visited(rdd)) { visited += rdd val locs = getCacheLocs(rdd) - for (p <- 0 until rdd.splits.size) { - if (locs(p) == Nil) { - for (dep <- rdd.dependencies) { - dep match { - case shufDep: ShuffleDependency[_,_] => - val mapStage = getShuffleMapStage(shufDep, stage.priority) - if (!mapStage.isAvailable) { - missing += mapStage - } - case narrowDep: NarrowDependency[_] => - visit(narrowDep.rdd) - } + val atLeastOneMissing = (0 until rdd.splits.size).exists(locs(_) == Nil) + if (atLeastOneMissing) { + for (dep <- rdd.dependencies) { + dep match { + case shufDep: ShuffleDependency[_,_] => + val mapStage = getShuffleMapStage(shufDep, stage.priority) + if (!mapStage.isAvailable) { + missing += mapStage + } + case narrowDep: NarrowDependency[_] => + visit(narrowDep.rdd) } } } From 5b0fc265c2f2ce461d61904c2a4e6e47b24d2bbe Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 31 Jan 2013 17:48:39 -0800 Subject: [PATCH 122/696] Changed PartitionPruningRDD's split to make sure it returns the correct split index. --- core/src/main/scala/spark/Dependency.scala | 8 ++++++++ core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 4 +++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 647aee6eb54cc..827eac850a50b 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -72,6 +72,14 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo @transient val partitions: Array[Split] = rdd.splits.filter(s => partitionFilterFunc(s.index)) + .zipWithIndex + .map { case(split, idx) => new PruneDependency.PartitionPruningRDDSplit(idx, split) : Split } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } + +object PruneDependency { + class PartitionPruningRDDSplit(idx: Int, val parentSplit: Split) extends Split { + override val index = idx + } +} diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index b8482338c6f8f..0989b149e1a9f 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -2,6 +2,7 @@ package spark.rdd import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} + /** * A RDD used to prune RDD partitions/splits so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, @@ -15,7 +16,8 @@ class PartitionPruningRDD[T: ClassManifest]( @transient partitionFilterFunc: Int => Boolean) extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { - override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context) + override def compute(split: Split, context: TaskContext) = firstParent[T].iterator( + split.asInstanceOf[PruneDependency.PartitionPruningRDDSplit].parentSplit, context) override protected def getSplits = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions From 6289d9654e32fc92418d41cc6e32fee30f85c833 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 31 Jan 2013 17:49:36 -0800 Subject: [PATCH 123/696] Removed the TODO comment from PartitionPruningRDD. --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 0989b149e1a9f..3756870faca12 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -8,8 +8,6 @@ import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. - * - * TODO: This currently doesn't give partition IDs properly! */ class PartitionPruningRDD[T: ClassManifest]( @transient prev: RDD[T], From 3446d5c8d6b385106ac85e46320d92faa8efb4e6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 31 Jan 2013 18:02:28 -0800 Subject: [PATCH 124/696] SPARK-673: Capture and re-throw Python exceptions This patch alters the Python <-> executor protocol to pass on exception data when they occur in user Python code. --- .../scala/spark/api/python/PythonRDD.scala | 40 ++++++++++++------- python/pyspark/worker.py | 10 ++++- 2 files changed, 34 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index f43a152ca7498..6b9ef62529bdc 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -103,21 +103,30 @@ private[spark] class PythonRDD[T: ClassManifest]( private def read(): Array[Byte] = { try { - val length = stream.readInt() - if (length != -1) { - val obj = new Array[Byte](length) - stream.readFully(obj) - obj - } else { - // We've finished the data section of the output, but we can still read some - // accumulator updates; let's do that, breaking when we get EOFException - while (true) { - val len2 = stream.readInt() - val update = new Array[Byte](len2) - stream.readFully(update) - accumulator += Collections.singletonList(update) + stream.readInt() match { + case length if length > 0 => { + val obj = new Array[Byte](length) + stream.readFully(obj) + obj } - new Array[Byte](0) + case -2 => { + // Signals that an exception has been thrown in python + val exLength = stream.readInt() + val obj = new Array[Byte](exLength) + stream.readFully(obj) + throw new PythonException(new String(obj)) + } + case -1 => { + // We've finished the data section of the output, but we can still read some + // accumulator updates; let's do that, breaking when we get EOFException + while (true) { + val len2 = stream.readInt() + val update = new Array[Byte](len2) + stream.readFully(update) + accumulator += Collections.singletonList(update) + } + new Array[Byte](0) + } } } catch { case eof: EOFException => { @@ -140,6 +149,9 @@ private[spark] class PythonRDD[T: ClassManifest]( val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) } +/** Thrown for exceptions in user Python code. */ +private class PythonException(msg: String) extends Exception(msg) + /** * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. * This is used by PySpark's shuffle operations. diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index d33d6dd15f0de..9622e0cfe4562 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -2,6 +2,7 @@ Worker that receives input from Piped RDD. """ import sys +import traceback from base64 import standard_b64decode # CloudPickler needs to be imported so that depicklers are registered using the # copy_reg module. @@ -40,8 +41,13 @@ def main(): else: dumps = dump_pickle iterator = read_from_pickle_file(sys.stdin) - for obj in func(split_index, iterator): - write_with_length(dumps(obj), old_stdout) + try: + for obj in func(split_index, iterator): + write_with_length(dumps(obj), old_stdout) + except Exception as e: + write_int(-2, old_stdout) + write_with_length(traceback.format_exc(), old_stdout) + sys.exit(-1) # Mark the beginning of the accumulators section of the output write_int(-1, old_stdout) for aid, accum in _accumulatorRegistry.items(): From bf675ab4f90fdbea67e42a8df828ef15ed87a086 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 31 Jan 2013 21:43:45 -0800 Subject: [PATCH 125/696] Turn on ganglia by default --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index cafb7bf011064..ce1072fd3963a 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -84,7 +84,7 @@ def parse_args(): "maximum price (in dollars)") parser.add_option("-c", "--cluster-type", default="mesos", help="'mesos' for a mesos cluster, 'standalone' for a standalone spark cluster (default: mesos)") - parser.add_option("-g", "--ganglia", action="store_true", default=False, + parser.add_option("-g", "--ganglia", action="store_true", default=True, help="Setup ganglia monitoring for the cluster. NOTE: The ganglia " + "monitoring page will be publicly accessible") parser.add_option("--mesos-scripts", action="store_true", default=False, From c33f0ef41a1865de2bae01b52b860650d3734da4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 31 Jan 2013 21:50:02 -0800 Subject: [PATCH 126/696] Some style cleanup --- core/src/main/scala/spark/api/python/PythonRDD.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 6b9ef62529bdc..23e3149248e5e 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -104,19 +104,17 @@ private[spark] class PythonRDD[T: ClassManifest]( private def read(): Array[Byte] = { try { stream.readInt() match { - case length if length > 0 => { + case length if length > 0 => val obj = new Array[Byte](length) stream.readFully(obj) obj - } - case -2 => { + case -2 => // Signals that an exception has been thrown in python val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) throw new PythonException(new String(obj)) - } - case -1 => { + case -1 => // We've finished the data section of the output, but we can still read some // accumulator updates; let's do that, breaking when we get EOFException while (true) { @@ -124,9 +122,8 @@ private[spark] class PythonRDD[T: ClassManifest]( val update = new Array[Byte](len2) stream.readFully(update) accumulator += Collections.singletonList(update) + new Array[Byte](0) } - new Array[Byte](0) - } } } catch { case eof: EOFException => { From 39ab83e9577a5449fb0d6ef944dffc0d7cd00b4a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 31 Jan 2013 21:52:52 -0800 Subject: [PATCH 127/696] Small fix from last commit --- core/src/main/scala/spark/api/python/PythonRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 23e3149248e5e..39758e94f46fe 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -122,8 +122,8 @@ private[spark] class PythonRDD[T: ClassManifest]( val update = new Array[Byte](len2) stream.readFully(update) accumulator += Collections.singletonList(update) - new Array[Byte](0) } + new Array[Byte](0) } } catch { case eof: EOFException => { From f9af9cee6fed9c6af896fb92556ad4f48c7f8e64 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 1 Feb 2013 00:02:46 -0800 Subject: [PATCH 128/696] Moved PruneDependency into PartitionPruningRDD.scala. --- core/src/main/scala/spark/Dependency.scala | 22 ---------------- .../scala/spark/rdd/PartitionPruningRDD.scala | 26 ++++++++++++++++--- 2 files changed, 22 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 827eac850a50b..5eea9073220f1 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -61,25 +61,3 @@ class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) } } } - - -/** - * Represents a dependency between the PartitionPruningRDD and its parent. In this - * case, the child RDD contains a subset of partitions of the parents'. - */ -class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) - extends NarrowDependency[T](rdd) { - - @transient - val partitions: Array[Split] = rdd.splits.filter(s => partitionFilterFunc(s.index)) - .zipWithIndex - .map { case(split, idx) => new PruneDependency.PartitionPruningRDDSplit(idx, split) : Split } - - override def getParents(partitionId: Int) = List(partitions(partitionId).index) -} - -object PruneDependency { - class PartitionPruningRDDSplit(idx: Int, val parentSplit: Split) extends Split { - override val index = idx - } -} diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 3756870faca12..a50ce751718c4 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -1,6 +1,26 @@ package spark.rdd -import spark.{PruneDependency, RDD, SparkEnv, Split, TaskContext} +import spark.{NarrowDependency, RDD, SparkEnv, Split, TaskContext} + + +class PartitionPruningRDDSplit(idx: Int, val parentSplit: Split) extends Split { + override val index = idx +} + + +/** + * Represents a dependency between the PartitionPruningRDD and its parent. In this + * case, the child RDD contains a subset of partitions of the parents'. + */ +class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) + extends NarrowDependency[T](rdd) { + + @transient + val partitions: Array[Split] = rdd.splits.filter(s => partitionFilterFunc(s.index)) + .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDSplit(idx, split) : Split } + + override def getParents(partitionId: Int) = List(partitions(partitionId).index) +} /** @@ -15,10 +35,8 @@ class PartitionPruningRDD[T: ClassManifest]( extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { override def compute(split: Split, context: TaskContext) = firstParent[T].iterator( - split.asInstanceOf[PruneDependency.PartitionPruningRDDSplit].parentSplit, context) + split.asInstanceOf[PartitionPruningRDDSplit].parentSplit, context) override protected def getSplits = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions - - override val partitioner = firstParent[T].partitioner } From f127f2ae76692b189d86b5a47293579d5657c6d5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 1 Feb 2013 00:20:49 -0800 Subject: [PATCH 129/696] fixup merge (master -> driver renaming) --- core/src/main/scala/spark/storage/BlockManagerMaster.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 99324445caeee..0372cb080aa0a 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -118,7 +118,7 @@ private[spark] class BlockManagerMaster( } def getStorageStatus: Array[StorageStatus] = { - askMasterWithRetry[ArrayBuffer[StorageStatus]](GetStorageStatus).toArray + askDriverWithReply[ArrayBuffer[StorageStatus]](GetStorageStatus).toArray } /** Stop the driver actor, called only on the Spark driver node */ From 8a0a5ed53353ad6aa5656eb729d55ca7af2ab096 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 1 Feb 2013 00:23:38 -0800 Subject: [PATCH 130/696] track total partitions, in addition to cached partitions; use scala string formatting --- core/src/main/scala/spark/storage/StorageUtils.scala | 10 ++++------ core/src/main/twirl/spark/storage/rdd.scala.html | 6 +++++- core/src/main/twirl/spark/storage/rdd_table.scala.html | 6 ++++-- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index ce7c067eea57e..5367b74bb6257 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -22,12 +22,11 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numPartitions: Int, memSize: Long, diskSize: Long) { + numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) { override def toString = { import Utils.memoryBytesToString - import java.lang.{Integer => JInt} - String.format("RDD \"%s\" (%d) Storage: %s; Partitions: %d; MemorySize: %s; DiskSize: %s", name, id.asInstanceOf[JInt], - storageLevel.toString, numPartitions.asInstanceOf[JInt], memoryBytesToString(memSize), memoryBytesToString(diskSize)) + "RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id, + storageLevel.toString, numCachedPartitions, numPartitions, memoryBytesToString(memSize), memoryBytesToString(diskSize)) } } @@ -65,9 +64,8 @@ object StorageUtils { val rdd = sc.persistentRdds(rddId) val rddName = Option(rdd.name).getOrElse(rddKey) val rddStorageLevel = rdd.getStorageLevel - //TODO get total number of partitions in rdd - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, memSize, diskSize) + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.splits.size, memSize, diskSize) }.toArray } diff --git a/core/src/main/twirl/spark/storage/rdd.scala.html b/core/src/main/twirl/spark/storage/rdd.scala.html index ac7f8c981fa83..d85addeb1755e 100644 --- a/core/src/main/twirl/spark/storage/rdd.scala.html +++ b/core/src/main/twirl/spark/storage/rdd.scala.html @@ -11,7 +11,11 @@ Storage Level: @(rddInfo.storageLevel.description)
  • - Partitions: + Cached Partitions: + @(rddInfo.numCachedPartitions) +
  • +
  • + Total Partitions: @(rddInfo.numPartitions)
  • diff --git a/core/src/main/twirl/spark/storage/rdd_table.scala.html b/core/src/main/twirl/spark/storage/rdd_table.scala.html index af801cf229569..a51e64aed0072 100644 --- a/core/src/main/twirl/spark/storage/rdd_table.scala.html +++ b/core/src/main/twirl/spark/storage/rdd_table.scala.html @@ -6,7 +6,8 @@ RDD Name Storage Level - Partitions + Cached Partitions + Fraction Partitions Cached Size in Memory Size on Disk @@ -21,7 +22,8 @@ @(rdd.storageLevel.description) - @rdd.numPartitions + @rdd.numCachedPartitions + @(rdd.numCachedPartitions / rdd.numPartitions.toDouble) @{Utils.memoryBytesToString(rdd.memSize)} @{Utils.memoryBytesToString(rdd.diskSize)} From 57b64d0d1902eb51bf79f595626c2b9f80a9d1e2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 1 Feb 2013 00:25:19 -0800 Subject: [PATCH 131/696] Fix stdout redirection in PySpark. --- python/pyspark/tests.py | 9 +++++++++ python/pyspark/worker.py | 5 +++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index df7235756d37d..52297d44e630a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -108,5 +108,14 @@ def func(): self.assertEqual("Hello World!", UserClass().hello()) +class TestIO(PySparkTestCase): + + def test_stdout_redirection(self): + import subprocess + def func(x): + subprocess.check_call('ls', shell=True) + self.sc.parallelize([1]).foreach(func) + + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 9622e0cfe4562..812e7a9da5a03 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,6 +1,7 @@ """ Worker that receives input from Piped RDD. """ +import os import sys import traceback from base64 import standard_b64decode @@ -15,8 +16,8 @@ # Redirect stdout to stderr so that users must return values from functions. -old_stdout = sys.stdout -sys.stdout = sys.stderr +old_stdout = os.fdopen(os.dup(1), 'w') +os.dup2(2, 1) def load_obj(): From 59c57e48dfb362923610785b230d5b3b56c620c3 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 1 Feb 2013 10:34:02 -0600 Subject: [PATCH 132/696] Stop BlockManagers metadataCleaner. --- core/src/main/scala/spark/storage/BlockManager.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index c61fd75c2bc11..9893e9625d909 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -950,6 +950,7 @@ class BlockManager( blockInfo.clear() memoryStore.clear() diskStore.clear() + metadataCleaner.cancel() logInfo("BlockManager stopped") } } From c6190067ae40cf457b7f2e58619904b6fd2b1cb6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 1 Feb 2013 09:55:25 -0800 Subject: [PATCH 133/696] remove unneeded (and unused) filter on block info --- core/src/main/scala/spark/storage/StorageUtils.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 5367b74bb6257..5f72b67b2cc27 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -43,8 +43,6 @@ object StorageUtils { /* Given a list of BlockStatus objets, returns information for each RDD */ def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], sc: SparkContext) : Array[RDDInfo] = { - // Find all RDD Blocks (ignore broadcast variables) - val rddBlocks = infos.filterKeys(_.startsWith("rdd")) // Group by rddId, ignore the partition name val groupedRddBlocks = infos.groupBy { case(k, v) => From 9cc6ff9c4e7eec2d62261fc166ad2ebade148752 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 1 Feb 2013 11:09:56 -0800 Subject: [PATCH 134/696] Do not launch JavaGateways on workers (SPARK-674). The problem was that the gateway was being initialized whenever the pyspark.context module was loaded. The fix uses lazy initialization that occurs only when SparkContext instances are actually constructed. I also made the gateway and jvm variables private. This change results in ~3-4x performance improvement when running the PySpark unit tests. --- python/pyspark/context.py | 27 +++++++++++++++++---------- python/pyspark/files.py | 2 +- python/pyspark/rdd.py | 12 ++++++------ python/pyspark/tests.py | 2 +- 4 files changed, 25 insertions(+), 18 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 783e3dc148609..ba6896dda3526 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -24,11 +24,10 @@ class SparkContext(object): broadcast variables on that cluster. """ - gateway = launch_gateway() - jvm = gateway.jvm - _readRDDFromPickleFile = jvm.PythonRDD.readRDDFromPickleFile - _writeIteratorToPickleFile = jvm.PythonRDD.writeIteratorToPickleFile - _takePartition = jvm.PythonRDD.takePartition + _gateway = None + _jvm = None + _writeIteratorToPickleFile = None + _takePartition = None _next_accum_id = 0 _active_spark_context = None _lock = Lock() @@ -56,6 +55,13 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, raise ValueError("Cannot run multiple SparkContexts at once") else: SparkContext._active_spark_context = self + if not SparkContext._gateway: + SparkContext._gateway = launch_gateway() + SparkContext._jvm = SparkContext._gateway.jvm + SparkContext._writeIteratorToPickleFile = \ + SparkContext._jvm.PythonRDD.writeIteratorToPickleFile + SparkContext._takePartition = \ + SparkContext._jvm.PythonRDD.takePartition self.master = master self.jobName = jobName self.sparkHome = sparkHome or None # None becomes null in Py4J @@ -63,8 +69,8 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, self.batchSize = batchSize # -1 represents a unlimited batch size # Create the Java SparkContext through Py4J - empty_string_array = self.gateway.new_array(self.jvm.String, 0) - self._jsc = self.jvm.JavaSparkContext(master, jobName, sparkHome, + empty_string_array = self._gateway.new_array(self._jvm.String, 0) + self._jsc = self._jvm.JavaSparkContext(master, jobName, sparkHome, empty_string_array) # Create a single Accumulator in Java that we'll send all our updates through; @@ -72,8 +78,8 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, self._accumulatorServer = accumulators._start_update_server() (host, port) = self._accumulatorServer.server_address self._javaAccumulator = self._jsc.accumulator( - self.jvm.java.util.ArrayList(), - self.jvm.PythonAccumulatorParam(host, port)) + self._jvm.java.util.ArrayList(), + self._jvm.PythonAccumulatorParam(host, port)) self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') # Broadcast's __reduce__ method stores Broadcast instances here. @@ -127,7 +133,8 @@ def parallelize(self, c, numSlices=None): for x in c: write_with_length(dump_pickle(x), tempFile) tempFile.close() - jrdd = self._readRDDFromPickleFile(self._jsc, tempFile.name, numSlices) + readRDDFromPickleFile = self._jvm.PythonRDD.readRDDFromPickleFile + jrdd = readRDDFromPickleFile(self._jsc, tempFile.name, numSlices) return RDD(jrdd, self) def textFile(self, name, minSplits=None): diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 98f6a399cc26b..001b7a28b6691 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -35,4 +35,4 @@ def getRootDirectory(cls): return cls._root_directory else: # This will have to change if we support multiple SparkContexts: - return cls._sc.jvm.spark.SparkFiles.getRootDirectory() + return cls._sc._jvm.spark.SparkFiles.getRootDirectory() diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d53355a8f1db4..d7cad2f3725aa 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -407,7 +407,7 @@ def func(split, iterator): return (str(x).encode("utf-8") for x in iterator) keyed = PipelinedRDD(self, func) keyed._bypass_serializer = True - keyed._jrdd.map(self.ctx.jvm.BytesToString()).saveAsTextFile(path) + keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) # Pair functions @@ -550,8 +550,8 @@ def add_shuffle_key(split, iterator): yield dump_pickle(Batch(items)) keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True - pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() - partitioner = self.ctx.jvm.PythonPartitioner(numSplits, + pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() + partitioner = self.ctx._jvm.PythonPartitioner(numSplits, id(partitionFunc)) jrdd = pairRDD.partitionBy(partitioner).values() rdd = RDD(jrdd, self.ctx) @@ -730,13 +730,13 @@ def batched_func(split, iterator): pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], - self.ctx.gateway._gateway_client) + self.ctx._gateway._gateway_client) self.ctx._pickled_broadcast_vars.clear() class_manifest = self._prev_jrdd.classManifest() env = copy.copy(self.ctx.environment) env['PYTHONPATH'] = os.environ.get("PYTHONPATH", "") - env = MapConverter().convert(env, self.ctx.gateway._gateway_client) - python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), + env = MapConverter().convert(env, self.ctx._gateway._gateway_client) + python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), pipe_command, env, self.preservesPartitioning, self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator, class_manifest) self._jrdd_val = python_rdd.asJavaRDD() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 52297d44e630a..6a1962d26795d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -26,7 +26,7 @@ def tearDown(self): sys.path = self._old_sys_path # To avoid Akka rebinding to the same port, since it doesn't unbind # immediately on shutdown - self.sc.jvm.System.clearProperty("spark.driver.port") + self.sc._jvm.System.clearProperty("spark.driver.port") class TestCheckpoint(PySparkTestCase): From e211f405bcb3cf02c3ae589cf81d9c9dfc70bc03 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 1 Feb 2013 11:48:11 -0800 Subject: [PATCH 135/696] Use spark.local.dir for PySpark temp files (SPARK-580). --- python/pyspark/context.py | 12 ++++++++---- python/pyspark/rdd.py | 7 +------ 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index ba6896dda3526..6831f9b7f8b95 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1,8 +1,6 @@ import os -import atexit import shutil import sys -import tempfile from threading import Lock from tempfile import NamedTemporaryFile @@ -94,6 +92,11 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, SparkFiles._sc = self sys.path.append(SparkFiles.getRootDirectory()) + # Create a temporary directory inside spark.local.dir: + local_dir = self._jvm.spark.Utils.getLocalDir() + self._temp_dir = \ + self._jvm.spark.Utils.createTempDir(local_dir).getAbsolutePath() + @property def defaultParallelism(self): """ @@ -126,8 +129,7 @@ def parallelize(self, c, numSlices=None): # Calling the Java parallelize() method with an ArrayList is too slow, # because it sends O(n) Py4J commands. As an alternative, serialized # objects are written to a file and loaded through textFile(). - tempFile = NamedTemporaryFile(delete=False) - atexit.register(lambda: os.unlink(tempFile.name)) + tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) if self.batchSize != 1: c = batched(c, self.batchSize) for x in c: @@ -247,7 +249,9 @@ def setCheckpointDir(self, dirName, useExisting=False): def _test(): + import atexit import doctest + import tempfile globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) globs['tempdir'] = tempfile.mkdtemp() diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d7cad2f3725aa..41ea6e6e14c07 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1,4 +1,3 @@ -import atexit from base64 import standard_b64encode as b64enc import copy from collections import defaultdict @@ -264,12 +263,8 @@ def _collect_iterator_through_file(self, iterator): # Transferring lots of data through Py4J can be slow because # socket.readline() is inefficient. Instead, we'll dump the data to a # file and read it back. - tempFile = NamedTemporaryFile(delete=False) + tempFile = NamedTemporaryFile(delete=False, dir=self.ctx._temp_dir) tempFile.close() - def clean_up_file(): - try: os.unlink(tempFile.name) - except: pass - atexit.register(clean_up_file) self.ctx._writeIteratorToPickleFile(iterator, tempFile.name) # Read the data into Python and deserialize it: with open(tempFile.name, 'rb') as tempFile: From 9970926ede0d5a719b8f22e97977804d3c811e97 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 1 Feb 2013 14:07:34 -0800 Subject: [PATCH 136/696] formatting --- core/src/main/scala/spark/RDD.scala | 2 +- core/src/main/scala/spark/scheduler/ShuffleMapTask.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 210404d540cf3..010e61dfdcb25 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -385,7 +385,7 @@ abstract class RDD[T: ClassManifest]( val reducePartition: Iterator[T] => Option[T] = iter => { if (iter.hasNext) { Some(iter.reduceLeft(cleanF)) - }else { + } else { None } } diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 83641a2a8427b..20f2c9e489d60 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -32,7 +32,7 @@ private[spark] object ShuffleMapTask { return old } else { val out = new ByteArrayOutputStream - val ser = SparkEnv.get.closureSerializer.newInstance + val ser = SparkEnv.get.closureSerializer.newInstance() val objOut = ser.serializeStream(new GZIPOutputStream(out)) objOut.writeObject(rdd) objOut.writeObject(dep) @@ -48,7 +48,7 @@ private[spark] object ShuffleMapTask { synchronized { val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val ser = SparkEnv.get.closureSerializer.newInstance + val ser = SparkEnv.get.closureSerializer.newInstance() val objIn = ser.deserializeStream(in) val rdd = objIn.readObject().asInstanceOf[RDD[_]] val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]] From 8b3041c7233011c4a96fab045a86df91eae7b6f3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 1 Feb 2013 15:38:42 -0800 Subject: [PATCH 137/696] Reduced the memory usage of reduce and similar operations These operations used to wait for all the results to be available in an array on the driver program before merging them. They now merge values incrementally as they arrive. --- .../main/scala/spark/PairRDDFunctions.scala | 4 +- core/src/main/scala/spark/RDD.scala | 41 ++++++++------ core/src/main/scala/spark/SparkContext.scala | 53 ++++++++++++++++--- core/src/main/scala/spark/Utils.scala | 8 +++ .../partial/ApproximateActionListener.scala | 4 +- .../scala/spark/scheduler/DAGScheduler.scala | 15 +++--- .../scala/spark/scheduler/JobResult.scala | 2 +- .../scala/spark/scheduler/JobWaiter.scala | 14 ++--- core/src/test/scala/spark/RDDSuite.scala | 12 +++-- 9 files changed, 107 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 231e23a7deacc..cc3cca2571359 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -465,7 +465,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val res = self.context.runJob(self, process _, Array(index), false) res(0) case None => - self.filter(_._1 == key).map(_._2).collect + self.filter(_._1 == key).map(_._2).collect() } } @@ -590,7 +590,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( var count = 0 while(iter.hasNext) { - val record = iter.next + val record = iter.next() count += 1 writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 010e61dfdcb25..9d6ea782bd83c 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -389,16 +389,18 @@ abstract class RDD[T: ClassManifest]( None } } - val options = sc.runJob(this, reducePartition) - val results = new ArrayBuffer[T] - for (opt <- options; elem <- opt) { - results += elem - } - if (results.size == 0) { - throw new UnsupportedOperationException("empty collection") - } else { - return results.reduceLeft(cleanF) + var jobResult: Option[T] = None + val mergeResult = (index: Int, taskResult: Option[T]) => { + if (taskResult != None) { + jobResult = jobResult match { + case Some(value) => Some(f(value, taskResult.get)) + case None => taskResult + } + } } + sc.runJob(this, reducePartition, mergeResult) + // Get the final result out of our Option, or throw an exception if the RDD was empty + jobResult.getOrElse(throw new UnsupportedOperationException("empty collection")) } /** @@ -408,9 +410,13 @@ abstract class RDD[T: ClassManifest]( * modify t2. */ def fold(zeroValue: T)(op: (T, T) => T): T = { + // Clone the zero value since we will also be serializing it as part of tasks + var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) val cleanOp = sc.clean(op) - val results = sc.runJob(this, (iter: Iterator[T]) => iter.fold(zeroValue)(cleanOp)) - return results.fold(zeroValue)(cleanOp) + val foldPartition = (iter: Iterator[T]) => iter.fold(zeroValue)(cleanOp) + val mergeResult = (index: Int, taskResult: T) => jobResult = op(jobResult, taskResult) + sc.runJob(this, foldPartition, mergeResult) + jobResult } /** @@ -422,11 +428,14 @@ abstract class RDD[T: ClassManifest]( * allocation. */ def aggregate[U: ClassManifest](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { + // Clone the zero value since we will also be serializing it as part of tasks + var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) val cleanSeqOp = sc.clean(seqOp) val cleanCombOp = sc.clean(combOp) - val results = sc.runJob(this, - (iter: Iterator[T]) => iter.aggregate(zeroValue)(cleanSeqOp, cleanCombOp)) - return results.fold(zeroValue)(cleanCombOp) + val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) + val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) + sc.runJob(this, aggregatePartition, mergeResult) + jobResult } /** @@ -437,7 +446,7 @@ abstract class RDD[T: ClassManifest]( var result = 0L while (iter.hasNext) { result += 1L - iter.next + iter.next() } result }).sum @@ -452,7 +461,7 @@ abstract class RDD[T: ClassManifest]( var result = 0L while (iter.hasNext) { result += 1L - iter.next + iter.next() } result } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b0d4b58240222..ddbf8f95d9390 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -543,26 +543,42 @@ class SparkContext( } /** - * Run a function on a given set of partitions in an RDD and return the results. This is the main - * entry point to the scheduler, by which all actions get launched. The allowLocal flag specifies - * whether the scheduler can run the computation on the driver rather than shipping it out to the - * cluster, for short actions like first(). + * Run a function on a given set of partitions in an RDD and pass the results to the given + * handler function. This is the main entry point for all actions in Spark. The allowLocal + * flag specifies whether the scheduler can run the computation on the driver rather than + * shipping it out to the cluster, for short actions like first(). */ def runJob[T, U: ClassManifest]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], - allowLocal: Boolean - ): Array[U] = { + allowLocal: Boolean, + resultHandler: (Int, U) => Unit) { val callSite = Utils.getSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal) + val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler) logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() result } + /** + * Run a function on a given set of partitions in an RDD and return the results as an array. The + * allowLocal flag specifies whether the scheduler can run the computation on the driver rather + * than shipping it out to the cluster, for short actions like first(). + */ + def runJob[T, U: ClassManifest]( + rdd: RDD[T], + func: (TaskContext, Iterator[T]) => U, + partitions: Seq[Int], + allowLocal: Boolean + ): Array[U] = { + val results = new Array[U](partitions.size) + runJob[T, U](rdd, func, partitions, allowLocal, (index, res) => results(index) = res) + results + } + /** * Run a job on a given set of partitions of an RDD, but take a function of type * `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`. @@ -590,6 +606,29 @@ class SparkContext( runJob(rdd, func, 0 until rdd.splits.size, false) } + /** + * Run a job on all partitions in an RDD and pass the results to a handler function. + */ + def runJob[T, U: ClassManifest]( + rdd: RDD[T], + processPartition: (TaskContext, Iterator[T]) => U, + resultHandler: (Int, U) => Unit) + { + runJob[T, U](rdd, processPartition, 0 until rdd.splits.size, false, resultHandler) + } + + /** + * Run a job on all partitions in an RDD and pass the results to a handler function. + */ + def runJob[T, U: ClassManifest]( + rdd: RDD[T], + processPartition: Iterator[T] => U, + resultHandler: (Int, U) => Unit) + { + val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter) + runJob[T, U](rdd, processFunc, 0 until rdd.splits.size, false, resultHandler) + } + /** * Run a job that can return approximate results. */ diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1e58d012731b5..28d643abca8f4 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -12,6 +12,7 @@ import scala.io.Source import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import scala.Some +import spark.serializer.SerializerInstance /** * Various utility methods used by Spark. @@ -446,4 +447,11 @@ private object Utils extends Logging { socket.close() portBound } + + /** + * Clone an object using a Spark serializer. + */ + def clone[T](value: T, serializer: SerializerInstance): T = { + serializer.deserialize[T](serializer.serialize(value)) + } } diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala index 42f46e06ed0f4..24b4909380c66 100644 --- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala @@ -32,7 +32,7 @@ private[spark] class ApproximateActionListener[T, U, R]( if (finishedTasks == totalTasks) { // If we had already returned a PartialResult, set its final value resultObject.foreach(r => r.setFinalValue(evaluator.currentResult())) - // Notify any waiting thread that may have called getResult + // Notify any waiting thread that may have called awaitResult this.notifyAll() } } @@ -49,7 +49,7 @@ private[spark] class ApproximateActionListener[T, U, R]( * Waits for up to timeout milliseconds since the listener was created and then returns a * PartialResult with the result so far. This may be complete if the whole job is done. */ - def getResult(): PartialResult[R] = synchronized { + def awaitResult(): PartialResult[R] = synchronized { val finishTime = startTime + timeout while (true) { val time = System.currentTimeMillis() diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 14f61f7e87bc8..908a22b2dfa0e 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -203,18 +203,17 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], callSite: String, - allowLocal: Boolean) - : Array[U] = + allowLocal: Boolean, + resultHandler: (Int, U) => Unit) { if (partitions.size == 0) { - return new Array[U](0) + return } - val waiter = new JobWaiter(partitions.size) + val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] eventQueue.put(JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter)) - waiter.getResult() match { - case JobSucceeded(results: Seq[_]) => - return results.asInstanceOf[Seq[U]].toArray + waiter.awaitResult() match { + case JobSucceeded => {} case JobFailed(exception: Exception) => logInfo("Failed to run " + callSite) throw exception @@ -233,7 +232,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.splits.size).toArray eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener)) - return listener.getResult() // Will throw an exception if the job fails + return listener.awaitResult() // Will throw an exception if the job fails } /** diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala index c4a74e526f003..654131ee8417b 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/spark/scheduler/JobResult.scala @@ -5,5 +5,5 @@ package spark.scheduler */ private[spark] sealed trait JobResult -private[spark] case class JobSucceeded(results: Seq[_]) extends JobResult +private[spark] case object JobSucceeded extends JobResult private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala index b3d4feebe5e62..3cc6a86345fd1 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala @@ -3,10 +3,12 @@ package spark.scheduler import scala.collection.mutable.ArrayBuffer /** - * An object that waits for a DAGScheduler job to complete. + * An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their + * results to the given handler function. */ -private[spark] class JobWaiter(totalTasks: Int) extends JobListener { - private val taskResults = ArrayBuffer.fill[Any](totalTasks)(null) +private[spark] class JobWaiter[T](totalTasks: Int, resultHandler: (Int, T) => Unit) + extends JobListener { + private var finishedTasks = 0 private var jobFinished = false // Is the job as a whole finished (succeeded or failed)? @@ -17,11 +19,11 @@ private[spark] class JobWaiter(totalTasks: Int) extends JobListener { if (jobFinished) { throw new UnsupportedOperationException("taskSucceeded() called on a finished JobWaiter") } - taskResults(index) = result + resultHandler(index, result.asInstanceOf[T]) finishedTasks += 1 if (finishedTasks == totalTasks) { jobFinished = true - jobResult = JobSucceeded(taskResults) + jobResult = JobSucceeded this.notifyAll() } } @@ -38,7 +40,7 @@ private[spark] class JobWaiter(totalTasks: Int) extends JobListener { } } - def getResult(): JobResult = synchronized { + def awaitResult(): JobResult = synchronized { while (!jobFinished) { this.wait() } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ed03e651537fd..95d2e627306f6 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -12,9 +12,9 @@ class RDDSuite extends FunSuite with LocalSparkContext { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) - assert(dups.distinct.count === 4) - assert(dups.distinct().collect === dups.distinct.collect) - assert(dups.distinct(2).collect === dups.distinct.collect) + assert(dups.distinct().count === 4) + assert(dups.distinct().collect === dups.distinct().collect) + assert(dups.distinct(2).collect === dups.distinct().collect) assert(nums.reduce(_ + _) === 10) assert(nums.fold(0)(_ + _) === 10) assert(nums.map(_.toString).collect().toList === List("1", "2", "3", "4")) @@ -31,6 +31,10 @@ class RDDSuite extends FunSuite with LocalSparkContext { case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) } assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7))) + + intercept[UnsupportedOperationException] { + nums.filter(_ > 5).reduce(_ + _) + } } test("SparkContext.union") { @@ -164,7 +168,7 @@ class RDDSuite extends FunSuite with LocalSparkContext { // Note that split number starts from 0, so > 8 means only 10th partition left. val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8) assert(prunedRdd.splits.size === 1) - val prunedData = prunedRdd.collect + val prunedData = prunedRdd.collect() assert(prunedData.size === 1) assert(prunedData(0) === 10) } From 12c1eb47568060efac57d6df7df7e5704a8d3fab Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 1 Feb 2013 21:21:44 -0600 Subject: [PATCH 138/696] Reduce the amount of duplicate logging Akka does to stdout. Given we have Akka logging go through SLF4j to log4j, we don't need all the extra noise of Akka's stdout logger that is supposedly only used during Akka init time but seems to continue logging lots of noisy network events that we either don't care about or are in the log4j logs anyway. See: http://doc.akka.io/docs/akka/2.0/general/configuration.html # Log level for the very basic logger activated during AkkaApplication startup # Options: ERROR, WARNING, INFO, DEBUG # stdout-loglevel = "WARNING" --- core/src/main/scala/spark/util/AkkaUtils.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e0fdeffbc44d8..e43fbd6b1cfa5 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -30,6 +30,7 @@ private[spark] object AkkaUtils { val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] + akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" akka.remote.log-remote-lifecycle-events = on From ae26911ec0d768dcdae8b7d706ca4544e36535e6 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 1 Feb 2013 21:07:24 -0800 Subject: [PATCH 139/696] Add back test for distinct without parens --- core/src/test/scala/spark/RDDSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 95d2e627306f6..89a3687386e73 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -12,7 +12,8 @@ class RDDSuite extends FunSuite with LocalSparkContext { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) - assert(dups.distinct().count === 4) + assert(dups.distinct().count() === 4) + assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? assert(dups.distinct().collect === dups.distinct().collect) assert(dups.distinct(2).collect === dups.distinct().collect) assert(nums.reduce(_ + _) === 10) From 1fd5ee323d127499bb3f173d4142c37532ec29b2 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Fri, 1 Feb 2013 22:33:38 -0800 Subject: [PATCH 140/696] Code review changes: add sc.stop; style of multiline comments; parens on procedure calls. --- .../spark/scheduler/DAGSchedulerSuite.scala | 69 +++++++++++++------ 1 file changed, 47 insertions(+), 22 deletions(-) diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index c31e2e7064715..adce1f38bb784 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -31,7 +31,7 @@ import spark.TaskEndReason import spark.{FetchFailed, Success} /** - * Tests for DAGScheduler. These tests directly call the event processing functinos in DAGScheduler + * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler * rather than spawning an event loop thread as happens in the real code. They use EasyMock * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead @@ -56,29 +56,34 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar var schedulerThread: Thread = null var schedulerException: Throwable = null - /** Set of EasyMock argument matchers that match a TaskSet for a given RDD. + /** + * Set of EasyMock argument matchers that match a TaskSet for a given RDD. * We cache these so we do not create duplicate matchers for the same RDD. * This allows us to easily setup a sequence of expectations for task sets for * that RDD. */ val taskSetMatchers = new HashMap[MyRDD, IArgumentMatcher] - /** Set of cache locations to return from our mock BlockManagerMaster. + /** + * Set of cache locations to return from our mock BlockManagerMaster. * Keys are (rdd ID, partition ID). Anything not present will return an empty * list of cache locations silently. */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] - /** JobWaiter for the last JobSubmitted event we pushed. To keep tests (most of which + /** + * JobWaiter for the last JobSubmitted event we pushed. To keep tests (most of which * will only submit one job) from needing to explicitly track it. */ var lastJobWaiter: JobWaiter = null - /** Tell EasyMockSugar what mock objects we want to be configured by expecting {...} + /** + * Tell EasyMockSugar what mock objects we want to be configured by expecting {...} * and whenExecuting {...} */ implicit val mocks = MockObjects(taskScheduler, blockManagerMaster) - /** Utility function to reset mocks and set expectations on them. EasyMock wants mock objects + /** + * Utility function to reset mocks and set expectations on them. EasyMock wants mock objects * to be reset after each time their expectations are set, and we tend to check mock object * calls over a single call to DAGScheduler. * @@ -115,17 +120,21 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting { scheduler.stop() } + sc.stop() System.clearProperty("spark.master.port") } def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) - /** Type of RDD we use for testing. Note that we should never call the real RDD compute methods. - * This is a pair RDD type so it can always be used in ShuffleDependencies. */ + /** + * Type of RDD we use for testing. Note that we should never call the real RDD compute methods. + * This is a pair RDD type so it can always be used in ShuffleDependencies. + */ type MyRDD = RDD[(Int, Int)] - /** Create an RDD for passing to DAGScheduler. These RDDs will use the dependencies and + /** + * Create an RDD for passing to DAGScheduler. These RDDs will use the dependencies and * preferredLocations (if any) that are passed to them. They are deliberately not executable * so we can test that DAGScheduler does not try to execute RDDs locally. */ @@ -150,7 +159,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } - /** EasyMock matcher method. For use as an argument matcher for a TaskSet whose first task + /** + * EasyMock matcher method. For use as an argument matcher for a TaskSet whose first task * is from a particular RDD. */ def taskSetForRdd(rdd: MyRDD): TaskSet = { @@ -172,7 +182,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar return null } - /** Setup an EasyMock expectation to repsond to blockManagerMaster.getLocations() called from + /** + * Setup an EasyMock expectation to repsond to blockManagerMaster.getLocations() called from * cacheLocations. */ def expectGetLocations(): Unit = { @@ -197,7 +208,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar }).anyTimes() } - /** Process the supplied event as if it were the top of the DAGScheduler event queue, expecting + /** + * Process the supplied event as if it were the top of the DAGScheduler event queue, expecting * the scheduler not to exit. * * After processing the event, submit waiting stages as is done on most iterations of the @@ -208,7 +220,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar scheduler.submitWaitingStages() } - /** Expect a TaskSet for the specified RDD to be submitted to the TaskScheduler. Should be + /** + * Expect a TaskSet for the specified RDD to be submitted to the TaskScheduler. Should be * called from a resetExpecting { ... } block. * * Returns a easymock Capture that will contain the task set after the stage is submitted. @@ -220,7 +233,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar return taskSetCapture } - /** Expect the supplied code snippet to submit a stage for the specified RDD. + /** + * Expect the supplied code snippet to submit a stage for the specified RDD. * Return the resulting TaskSet. First marks all the tasks are belonging to the * current MapOutputTracker generation. */ @@ -239,7 +253,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar return taskSet } - /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ + /** + * Send the given CompletionEvent messages for the tasks in the TaskSet. + */ def respondToTaskSet(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { @@ -249,7 +265,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } - /** Assert that the supplied TaskSet has exactly the given preferredLocations. */ + /** + * Assert that the supplied TaskSet has exactly the given preferredLocations. + */ def expectTaskSetLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { assert(locations.size === taskSet.tasks.size) for ((expectLocs, taskLocs) <- @@ -258,7 +276,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } - /** When we submit dummy Jobs, this is the compute function we supply. Except in a local test + /** + * When we submit dummy Jobs, this is the compute function we supply. Except in a local test * below, we do not expect this function to ever be executed; instead, we will return results * directly through CompletionEvents. */ @@ -266,8 +285,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar it.next._1.asInstanceOf[Int] - /** Start a job to compute the given RDD. Returns the JobWaiter that will - * collect the result of the job via callbacks from DAGScheduler. */ + /** + * Start a job to compute the given RDD. Returns the JobWaiter that will + * collect the result of the job via callbacks from DAGScheduler. + */ def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): JobWaiter = { val (toSubmit, waiter) = scheduler.prepareJob[(Int, Int), Int]( rdd, @@ -281,7 +302,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar return waiter } - /** Assert that a job we started has failed. */ + /** + * Assert that a job we started has failed. + */ def expectJobException(waiter: JobWaiter = lastJobWaiter) { waiter.getResult match { case JobSucceeded(_) => fail() @@ -289,7 +312,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } - /** Assert that a job we started has succeeded and has the given result. */ + /** + * Assert that a job we started has succeeded and has the given result. + */ def expectJobResult(expected: Array[Int], waiter: JobWaiter = lastJobWaiter) { waiter.getResult match { case JobSucceeded(answer) => @@ -500,7 +525,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar )) } val recomputeOne = interceptStage(shuffleOneRdd) { - scheduler.resubmitFailedStages + scheduler.resubmitFailedStages() } val recomputeTwo = interceptStage(shuffleTwoRdd) { respondToTaskSet(recomputeOne, List( From 28e0cb9f312b7fb1b0236fd15ba0dd2f423e826d Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 2 Feb 2013 01:11:37 -0600 Subject: [PATCH 141/696] Fix createActorSystem not actually using the systemName parameter. This meant all system names were "spark", which worked, but didn't lead to the most intuitive log output. This fixes createActorSystem to use the passed system name, and refactors Master/Worker to encapsulate their system/actor names instead of having the clients guess at them. Note that the driver system name, "spark", is left as is, and is still repeated a few times, but that seems like a separate issue. --- .../spark/deploy/LocalSparkCluster.scala | 38 +++++------- .../scala/spark/deploy/client/Client.scala | 13 +---- .../scala/spark/deploy/master/Master.scala | 24 +++++++- .../scala/spark/deploy/worker/Worker.scala | 58 +++++++++---------- .../spark/storage/BlockManagerMaster.scala | 2 - .../src/main/scala/spark/util/AkkaUtils.scala | 6 +- 6 files changed, 68 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 2836574ecb23b..22319a96caef7 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -18,35 +18,23 @@ import scala.collection.mutable.ArrayBuffer private[spark] class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging { - val localIpAddress = Utils.localIpAddress + private val localIpAddress = Utils.localIpAddress + private val masterActorSystems = ArrayBuffer[ActorSystem]() + private val workerActorSystems = ArrayBuffer[ActorSystem]() - var masterActor : ActorRef = _ - var masterActorSystem : ActorSystem = _ - var masterPort : Int = _ - var masterUrl : String = _ - - val workerActorSystems = ArrayBuffer[ActorSystem]() - val workerActors = ArrayBuffer[ActorRef]() - - def start() : String = { + def start(): String = { logInfo("Starting a local Spark cluster with " + numWorkers + " workers.") /* Start the Master */ - val (actorSystem, masterPort) = AkkaUtils.createActorSystem("sparkMaster", localIpAddress, 0) - masterActorSystem = actorSystem - masterUrl = "spark://" + localIpAddress + ":" + masterPort - masterActor = masterActorSystem.actorOf( - Props(new Master(localIpAddress, masterPort, 0)), name = "Master") + val (masterSystem, masterPort) = Master.startSystemAndActor(localIpAddress, 0, 0) + masterActorSystems += masterSystem + val masterUrl = "spark://" + localIpAddress + ":" + masterPort - /* Start the Slaves */ + /* Start the Workers */ for (workerNum <- 1 to numWorkers) { - val (actorSystem, boundPort) = - AkkaUtils.createActorSystem("sparkWorker" + workerNum, localIpAddress, 0) - workerActorSystems += actorSystem - val actor = actorSystem.actorOf( - Props(new Worker(localIpAddress, boundPort, 0, coresPerWorker, memoryPerWorker, masterUrl)), - name = "Worker") - workerActors += actor + val (workerSystem, _) = Worker.startSystemAndActor(localIpAddress, 0, 0, coresPerWorker, + memoryPerWorker, masterUrl, null, Some(workerNum)) + workerActorSystems += workerSystem } return masterUrl @@ -57,7 +45,7 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I // Stop the workers before the master so they don't get upset that it disconnected workerActorSystems.foreach(_.shutdown()) workerActorSystems.foreach(_.awaitTermination()) - masterActorSystem.shutdown() - masterActorSystem.awaitTermination() + masterActorSystems.foreach(_.shutdown()) + masterActorSystems.foreach(_.awaitTermination()) } } diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 90fe9508cdba9..a63eee12339d1 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -9,6 +9,7 @@ import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown import spark.deploy.RegisterJob +import spark.deploy.master.Master import akka.remote.RemoteClientDisconnected import akka.actor.Terminated import akka.dispatch.Await @@ -24,26 +25,18 @@ private[spark] class Client( listener: ClientListener) extends Logging { - val MASTER_REGEX = "spark://([^:]+):([0-9]+)".r - var actor: ActorRef = null var jobId: String = null - if (MASTER_REGEX.unapplySeq(masterUrl) == None) { - throw new SparkException("Invalid master URL: " + masterUrl) - } - class ClientActor extends Actor with Logging { var master: ActorRef = null var masterAddress: Address = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times override def preStart() { - val Seq(masterHost, masterPort) = MASTER_REGEX.unapplySeq(masterUrl).get - logInfo("Connecting to master spark://" + masterHost + ":" + masterPort) - val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort) + logInfo("Connecting to master " + masterUrl) try { - master = context.actorFor(akkaUrl) + master = context.actorFor(Master.toAkkaUrl(masterUrl)) masterAddress = master.path.address master ! RegisterJob(jobDescription) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index c618e87cdd2b1..92e7914b1b931 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -262,11 +262,29 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } private[spark] object Master { + private val systemName = "sparkMaster" + private val actorName = "Master" + private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r + def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port) - val actor = actorSystem.actorOf( - Props(new Master(args.ip, boundPort, args.webUiPort)), name = "Master") + val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort) actorSystem.awaitTermination() } + + /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ + def toAkkaUrl(sparkUrl: String): String = { + sparkUrl match { + case sparkUrlRegex(host, port) => + "akka://%s@%s:%s/user/%s".format(systemName, host, port, actorName) + case _ => + throw new SparkException("Invalid master URL: " + sparkUrl) + } + } + + def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int) = { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName) + (actorSystem, boundPort) + } } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 8b41620d98232..2219dd6262f3a 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -1,7 +1,7 @@ package spark.deploy.worker import scala.collection.mutable.{ArrayBuffer, HashMap} -import akka.actor.{ActorRef, Props, Actor} +import akka.actor.{ActorRef, Props, Actor, ActorSystem} import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ @@ -13,6 +13,7 @@ import akka.remote.RemoteClientDisconnected import spark.deploy.RegisterWorker import spark.deploy.LaunchExecutor import spark.deploy.RegisterWorkerFailed +import spark.deploy.master.Master import akka.actor.Terminated import java.io.File @@ -27,7 +28,6 @@ private[spark] class Worker( extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs - val MASTER_REGEX = "spark://([^:]+):([0-9]+)".r var master: ActorRef = null var masterWebUiUrl : String = "" @@ -48,11 +48,7 @@ private[spark] class Worker( def memoryFree: Int = memory - memoryUsed def createWorkDir() { - workDir = if (workDirPath != null) { - new File(workDirPath) - } else { - new File(sparkHome, "work") - } + workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) try { if (!workDir.exists() && !workDir.mkdirs()) { logError("Failed to create work directory " + workDir) @@ -68,8 +64,7 @@ private[spark] class Worker( override def preStart() { logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format( ip, port, cores, Utils.memoryMegabytesToString(memory))) - val envVar = System.getenv("SPARK_HOME") - sparkHome = new File(if (envVar == null) "." else envVar) + sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() connectToMaster() @@ -77,24 +72,15 @@ private[spark] class Worker( } def connectToMaster() { - masterUrl match { - case MASTER_REGEX(masterHost, masterPort) => { - logInfo("Connecting to master spark://" + masterHost + ":" + masterPort) - val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort) - try { - master = context.actorFor(akkaUrl) - master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing - } catch { - case e: Exception => - logError("Failed to connect to master", e) - System.exit(1) - } - } - - case _ => - logError("Invalid master URL: " + masterUrl) + logInfo("Connecting to master " + masterUrl) + try { + master = context.actorFor(Master.toAkkaUrl(masterUrl)) + master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(master) // Doesn't work with remote actors, but useful for testing + } catch { + case e: Exception => + logError("Failed to connect to master", e) System.exit(1) } } @@ -183,11 +169,19 @@ private[spark] class Worker( private[spark] object Worker { def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port) - val actor = actorSystem.actorOf( - Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory, - args.master, args.workDir)), - name = "Worker") + val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort, args.cores, + args.memory, args.master, args.workDir) actorSystem.awaitTermination() } + + def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int, + masterUrl: String, workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = { + // The LocalSparkCluster runs multiple local sparkWorkerX actor systems + val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val actor = actorSystem.actorOf(Props(new Worker(host, boundPort, webUiPort, cores, memory, + masterUrl, workDir)), name = "Worker") + (actorSystem, boundPort) + } + } diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 36398095a2e8a..7be6b9fa87999 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -27,8 +27,6 @@ private[spark] class BlockManagerMaster( val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockMasterManager" - val SLAVE_AKKA_ACTOR_NAME = "BlockSlaveManager" - val DEFAULT_MANAGER_IP: String = Utils.localHostName() val timeout = 10.seconds var driverActor: ActorRef = { diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e0fdeffbc44d8..3a3626e8a003c 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -18,9 +18,13 @@ import java.util.concurrent.TimeoutException * Various utility classes for working with Akka. */ private[spark] object AkkaUtils { + /** * Creates an ActorSystem ready for remoting, with various Spark features. Returns both the * ActorSystem itself and its port (which is hard to get from Akka). + * + * Note: the `name` parameter is important, as even if a client sends a message to right + * host + port, if the system name is incorrect, Akka will drop the message. */ def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt @@ -41,7 +45,7 @@ private[spark] object AkkaUtils { akka.actor.default-dispatcher.throughput = %d """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize)) - val actorSystem = ActorSystem("spark", akkaConf, getClass.getClassLoader) + val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader) // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. From 696eec32c982ca516c506de33f383a173bcbd131 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 2 Feb 2013 02:03:26 -0600 Subject: [PATCH 142/696] Move executorMemory up into SchedulerBackend. --- .../spark/scheduler/cluster/SchedulerBackend.scala | 12 ++++++++++++ .../cluster/SparkDeploySchedulerBackend.scala | 9 --------- .../mesos/CoarseMesosSchedulerBackend.scala | 10 ---------- .../scheduler/mesos/MesosSchedulerBackend.scala | 10 ---------- 4 files changed, 12 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index ddcd64d7c6d1f..9ac875de3a16a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -1,5 +1,7 @@ package spark.scheduler.cluster +import spark.Utils + /** * A backend interface for cluster scheduling systems that allows plugging in different ones under * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as @@ -11,5 +13,15 @@ private[spark] trait SchedulerBackend { def reviveOffers(): Unit def defaultParallelism(): Int + // Memory used by each executor (in megabytes) + protected val executorMemory = { + // TODO: Might need to add some extra memory for the non-heap parts of the JVM + Option(System.getProperty("spark.executor.memory")) + .orElse(Option(System.getenv("SPARK_MEM"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + } + + // TODO: Probably want to add a killTask too } diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 2f7099c5b9cb7..59ff8bcb90fc2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -20,15 +20,6 @@ private[spark] class SparkDeploySchedulerBackend( val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt - // Memory used by each executor (in megabytes) - val executorMemory = { - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - Option(System.getProperty("spark.executor.memory")) - .orElse(Option(System.getenv("SPARK_MEM"))) - .map(Utils.memoryStringToMb) - .getOrElse(512) - } - override def start() { super.start() diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 7bf56a05d64c0..b481ec0a72dce 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -35,16 +35,6 @@ private[spark] class CoarseMesosSchedulerBackend( val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures - // Memory used by each executor (in megabytes) - val executorMemory = { - if (System.getenv("SPARK_MEM") != null) { - Utils.memoryStringToMb(System.getenv("SPARK_MEM")) - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - } else { - 512 - } - } - // Lock used to wait for scheduler to be registered var isRegistered = false val registeredLock = new Object() diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index eab1c60e0b2f9..5c8b531de3731 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -29,16 +29,6 @@ private[spark] class MesosSchedulerBackend( with MScheduler with Logging { - // Memory used by each executor (in megabytes) - val EXECUTOR_MEMORY = { - if (System.getenv("SPARK_MEM") != null) { - Utils.memoryStringToMb(System.getenv("SPARK_MEM")) - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - } else { - 512 - } - } - // Lock used to wait for scheduler to be registered var isRegistered = false val registeredLock = new Object() From cae8a6795c7f454b74c8d3c4425a6ced151d6d9b Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 2 Feb 2013 02:15:39 -0600 Subject: [PATCH 143/696] Fix dangling old variable names. --- .../scala/spark/scheduler/mesos/MesosSchedulerBackend.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 5c8b531de3731..300766d0f5c72 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -79,7 +79,7 @@ private[spark] class MesosSchedulerBackend( val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(EXECUTOR_MEMORY).build()) + .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build()) .build() val command = CommandInfo.newBuilder() .setValue(execScript) @@ -151,7 +151,7 @@ private[spark] class MesosSchedulerBackend( def enoughMemory(o: Offer) = { val mem = getResource(o.getResourcesList, "mem") val slaveId = o.getSlaveId.getValue - mem >= EXECUTOR_MEMORY || slaveIdsWithExecutors.contains(slaveId) + mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) } for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { From 7aba123f0c0fd024105462b3a0b203cd357c67e9 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 2 Feb 2013 13:53:28 -0600 Subject: [PATCH 144/696] Further simplify checking for Nil. --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b62b25f6880a7..2a646dd0f5f6d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -176,9 +176,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with def visit(rdd: RDD[_]) { if (!visited(rdd)) { visited += rdd - val locs = getCacheLocs(rdd) - val atLeastOneMissing = (0 until rdd.splits.size).exists(locs(_) == Nil) - if (atLeastOneMissing) { + if (getCacheLocs(rdd).contains(Nil)) { for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => From 34a7bcdb3a19deed18b25225daf47ff22ee20869 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 2 Feb 2013 19:40:30 -0800 Subject: [PATCH 145/696] Formatting --- .../main/scala/spark/scheduler/DAGScheduler.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 8cfc08e5acac3..2a35915560f39 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -205,8 +205,9 @@ class DAGScheduler( missing.toList } - /** Returns (and does not) submit a JobSubmitted event suitable to run a given job, and - * a JobWaiter whose getResult() method will return the result of the job when it is complete. + /** + * Returns (and does not submit) a JobSubmitted event suitable to run a given job, and a + * JobWaiter whose getResult() method will return the result of the job when it is complete. * * The job is assumed to have at least one partition; zero partition jobs should be handled * without a JobSubmitted event. @@ -308,7 +309,8 @@ class DAGScheduler( return false } - /** Resubmit any failed stages. Ordinarily called after a small amount of time has passed since + /** + * Resubmit any failed stages. Ordinarily called after a small amount of time has passed since * the last fetch failure. */ private[scheduler] def resubmitFailedStages() { @@ -321,7 +323,8 @@ class DAGScheduler( } } - /** Check for waiting or failed stages which are now eligible for resubmission. + /** + * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. */ private[scheduler] def submitWaitingStages() { @@ -366,9 +369,9 @@ class DAGScheduler( // the same time, so we want to make sure we've identified all the reduce tasks that depend // on the failed node. if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) { - resubmitFailedStages + resubmitFailedStages() } else { - submitWaitingStages + submitWaitingStages() } } } From 2415c18f48fc28d88f29b88c312f98054f530f20 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 3 Feb 2013 06:44:11 +0000 Subject: [PATCH 146/696] Fix reporting of PySpark doctest failures. --- python/pyspark/context.py | 4 +++- python/pyspark/rdd.py | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6831f9b7f8b95..657fe6f98975b 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -256,8 +256,10 @@ def _test(): globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) globs['tempdir'] = tempfile.mkdtemp() atexit.register(lambda: shutil.rmtree(globs['tempdir'])) - doctest.testmod(globs=globs) + (failure_count, test_count) = doctest.testmod(globs=globs) globs['sc'].stop() + if failure_count: + exit(-1) if __name__ == "__main__": diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 41ea6e6e14c07..fb144bc45d271 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -748,8 +748,10 @@ def _test(): # The small batch size here ensures that we see multiple batches, # even in these small test examples: globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - doctest.testmod(globs=globs) + (failure_count, test_count) = doctest.testmod(globs=globs) globs['sc'].stop() + if failure_count: + exit(-1) if __name__ == "__main__": From 8fbd5380b7f36842297f624bad3a2513f7eca47b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 3 Feb 2013 06:44:49 +0000 Subject: [PATCH 147/696] Fetch fewer objects in PySpark's take() method. --- core/src/main/scala/spark/api/python/PythonRDD.scala | 11 +++++++++-- python/pyspark/rdd.py | 4 ++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 39758e94f46fe..ab8351e55e9ef 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -238,6 +238,11 @@ private[spark] object PythonRDD { } def writeIteratorToPickleFile[T](items: java.util.Iterator[T], filename: String) { + import scala.collection.JavaConverters._ + writeIteratorToPickleFile(items.asScala, filename) + } + + def writeIteratorToPickleFile[T](items: Iterator[T], filename: String) { val file = new DataOutputStream(new FileOutputStream(filename)) for (item <- items) { writeAsPickle(item, file) @@ -245,8 +250,10 @@ private[spark] object PythonRDD { file.close() } - def takePartition[T](rdd: RDD[T], partition: Int): java.util.Iterator[T] = - rdd.context.runJob(rdd, ((x: Iterator[T]) => x), Seq(partition), true).head + def takePartition[T](rdd: RDD[T], partition: Int): Iterator[T] = { + implicit val cm : ClassManifest[T] = rdd.elementClassManifest + rdd.context.runJob(rdd, ((x: Iterator[T]) => x.toArray), Seq(partition), true).head.iterator + } } private object Pickle { diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index fb144bc45d271..4cda6cf661197 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -372,6 +372,10 @@ def take(self, num): items = [] for partition in range(self._jrdd.splits().size()): iterator = self.ctx._takePartition(self._jrdd.rdd(), partition) + # Each item in the iterator is a string, Python object, batch of + # Python objects. Regardless, it is sufficient to take `num` + # of these objects in order to collect `num` Python objects: + iterator = iterator.take(num) items.extend(self._collect_iterator_through_file(iterator)) if len(items) >= num: break From 9163c3705d98ca19c09fe5618e347b9d20f88f63 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 2 Feb 2013 23:34:47 -0800 Subject: [PATCH 148/696] Formatting --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 2a35915560f39..edbfd1c45f500 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -265,7 +265,8 @@ class DAGScheduler( return listener.awaitResult() // Will throw an exception if the job fails } - /** Process one event retrieved from the event queue. + /** + * Process one event retrieved from the event queue. * Returns true if we should stop the event loop. */ private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { From 667860448ad5f705dd7548263cf7f240def25d87 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 2 Feb 2013 22:21:36 -0800 Subject: [PATCH 149/696] Starvation check in ClusterScheduler --- .../scheduler/cluster/ClusterScheduler.scala | 33 ++++++++++++++++++- .../scheduler/cluster/TaskSetManager.scala | 9 +++++ 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 1e4fbdb8742fd..aed9826377ddf 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -22,6 +22,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // How often to check for speculative tasks val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong + // How often to check for starved TaskSets + val STARVATION_CHECK_INTERVAL = System.getProperty("spark.starvation_check.interval", "5000").toLong val activeTaskSets = new HashMap[String, TaskSetManager] var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -84,6 +86,21 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } }.start() } + + new Thread("ClusterScheduler starvation check") { + setDaemon(true) + + override def run() { + while (true) { + try { + Thread.sleep(STARVATION_CHECK_INTERVAL) + } catch { + case e: InterruptedException => {} + } + detectStarvedTaskSets() + } + } + }.start() } override def submitTasks(taskSet: TaskSet) { @@ -235,7 +252,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } override def defaultParallelism() = backend.defaultParallelism() - + // Check for speculatable tasks in all our active jobs. def checkSpeculatableTasks() { var shouldRevive = false @@ -249,6 +266,20 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } + // Find and resource-starved TaskSets and alert the user + def detectStarvedTaskSets() { + val noOfferThresholdSeconds = 5 + synchronized { + for (ts <- activeTaskSetsQueue) { + if (ts == TaskSetManager.firstTaskSet.get && + (System.currentTimeMillis - ts.creationTime > noOfferThresholdSeconds * 1000) && + ts.receivedOffers == 0) { + logWarning("No offers received. Check the scheduler UI to ensure slaves are registered.") + } + } + } + } + def executorLost(executorId: String, reason: ExecutorLossReason) { var failedExecutor: Option[String] = None synchronized { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 3dabdd76b1aa5..58c5d4553e7ce 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -43,6 +43,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val numFailures = new Array[Int](numTasks) val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 + val creationTime = System.currentTimeMillis + var receivedOffers = 0 // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -96,6 +98,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } + if (!TaskSetManager.firstTaskSet.isDefined) TaskSetManager.firstTaskSet = Some(this) + // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive @@ -188,6 +192,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Respond to an offer of a single slave from the scheduler by finding a task def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + receivedOffers += 1 if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -427,3 +432,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return foundTasks } } + +object TaskSetManager { + var firstTaskSet: Option[TaskSetManager] = None +} From b14322956cbf268b0c880f17188af24ba4884d5b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 3 Feb 2013 12:17:20 -0800 Subject: [PATCH 150/696] Starvation check in Standlone scheduler --- core/src/main/scala/spark/deploy/master/Master.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index c618e87cdd2b1..8513dcefa0746 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -31,6 +31,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor val waitingJobs = new ArrayBuffer[JobInfo] val completedJobs = new ArrayBuffer[JobInfo] + var firstJob: Option[JobInfo] = None + val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else ip @@ -191,6 +193,11 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } } } + if (workers.toArray.filter(_.state == WorkerState.ALIVE).size > 0 && + firstJob.isDefined && + firstJob.get.executors.size == 0) { + logWarning("Could not find any machines with enough memory. Ensure that SPARK_WORKER_MEM > SPARK_MEM.") + } } def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) { @@ -232,6 +239,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor idToJob(job.id) = job actorToJob(driver) = job addressToJob(driver.path.address) = job + if (!firstJob.isDefined) firstJob = Some(job) return job } From e61729113d3bf165d1ab9bd83ea55d52fd0bb72e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 3 Feb 2013 21:29:40 -0800 Subject: [PATCH 151/696] Remove unnecessary doctest __main__ methods. --- python/pyspark/accumulators.py | 9 --------- python/pyspark/broadcast.py | 9 --------- 2 files changed, 18 deletions(-) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 61fcbbd37679f..3e9d7d36da8a1 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -196,12 +196,3 @@ def _start_update_server(): thread.daemon = True thread.start() return server - - -def _test(): - import doctest - doctest.testmod() - - -if __name__ == "__main__": - _test() diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 93876fa738f63..def810dd461da 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -37,12 +37,3 @@ def __init__(self, bid, value, java_broadcast=None, pickle_registry=None): def __reduce__(self): self._pickle_registry.add(self) return (_from_id, (self.bid, )) - - -def _test(): - import doctest - doctest.testmod() - - -if __name__ == "__main__": - _test() From aa4ee1e9e5485c1b96474e704c76225a2b8a7da9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 4 Feb 2013 11:06:31 -0800 Subject: [PATCH 152/696] Fix failing test --- core/src/test/scala/spark/MapOutputTrackerSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index f4e7ec39fe301..dd19442dcb4ad 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -79,8 +79,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { try { System.clearProperty("spark.driver.host") // In case some previous test had set it - val (actorSystem, boundPort) = - AkkaUtils.createActorSystem("test", "localhost", 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) System.setProperty("spark.driver.port", boundPort.toString) val masterTracker = new MapOutputTracker(actorSystem, true) val slaveTracker = new MapOutputTracker(actorSystem, false) From f6ec547ea7b56ee607a4c2a69206f8952318eaf1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 4 Feb 2013 13:14:54 -0800 Subject: [PATCH 153/696] Small fix to test for distinct --- core/src/test/scala/spark/RDDSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 89a3687386e73..fe7deb10d63b0 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -14,7 +14,7 @@ class RDDSuite extends FunSuite with LocalSparkContext { val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? - assert(dups.distinct().collect === dups.distinct().collect) + assert(dups.distinct.collect === dups.distinct().collect) assert(dups.distinct(2).collect === dups.distinct().collect) assert(nums.reduce(_ + _) === 10) assert(nums.fold(0)(_ + _) === 10) From cc37601ecb72abd1351ed73b3be1fb517a31a4e1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 4 Feb 2013 14:15:50 -0800 Subject: [PATCH 154/696] Adding an example with an OLAP roll-up --- .../main/scala/spark/examples/OLAPQuery.scala | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 examples/src/main/scala/spark/examples/OLAPQuery.scala diff --git a/examples/src/main/scala/spark/examples/OLAPQuery.scala b/examples/src/main/scala/spark/examples/OLAPQuery.scala new file mode 100644 index 0000000000000..ff3af01b1734b --- /dev/null +++ b/examples/src/main/scala/spark/examples/OLAPQuery.scala @@ -0,0 +1,66 @@ +package spark.examples + +import spark.SparkContext +import spark.SparkContext._ +/** + * Executes a roll up-style query against Apache logs. + */ +object OLAPQuery { + val exampleApacheLogs = List( + """10.10.10.10 - "FRED" [18/Jan/2013:17:56:07 +1100] "GET http://images.com/2013/Generic.jpg + | HTTP/1.1" 304 315 "http://referall.com/" "Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; + | GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR + | 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR + | 3.5.30729; Release=ARP)" "UD-1" - "image/jpeg" "whatever" 0.350 "-" - "" 265 923 934 "" + | 62.24.11.25 images.com 1358492167 - Whatup""".stripMargin.replace("\n", ""), + """10.10.10.10 - "FRED" [18/Jan/2013:18:02:37 +1100] "GET http://images.com/2013/Generic.jpg + | HTTP/1.1" 304 306 "http:/referall.com" "Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; + | GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR + | 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR + | 3.5.30729; Release=ARP)" "UD-1" - "image/jpeg" "whatever" 0.352 "-" - "" 256 977 988 "" + | 0 73.23.2.15 images.com 1358492557 - Whatup""".stripMargin.replace("\n", "") + ) + + def main(args: Array[String]) { + if (args.length == 0) { + System.err.println("Usage: OLAPQuery [logFile]") + System.exit(1) + } + val sc = new SparkContext(args(0), "OLAP Query") + + val dataSet = + if (args.length == 2) sc.textFile(args(1)) + else sc.parallelize(exampleApacheLogs) + + val apache_log_regex = + """^([\d.]+) (\S+) (\S+) \[([\w\d:/]+\s[+\-]\d{4})\] "(.+?)" (\d{3}) ([\d\-]+) "([^"]+)" "([^"]+)".*""".r + + /** Tracks the total query count and number of aggregate bytes for a particular group. */ + class Stats(val count: Int, val numBytes: Int) extends Serializable { + def merge(other: Stats) = new Stats(count + other.count, numBytes + other.numBytes) + override def toString = "bytes=%s\tn=%s".format(numBytes, count) + } + + def extractKey(line: String): (String, String, String) = { + apache_log_regex findFirstIn line match { + case Some(apache_log_regex(ip, _, user, dateTime, query, status, bytes, referer, ua)) => + if (user != "\"-\"") (ip, user, query) + else (null, null, null) + case _ => (null, null, null) + } + } + + def extractStats(line: String): Stats = { + apache_log_regex findFirstIn line match { + case Some(apache_log_regex(ip, _, user, dateTime, query, status, bytes, referer, ua)) => + new Stats(1, bytes.toInt) + case _ => new Stats(1, 0) + } + } + + dataSet.map(line => (extractKey(line), extractStats(line))) + .reduceByKey((a, b) => a.merge(b)) + .collect().foreach{ + case (user, query) => println("%s\t%s".format(user, query))} + } +} From 843084d69d1236c95d2b354c22b66dc5d9855406 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:15:28 -0800 Subject: [PATCH 155/696] track total bytes written by ShuffleMapTasks --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 5 ++++- core/src/main/scala/spark/scheduler/ShuffleMapTask.scala | 7 +++++++ core/src/main/scala/spark/scheduler/StageInfo.scala | 7 ++++++- 3 files changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 05aee790b471b..c5442492418bc 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -511,8 +511,11 @@ class DAGScheduler( } case smt: ShuffleMapTask => - val stage = idToStage(smt.stageId) val status = event.result.asInstanceOf[MapStatus] + smt.totalBytesWritten match { + case Some(b) => stageToInfos(stage).shuffleBytesWritten += b + case None => throw new RuntimeException("shuffle stask completed without tracking bytes written") + } val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index bed9f1864fffd..ef56b4318cf55 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -81,6 +81,9 @@ private[spark] class ShuffleMapTask( with Externalizable with Logging { + + var totalBytesWritten : Option[Long] = None + protected def this() = this(0, null, null, 0, null) var split = if (rdd == null) { @@ -130,14 +133,18 @@ private[spark] class ShuffleMapTask( val compressedSizes = new Array[Byte](numOutputSplits) + var totalBytes = 0l + val blockManager = SparkEnv.get.blockManager for (i <- 0 until numOutputSplits) { val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i // Get a Scala iterator from Java map val iter: Iterator[(Any, Any)] = buckets(i).iterator val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) + totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) } + totalBytesWritten = Some(totalBytes) return new MapStatus(blockManager.blockManagerId, compressedSizes) } finally { diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 21c6d1aee0064..489b9744dead2 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -4,7 +4,12 @@ import cluster.TaskInfo import collection._ import spark.util.Distribution -case class StageInfo(val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo]()) { +case class StageInfo( + val stage: Stage, + val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), + val shuffleBytesWritten : mutable.Buffer[Long] = mutable.Buffer[Long](), + val shuffleBytesRead : mutable.Buffer[Long] = mutable.Buffer[Long]() +) { def name = stage.rdd.name + "(" + stage.origin + ")" From 1ad77c4766313c479e89656928265f2016b0ebcb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:13:08 -0800 Subject: [PATCH 156/696] add TimedIterator --- .../main/scala/spark/util/TimedIterator.scala | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 core/src/main/scala/spark/util/TimedIterator.scala diff --git a/core/src/main/scala/spark/util/TimedIterator.scala b/core/src/main/scala/spark/util/TimedIterator.scala new file mode 100644 index 0000000000000..539b01f4ce47d --- /dev/null +++ b/core/src/main/scala/spark/util/TimedIterator.scala @@ -0,0 +1,32 @@ +package spark.util + +/** + * A utility for tracking the total time an iterator takes to iterate through its elements. + * + * In general, this should only be used if you expect it to take a considerable amount of time + * (eg. milliseconds) to get each element -- otherwise, the timing won't be very accurate, + * and you are probably just adding more overhead + */ +class TimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] { + private var netMillis = 0l + private var nElems = 0 + def hasNext = { + val start = System.currentTimeMillis() + val r = sub.hasNext + val end = System.currentTimeMillis() + netMillis += (end - start) + r + } + def next = { + val start = System.currentTimeMillis() + val r = sub.next + val end = System.currentTimeMillis() + netMillis += (end - start) + nElems += 1 + r + } + + def getNetMillis = netMillis + def getAverageTimePerItem = netMillis / nElems.toDouble + +} From 9df7e2ae55574f9058e26bd06ad976b6e1426647 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:14:00 -0800 Subject: [PATCH 157/696] Shuffle Fetchers use a timed iterator --- core/src/main/scala/spark/BlockStoreShuffleFetcher.scala | 3 ++- core/src/main/scala/spark/ShuffleFetcher.scala | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 86432d0127e3e..cab1975f63863 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import spark.storage.BlockManagerId +import util.TimedIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int) = { @@ -45,6 +46,6 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } } - blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock) + new TimedIterator(blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock)) } } diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index d9a94d4021ee3..237faa0a6ee2a 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,11 +1,13 @@ package spark +import util.TimedIterator + private[spark] abstract class ShuffleFetcher { /** * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int) : Iterator[(K, V)] + def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K, V)] /** Stop the fetcher */ def stop() {} From 295b53439819b3010293f021e82bf0192db04beb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:17:15 -0800 Subject: [PATCH 158/696] task context keeps a handle on Task -- giant hack, temporary for tracking shuffle times & amount --- core/src/main/scala/spark/TaskContext.scala | 12 +++++++----- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- core/src/main/scala/spark/rdd/HadoopRDD.scala | 2 +- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 2 +- .../main/scala/spark/scheduler/DAGScheduler.scala | 2 +- core/src/main/scala/spark/scheduler/ResultTask.scala | 2 +- .../main/scala/spark/scheduler/ShuffleMapTask.scala | 2 +- .../scala/spark/scheduler/TaskContextSuite.scala | 2 +- 8 files changed, 14 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index eab85f85a262b..11819db3a3814 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,19 +1,21 @@ package spark import scala.collection.mutable.ArrayBuffer +import scheduler.Task +class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val task: Task[_]) extends Serializable { + //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to + // be separate -class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long) extends Serializable { - - @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit] + @transient val onCompleteCallbacks = new ArrayBuffer[TaskContext => Unit] // Add a callback function to be executed on task completion. An example use // is for HadoopRDD to register a callback to close the input stream. - def addOnCompleteCallback(f: () => Unit) { + def addOnCompleteCallback(f: TaskContext => Unit) { onCompleteCallbacks += f } def executeOnCompleteCallbacks() { - onCompleteCallbacks.foreach{_()} + onCompleteCallbacks.foreach{_.apply(this)} } } diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 96b593ba7ca6d..ad085585904ec 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -104,7 +104,7 @@ private[spark] object CheckpointRDD extends Logging { val deserializeStream = serializer.deserializeStream(fileInputStream) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => deserializeStream.close()) + context.addOnCompleteCallback(_ => deserializeStream.close()) deserializeStream.asIterator.asInstanceOf[Iterator[T]] } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index f547f53812661..3f0b4ce23e9d5 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -74,7 +74,7 @@ class HadoopRDD[K, V]( reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => reader.close()) + context.addOnCompleteCallback(_ => reader.close()) val key: K = reader.createKey() val value: V = reader.createValue() diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index c3b155fcbddd6..c80d30e12542c 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -63,7 +63,7 @@ class NewHadoopRDD[K, V]( reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => reader.close()) + context.addOnCompleteCallback(_ => reader.close()) var havePair = false var finished = false diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c5442492418bc..f07ee50670408 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -394,7 +394,7 @@ class DAGScheduler( SparkEnv.set(env) val rdd = job.finalStage.rdd val split = rdd.splits(job.partitions(0)) - val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) + val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0, null) //TODO figure out what to do about Task here try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 8cd4c661eb70d..e1b26a0cf5fea 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -71,7 +71,7 @@ private[spark] class ResultTask[T, U]( } override def run(attemptId: Long): U = { - val context = new TaskContext(stageId, partition, attemptId) + val context = new TaskContext(stageId, partition, attemptId, this) try { func(context, rdd.iterator(split, context)) } finally { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index ef56b4318cf55..e97dd0ad3ab0b 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -121,7 +121,7 @@ private[spark] class ShuffleMapTask( override def run(attemptId: Long): MapStatus = { val numOutputSplits = dep.partitioner.numPartitions - val taskContext = new TaskContext(stageId, partition, attemptId) + val taskContext = new TaskContext(stageId, partition, attemptId, this) try { // Partition the map output. val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index a5db7103f5ce8..4af5de6cf66a1 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -16,7 +16,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte val rdd = new RDD[String](sc, List()) { override def getSplits = Array[Split](StubSplit(0)) override def compute(split: Split, context: TaskContext) = { - context.addOnCompleteCallback(() => completed = true) + context.addOnCompleteCallback(_ => completed = true) sys.error("failed") } } From e319ac74c1621500697fa48f6f9d6846d5ea086d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 10:17:56 -0800 Subject: [PATCH 159/696] cogrouped RDD stores the amount of time taken to read shuffle data in each task --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 4 +++- core/src/main/scala/spark/scheduler/Task.scala | 4 ++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 8fafd27bb6985..a257e8dd85041 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -110,7 +110,9 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) mySeq(depNum) += v } val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[K, Seq[Any]](shuffleId, split.index).foreach(mergePair) + val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index) + fetchItr.foreach(mergePair) + context.task.setShuffleReadMillis(fetchItr.getNetMillis) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index ef987fdeb696b..b99629f67a6d9 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -16,6 +16,10 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { def preferredLocations: Seq[String] = Nil var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. + + private var shufflerReadMillis : Option[Long] = None + def setShuffleReadMillis(millis: Long) = shufflerReadMillis = Some(millis) + def getShuffleReadMillis = shufflerReadMillis } /** From 7eea64aa4c0d6a51406e0d1b039906ee9559cd58 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 5 Feb 2013 11:41:31 -0800 Subject: [PATCH 160/696] Streaming constructor which takes JavaSparkContext It's sometimes helpful to directly pass a JavaSparkContext, and take advantage of the various constructors available for that. --- .../spark/streaming/api/java/JavaStreamingContext.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index f82e6a37ccb95..e7f446a49b581 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -33,6 +33,14 @@ class JavaStreamingContext(val ssc: StreamingContext) { def this(master: String, frameworkName: String, batchDuration: Duration) = this(new StreamingContext(master, frameworkName, batchDuration)) + /** + * Creates a StreamingContext. + * @param sparkContext The underlying JavaSparkContext to use + * @param batchDuration The time interval at which streaming data will be divided into batches + */ + def this(sparkContext: JavaSparkContext, batchDuration: Duration) = + this(new StreamingContext(sparkContext.sc, batchDuration)) + /** * Re-creates a StreamingContext from a checkpoint file. * @param path Path either to the directory that was specified as the checkpoint directory, or From b29f9cc978a31ae339186e0e64f64c6f045f482d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 13:40:30 -0800 Subject: [PATCH 161/696] BlockManager.getMultiple returns a custom iterator, to enable tracking of shuffle performance --- .../scala/spark/storage/BlockManager.scala | 310 +++++++++--------- 1 file changed, 164 insertions(+), 146 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9893e9625d909..96d1a20d1b3b1 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -446,152 +446,8 @@ class BlockManager( * so that we can control the maxMegabytesInFlight for the fetch. */ def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]) - : Iterator[(String, Option[Iterator[Any]])] = { - - if (blocksByAddress == null) { - throw new IllegalArgumentException("BlocksByAddress is null") - } - val totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + totalBlocks + " blocks") - var startTime = System.currentTimeMillis - val localBlockIds = new ArrayBuffer[String]() - val remoteBlockIds = new HashSet[String]() - - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - - // A queue to hold our results. - val results = new LinkedBlockingQueue[FetchResult] - - // A request to fetch one or more blocks, complete with their sizes - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight - val fetchRequests = new Queue[FetchRequest] - - // Current bytes in flight from our requests - var bytesInFlight = 0L - - def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) - val cmId = new ConnectionManagerId(req.address.ip, req.address.port) - val blockMessageArray = new BlockMessageArray(req.blocks.map { - case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) - }) - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onSuccess { - case Some(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - throw new SparkException( - "Unexpected message " + blockMessage.getType + " received from " + cmId) - } - val blockId = blockMessage.getId - results.put(new FetchResult( - blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - } - } - case None => { - logError("Could not get block(s) from " + cmId) - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } - } - } - } - - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - - val numGets = remoteBlockIds.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - startTime = System.currentTimeMillis - for (id <- localBlockIds) { - getLocal(id) match { - case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") - } - } - } - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - - // Return an iterator that will read fetched blocks off the queue as they arrive. - return new Iterator[(String, Option[Iterator[Any]])] { - var resultsGotten = 0 - - def hasNext: Boolean = resultsGotten < totalBlocks - - def next(): (String, Option[Iterator[Any]]) = { - resultsGotten += 1 - val result = results.take() - bytesInFlight -= result.size - if (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - } + : BlockFetcherIterator = { + return new BlockFetcherIterator(this, blocksByAddress) } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -986,3 +842,165 @@ object BlockManager extends Logging { } } } + + +class BlockFetcherIterator( + private val blockManager: BlockManager, + val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] +) extends Iterator[(String, Option[Iterator[Any]])] with Logging { + + import blockManager._ + + private var remoteBytesRead = 0l + + if (blocksByAddress == null) { + throw new IllegalArgumentException("BlocksByAddress is null") + } + val totalBlocks = blocksByAddress.map(_._2.size).sum + logDebug("Getting " + totalBlocks + " blocks") + var startTime = System.currentTimeMillis + val localBlockIds = new ArrayBuffer[String]() + val remoteBlockIds = new HashSet[String]() + + // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + // the block (since we want all deserializaton to happen in the calling thread); can also + // represent a fetch failure if size == -1. + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } + + // A queue to hold our results. + val results = new LinkedBlockingQueue[FetchResult] + + // A request to fetch one or more blocks, complete with their sizes + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + // the number of bytes in flight is limited to maxBytesInFlight + val fetchRequests = new Queue[FetchRequest] + + // Current bytes in flight from our requests + var bytesInFlight = 0L + + def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) + val cmId = new ConnectionManagerId(req.address.ip, req.address.port) + val blockMessageArray = new BlockMessageArray(req.blocks.map { + case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) + }) + bytesInFlight += req.size + val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) + future.onSuccess { + case Some(message) => { + val bufferMessage = message.asInstanceOf[BufferMessage] + val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + for (blockMessage <- blockMessageArray) { + if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { + throw new SparkException( + "Unexpected message " + blockMessage.getType + " received from " + cmId) + } + val blockId = blockMessage.getId + results.put(new FetchResult( + blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) + remoteBytesRead += req.size + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } + } + case None => { + logError("Could not get block(s) from " + cmId) + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + } + } + + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + curBlocks += ((blockId, size)) + curRequestSize += size + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + + val numGets = remoteBlockIds.size - fetchRequests.size + logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + startTime = System.currentTimeMillis + for (id <- localBlockIds) { + getLocal(id) match { + case Some(iter) => { + results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + logDebug("Got local block " + id) + } + case None => { + throw new BlockException(id, "Could not get block " + id + " from local machine") + } + } + } + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + + //an iterator that will read fetched blocks off the queue as they arrive. + var resultsGotten = 0 + + def hasNext: Boolean = resultsGotten < totalBlocks + + def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val result = results.take() + bytesInFlight -= result.size + if (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + + + //methods to profile the block fetching + + def numLocalBlocks = localBlockIds.size + def numRemoteBlocks = remoteBlockIds.size + +} From 696e4b21679a2f11e943abb06149476044e0bc82 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 14:29:16 -0800 Subject: [PATCH 162/696] track remoteFetchTime --- core/src/main/scala/spark/storage/BlockManager.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 96d1a20d1b3b1..d839a1eb8977c 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -852,6 +852,7 @@ class BlockFetcherIterator( import blockManager._ private var remoteBytesRead = 0l + private var remoteFetchTime = 0l if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") @@ -893,9 +894,12 @@ class BlockFetcherIterator( }) bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + val fetchStart = System.currentTimeMillis() val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) future.onSuccess { case Some(message) => { + val fetchDone = System.currentTimeMillis() + remoteFetchTime += fetchDone - fetchStart val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { From cfab1a35282c08cec351e468f6e57d1261f02d10 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 14:31:46 -0800 Subject: [PATCH 163/696] add as many fetch requests as we can, subject to maxBytesInFlight --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9893e9625d909..2e7db608414fe 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -585,7 +585,7 @@ class BlockManager( resultsGotten += 1 val result = results.take() bytesInFlight -= result.size - if (!fetchRequests.isEmpty && + while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { sendRequest(fetchRequests.dequeue()) } From 1704b124d84e50697f66b68120ba340693b1d936 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 14:33:52 -0800 Subject: [PATCH 164/696] add as many fetch requests as we can, subject to maxBytesInFlight --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index d839a1eb8977c..71528101dbab5 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -994,7 +994,7 @@ class BlockFetcherIterator( resultsGotten += 1 val result = results.take() bytesInFlight -= result.size - if (!fetchRequests.isEmpty && + while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { sendRequest(fetchRequests.dequeue()) } From 8bd0e888f377f13ac239df4ffd49fc666095e764 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 5 Feb 2013 17:50:25 -0600 Subject: [PATCH 165/696] Inline mergePair to look more like the narrow dep branch. No functionality changes, I think this is just more consistent given mergePair isn't called multiple times/recursive. Also added a comment to explain the usual case of having two parent RDDs. --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 8fafd27bb6985..4893fe8d784bf 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -84,6 +84,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { val split = s.asInstanceOf[CoGroupSplit] val numRdds = split.deps.size + // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) val map = new JHashMap[K, Seq[ArrayBuffer[Any]]] def getSeq(k: K): Seq[ArrayBuffer[Any]] = { val seq = map.get(k) @@ -104,13 +105,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) } case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle - def mergePair(pair: (K, Seq[Any])) { - val mySeq = getSeq(pair._1) - for (v <- pair._2) - mySeq(depNum) += v - } val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[K, Seq[Any]](shuffleId, split.index).foreach(mergePair) + for ((k, vs) <- fetcher.fetch[K, Seq[Any]](shuffleId, split.index)) { + getSeq(k)(depNum) ++= vs + } } } JavaConversions.mapAsScalaMap(map).iterator From 1ba3393ceb5709620a28b8bc01826153993fc444 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 5 Feb 2013 17:56:50 -0600 Subject: [PATCH 166/696] Increase DriverSuite timeout. --- core/src/test/scala/spark/DriverSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 342610e1dd93b..5e84b3a66a1a9 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -9,10 +9,11 @@ import org.scalatest.time.SpanSugar._ class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { + assert(System.getenv("SPARK_HOME") != null) // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => - failAfter(10 seconds) { + failAfter(30 seconds) { Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } From 0e19093fd89ec9740f98cdcffd1ec09f4faf2490 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 5 Feb 2013 18:58:00 -0600 Subject: [PATCH 167/696] Handle Terminated to avoid endless DeathPactExceptions. Credit to Roland Kuhn, Akka's tech lead, for pointing out this various obvious fix, but StandaloneExecutorBackend.preStart's catch block would never (ever) get hit, because all of the operation's in preStart are async. So, the System.exit in the catch block was skipped, and instead Akka was sending Terminated messages which, since we didn't handle, it turned into DeathPactException, which started a postRestart/preStart infinite loop. --- .../scala/spark/deploy/worker/Worker.scala | 7 ++---- .../executor/StandaloneExecutorBackend.scala | 25 ++++++++----------- 2 files changed, 13 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 8b41620d98232..48177a638a07e 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -1,19 +1,16 @@ package spark.deploy.worker import scala.collection.mutable.{ArrayBuffer, HashMap} -import akka.actor.{ActorRef, Props, Actor} +import akka.actor.{ActorRef, Props, Actor, Terminated} import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ -import akka.remote.RemoteClientLifeCycleEvent +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.text.SimpleDateFormat import java.util.Date -import akka.remote.RemoteClientShutdown -import akka.remote.RemoteClientDisconnected import spark.deploy.RegisterWorker import spark.deploy.LaunchExecutor import spark.deploy.RegisterWorkerFailed -import akka.actor.Terminated import java.io.File private[spark] class Worker( diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index e45288ff5300a..224c126fdd1ee 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -4,16 +4,15 @@ import java.nio.ByteBuffer import spark.Logging import spark.TaskState.TaskState import spark.util.AkkaUtils -import akka.actor.{ActorRef, Actor, Props} +import akka.actor.{ActorRef, Actor, Props, Terminated} +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} -import akka.remote.RemoteClientLifeCycleEvent import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask import spark.scheduler.cluster.RegisterExecutorFailed import spark.scheduler.cluster.RegisterExecutor - private[spark] class StandaloneExecutorBackend( executor: Executor, driverUrl: String, @@ -27,17 +26,11 @@ private[spark] class StandaloneExecutorBackend( var driver: ActorRef = null override def preStart() { - try { - logInfo("Connecting to driver: " + driverUrl) - driver = context.actorFor(driverUrl) - driver ! RegisterExecutor(executorId, hostname, cores) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - context.watch(driver) // Doesn't work with remote actors, but useful for testing - } catch { - case e: Exception => - logError("Failed to connect to driver", e) - System.exit(1) - } + logInfo("Connecting to driver: " + driverUrl) + driver = context.actorFor(driverUrl) + driver ! RegisterExecutor(executorId, hostname, cores) + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(driver) // Doesn't work with remote actors, but useful for testing } override def receive = { @@ -52,6 +45,10 @@ private[spark] class StandaloneExecutorBackend( case LaunchTask(taskDesc) => logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) + + case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + logError("Driver terminated or disconnected! Shutting down.") + System.exit(1) } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { From 379564c7e05cfd0acceaee5641e0d06c86985bff Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 5 Feb 2013 18:30:21 -0800 Subject: [PATCH 168/696] setup plumbing to get task metrics; lots of unfinished parts, but basic flow in place --- .../spark/BlockStoreShuffleFetcher.scala | 11 +++-- .../src/main/scala/spark/ShuffleFetcher.scala | 3 +- .../main/scala/spark/executor/Executor.scala | 2 +- .../scala/spark/executor/TaskMetrics.scala | 16 ++++++++ .../main/scala/spark/rdd/CoGroupedRDD.scala | 5 ++- .../src/main/scala/spark/scheduler/Task.scala | 7 ++-- .../scala/spark/scheduler/TaskResult.scala | 5 ++- .../scala/spark/storage/BlockManager.scala | 41 ++++++++++++++++--- core/src/test/scala/spark/JavaAPISuite.java | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 10 ++--- 10 files changed, 78 insertions(+), 24 deletions(-) create mode 100644 core/src/main/scala/spark/executor/TaskMetrics.scala diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index cab1975f63863..bb4e3de63ccb8 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -3,19 +3,19 @@ package spark import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import spark.storage.BlockManagerId +import storage.{DelegateBlockFetchTracker, BlockManagerId} import util.TimedIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int) = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager - + val startTime = System.currentTimeMillis val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId) logDebug("Fetching map output location for shuffle %d, reduce %d took %d ms".format( shuffleId, reduceId, System.currentTimeMillis - startTime)) - + val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] for (((address, size), index) <- statuses.zipWithIndex) { splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) @@ -46,6 +46,9 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } } - new TimedIterator(blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock)) + val blockFetcherItr = blockManager.getMultiple(blocksByAddress) + val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker + itr.setDelegate(blockFetcherItr) + itr } } diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 237faa0a6ee2a..d64b2611b7866 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,5 +1,6 @@ package spark +import storage.BlockFetchTracker import util.TimedIterator private[spark] abstract class ShuffleFetcher { @@ -7,7 +8,7 @@ private[spark] abstract class ShuffleFetcher { * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K, V)] + def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K,V)] with BlockFetchTracker /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index bd21ba719a77c..0906702a3dff7 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -95,7 +95,7 @@ private[spark] class Executor extends Logging { env.mapOutputTracker.updateGeneration(task.generation) val value = task.run(taskId.toInt) val accumUpdates = Accumulators.values - val result = new TaskResult(value, accumUpdates) + val result = new TaskResult(value, accumUpdates, TaskMetrics(task)) val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala new file mode 100644 index 0000000000000..6cdc6d0b7e292 --- /dev/null +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -0,0 +1,16 @@ +package spark.executor + +import spark.scheduler.Task + +case class TaskMetrics( + val totalBlocksFetched : Option[Int], + val remoteBlocksFetched: Option[Int], + val localBlocksFetched: Option[Int], + val remoteFetchWaitTime: Option[Long], + val remoteBytesRead: Option[Long] +) + +object TaskMetrics { + private[spark] def apply(task: Task[_]) : TaskMetrics = + TaskMetrics(None, None, None, task.remoteFetchWaitTime, task.remoteReadBytes) +} \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index a257e8dd85041..90b822ec857e4 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -112,7 +112,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) val fetcher = SparkEnv.get.shuffleFetcher val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index) fetchItr.foreach(mergePair) - context.task.setShuffleReadMillis(fetchItr.getNetMillis) + context.task.shuffleReadMillis = Some(fetchItr.getNetMillis) + context.task.remoteFetchTime = Some(fetchItr.remoteFetchTime) + context.task.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) + context.task.remoteReadBytes = Some(fetchItr.remoteBytesRead) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index b99629f67a6d9..9c1d5869be6b4 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -17,9 +17,10 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. - private var shufflerReadMillis : Option[Long] = None - def setShuffleReadMillis(millis: Long) = shufflerReadMillis = Some(millis) - def getShuffleReadMillis = shufflerReadMillis + var shuffleReadMillis : Option[Long] = None + var remoteReadBytes : Option[Long] = None + var remoteFetchWaitTime : Option[Long] = None + var remoteFetchTime : Option[Long] = None } /** diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 9a54d0e8541eb..003ee2bb2ee22 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -3,13 +3,14 @@ package spark.scheduler import java.io._ import scala.collection.mutable.Map +import spark.executor.TaskMetrics // Task result. Also contains updates to accumulator variables. // TODO: Use of distributed cache to return result is a hack to get around // what seems to be a bug with messages over 60KB in libprocess; fix it private[spark] -class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any]) extends Externalizable { - def this() = this(null.asInstanceOf[T], null) +class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: TaskMetrics) extends Externalizable { + def this() = this(null.asInstanceOf[T], null, null) override def writeExternal(out: ObjectOutput) { out.writeObject(value) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 71528101dbab5..677b2e6a42da5 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -844,15 +844,37 @@ object BlockManager extends Logging { } +private[spark] trait BlockFetchTracker { + def totalBlocks : Int + def numLocalBlocks: Int + def numRemoteBlocks: Int + def remoteFetchTime : Long + def remoteFetchWaitTime: Long + def remoteBytesRead : Long +} + +private[spark] trait DelegateBlockFetchTracker extends BlockFetchTracker { + var delegate : BlockFetchTracker = _ + def setDelegate(d: BlockFetchTracker) {delegate = d} + def totalBlocks = delegate.totalBlocks + def numLocalBlocks = delegate.numLocalBlocks + def numRemoteBlocks = delegate.numRemoteBlocks + def remoteFetchTime = delegate.remoteFetchTime + def remoteFetchWaitTime = delegate.remoteFetchWaitTime + def remoteBytesRead = delegate.remoteBytesRead +} + + class BlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] -) extends Iterator[(String, Option[Iterator[Any]])] with Logging { +) extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { import blockManager._ - private var remoteBytesRead = 0l - private var remoteFetchTime = 0l + private var _remoteBytesRead = 0l + private var _remoteFetchTime = 0l + private var _remoteFetchWaitTime = 0l if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") @@ -899,7 +921,7 @@ class BlockFetcherIterator( future.onSuccess { case Some(message) => { val fetchDone = System.currentTimeMillis() - remoteFetchTime += fetchDone - fetchStart + _remoteFetchTime += fetchDone - fetchStart val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { @@ -910,7 +932,7 @@ class BlockFetcherIterator( val blockId = blockMessage.getId results.put(new FetchResult( blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) - remoteBytesRead += req.size + _remoteBytesRead += req.size logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } } @@ -992,7 +1014,10 @@ class BlockFetcherIterator( def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 + val startFetchWait = System.currentTimeMillis() val result = results.take() + val stopFetchWait = System.currentTimeMillis() + _remoteFetchWaitTime += (stopFetchWait - startFetchWait) bytesInFlight -= result.size while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { @@ -1003,8 +1028,12 @@ class BlockFetcherIterator( //methods to profile the block fetching - def numLocalBlocks = localBlockIds.size def numRemoteBlocks = remoteBlockIds.size + def remoteFetchTime = _remoteFetchTime + def remoteFetchWaitTime = _remoteFetchWaitTime + + def remoteBytesRead = _remoteBytesRead + } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 934e4c2f6793b..2d13edfd4899e 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -423,7 +423,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0); + TaskContext context = new TaskContext(0, 0, 0, null); Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 83663ac702a5b..f74b425dd57a5 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -265,7 +265,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any]())) + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null)) } } } @@ -463,14 +463,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val noAccum = Map[Long, Any]() // We rely on the event queue being ordered and increasing the generation number by 1 // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) taskSet.tasks(1).generation = newGeneration val secondStage = interceptStage(reduceRdd) { - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum)) + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null)) } assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) From 67df7f2fa2e09487fe8dcf39ab80606d95383ea5 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 5 Feb 2013 21:08:21 -0600 Subject: [PATCH 169/696] Add private, minor formatting. --- .../spark/network/ConnectionManager.scala | 35 ++++++++----------- 1 file changed, 14 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index c7f226044d1e9..b6ec664d7e81b 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -66,31 +66,28 @@ private[spark] class ConnectionManager(port: Int) extends Logging { val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort) logInfo("Bound socket to port " + serverChannel.socket.getLocalPort() + " with id = " + id) - val thisInstance = this val selectorThread = new Thread("connection-manager-thread") { - override def run() { - thisInstance.run() - } + override def run() = ConnectionManager.this.run() } selectorThread.setDaemon(true) selectorThread.start() - def run() { + private def run() { try { while(!selectorThread.isInterrupted) { - for( (connectionManagerId, sendingConnection) <- connectionRequests) { + for ((connectionManagerId, sendingConnection) <- connectionRequests) { sendingConnection.connect() addConnection(sendingConnection) connectionRequests -= connectionManagerId } sendMessageRequests.synchronized { - while(!sendMessageRequests.isEmpty) { + while (!sendMessageRequests.isEmpty) { val (message, connection) = sendMessageRequests.dequeue connection.send(message) } } - while(!keyInterestChangeRequests.isEmpty) { + while (!keyInterestChangeRequests.isEmpty) { val (key, ops) = keyInterestChangeRequests.dequeue val connection = connectionsByKey(key) val lastOps = key.interestOps() @@ -126,14 +123,11 @@ private[spark] class ConnectionManager(port: Int) extends Logging { if (key.isValid) { if (key.isAcceptable) { acceptConnection(key) - } else - if (key.isConnectable) { + } else if (key.isConnectable) { connectionsByKey(key).asInstanceOf[SendingConnection].finishConnect() - } else - if (key.isReadable) { + } else if (key.isReadable) { connectionsByKey(key).read() - } else - if (key.isWritable) { + } else if (key.isWritable) { connectionsByKey(key).write() } } @@ -144,7 +138,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } } - def acceptConnection(key: SelectionKey) { + private def acceptConnection(key: SelectionKey) { val serverChannel = key.channel.asInstanceOf[ServerSocketChannel] val newChannel = serverChannel.accept() val newConnection = new ReceivingConnection(newChannel, selector) @@ -154,7 +148,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]") } - def addConnection(connection: Connection) { + private def addConnection(connection: Connection) { connectionsByKey += ((connection.key, connection)) if (connection.isInstanceOf[SendingConnection]) { val sendingConnection = connection.asInstanceOf[SendingConnection] @@ -165,7 +159,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { connection.onClose(removeConnection) } - def removeConnection(connection: Connection) { + private def removeConnection(connection: Connection) { connectionsByKey -= connection.key if (connection.isInstanceOf[SendingConnection]) { val sendingConnection = connection.asInstanceOf[SendingConnection] @@ -222,16 +216,16 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } } - def handleConnectionError(connection: Connection, e: Exception) { + private def handleConnectionError(connection: Connection, e: Exception) { logInfo("Handling connection error on connection to " + connection.remoteConnectionManagerId) removeConnection(connection) } - def changeConnectionKeyInterest(connection: Connection, ops: Int) { + private def changeConnectionKeyInterest(connection: Connection, ops: Int) { keyInterestChangeRequests += ((connection.key, ops)) } - def receiveMessage(connection: Connection, message: Message) { + private def receiveMessage(connection: Connection, message: Message) { val connectionManagerId = ConnectionManagerId.fromSocketAddress(message.senderAddress) logDebug("Received [" + message + "] from [" + connectionManagerId + "]") val runnable = new Runnable() { @@ -351,7 +345,6 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private[spark] object ConnectionManager { def main(args: Array[String]) { - val manager = new ConnectionManager(9999) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") From f2bc7480131c7468eb6d3bc6089a4deadf0a2a88 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 5 Feb 2013 21:23:36 -0600 Subject: [PATCH 170/696] Add RDD.coalesce. --- core/src/main/scala/spark/RDD.scala | 7 +++++++ core/src/main/scala/spark/api/java/JavaRDDLike.scala | 10 ++++++++++ core/src/test/scala/spark/CheckpointSuite.scala | 4 ++-- core/src/test/scala/spark/RDDSuite.scala | 8 ++++---- 4 files changed, 23 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 9d6ea782bd83c..f0bc85865c5e3 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -20,6 +20,7 @@ import spark.partial.BoundedDouble import spark.partial.CountEvaluator import spark.partial.GroupedCountEvaluator import spark.partial.PartialResult +import spark.rdd.CoalescedRDD import spark.rdd.CartesianRDD import spark.rdd.FilteredRDD import spark.rdd.FlatMappedRDD @@ -231,6 +232,12 @@ abstract class RDD[T: ClassManifest]( def distinct(): RDD[T] = distinct(splits.size) + /** + * Return a new RDD that is reduced into `numSplits` partitions. + */ + def coalesce(numSplits: Int = sc.defaultParallelism): RDD[T] = + new CoalescedRDD(this, numSplits) + /** * Return a sampled subset of this RDD. */ diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 60025b459c383..295eaa57c0c2e 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -130,6 +130,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest, other.classManifest) + /** + * Return a new RDD that is reduced into the default number of partitions. + */ + def coalesce(): RDD[T] = coalesce(rdd.context.defaultParallelism) + + /** + * Return a new RDD that is reduced into `numSplits` partitions. + */ + def coalesce(numSplits: Int): RDD[T] = rdd.coalesce(numSplits) + /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 0b74607fb85a6..0d08fd239632b 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -114,12 +114,12 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } test("CoalescedRDD") { - testCheckpointing(new CoalescedRDD(_, 2)) + testCheckpointing(_.coalesce(2)) // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed // Current implementation of CoalescedRDDSplit has transient reference to parent RDD, // so only the RDD will reduce in serialized size, not the splits. - testParentCheckpointing(new CoalescedRDD(_, 2), true, false) + testParentCheckpointing(_.coalesce(2), true, false) // Test that the CoalescedRDDSplit updates parent splits (CoalescedRDDSplit.parents) after // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits. diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index fe7deb10d63b0..ffa866de7532f 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -122,7 +122,7 @@ class RDDSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) - val coalesced1 = new CoalescedRDD(data, 2) + val coalesced1 = data.coalesce(2) assert(coalesced1.collect().toList === (1 to 10).toList) assert(coalesced1.glom().collect().map(_.toList).toList === List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10))) @@ -133,19 +133,19 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList === List(5, 6, 7, 8, 9)) - val coalesced2 = new CoalescedRDD(data, 3) + val coalesced2 = data.coalesce(3) assert(coalesced2.collect().toList === (1 to 10).toList) assert(coalesced2.glom().collect().map(_.toList).toList === List(List(1, 2, 3), List(4, 5, 6), List(7, 8, 9, 10))) - val coalesced3 = new CoalescedRDD(data, 10) + val coalesced3 = data.coalesce(10) assert(coalesced3.collect().toList === (1 to 10).toList) assert(coalesced3.glom().collect().map(_.toList).toList === (1 to 10).map(x => List(x)).toList) // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. - val coalesced4 = new CoalescedRDD(data, 20) + val coalesced4 = data.coalesce(20) assert(coalesced4.collect().toList === (1 to 10).toList) assert(coalesced4.glom().collect().map(_.toList).toList === (1 to 10).map(x => List(x)).toList) From f4d43cb43e64ec3436a129cf3f7a177374451060 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 5 Feb 2013 21:26:44 -0600 Subject: [PATCH 171/696] Remove unneeded zipWithIndex. Also rename r->rdd and remove unneeded extra type info. --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 4893fe8d784bf..021118c8ba5de 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -47,7 +47,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) @transient var deps_ = { val deps = new ArrayBuffer[Dependency[_]] - for ((rdd, index) <- rdds.zipWithIndex) { + for (rdd <- rdds) { if (rdd.partitioner == Some(part)) { logInfo("Adding one-to-one dependency with " + rdd) deps += new OneToOneDependency(rdd) @@ -65,12 +65,14 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) @transient var splits_ : Array[Split] = { val array = new Array[Split](part.numPartitions) for (i <- 0 until array.size) { - array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (r, j) => + // Each CoGroupSplit will have a dependency per contributing RDD + array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (rdd, j) => + // Assume each RDD contributed a single dependency, and get it dependencies(j) match { case s: ShuffleDependency[_, _] => - new ShuffleCoGroupSplitDep(s.shuffleId): CoGroupSplitDep + new ShuffleCoGroupSplitDep(s.shuffleId) case _ => - new NarrowCoGroupSplitDep(r, i, r.splits(i)): CoGroupSplitDep + new NarrowCoGroupSplitDep(rdd, i, rdd.splits(i)) } }.toList) } @@ -97,7 +99,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) } } for ((dep, depNum) <- split.deps.zipWithIndex) dep match { - case NarrowCoGroupSplitDep(rdd, itsSplitIndex, itsSplit) => { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent for ((k, v) <- rdd.iterator(itsSplit, context)) { getSeq(k.asInstanceOf[K])(depNum) += v From a9c8d53cfa0bd09565799cec88344b286d7cc436 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 5 Feb 2013 22:14:18 -0600 Subject: [PATCH 172/696] Clean up RDDs, mainly to use getSplits. Also made sure clearDependencies() was calling super, to ensure the getSplits/getDependencies vars in the RDD base class get cleaned up. --- core/src/main/scala/spark/RDD.scala | 1 - core/src/main/scala/spark/rdd/BlockRDD.scala | 12 ++++------- .../main/scala/spark/rdd/CartesianRDD.scala | 3 ++- .../main/scala/spark/rdd/CheckpointRDD.scala | 4 +--- .../main/scala/spark/rdd/CoGroupedRDD.scala | 21 +++++++------------ .../main/scala/spark/rdd/CoalescedRDD.scala | 13 ++++++------ core/src/main/scala/spark/rdd/HadoopRDD.scala | 7 ++----- .../main/scala/spark/rdd/NewHadoopRDD.scala | 6 ++---- .../src/main/scala/spark/rdd/SampledRDD.scala | 8 +------ core/src/main/scala/spark/rdd/UnionRDD.scala | 8 ++----- core/src/main/scala/spark/rdd/ZippedRDD.scala | 6 +++--- 11 files changed, 30 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index f0bc85865c5e3..5f99591fd5458 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -656,7 +656,6 @@ abstract class RDD[T: ClassManifest]( */ private[spark] def markCheckpointed(checkpointRDD: RDD[_]) { clearDependencies() - dependencies_ = null splits_ = null deps = null // Forget the constructor argument for dependencies too } diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 2c022f88e0def..4214817c653a7 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -11,10 +11,6 @@ private[spark] class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { - @transient var splits_ : Array[Split] = (0 until blockIds.size).map(i => { - new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split] - }).toArray - @transient lazy val locations_ = { val blockManager = SparkEnv.get.blockManager /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ @@ -22,7 +18,10 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St HashMap(blockIds.zip(locations):_*) } - override def getSplits = splits_ + override def getSplits = (0 until blockIds.size).map(i => { + new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split] + }).toArray + override def compute(split: Split, context: TaskContext): Iterator[T] = { val blockManager = SparkEnv.get.blockManager @@ -37,8 +36,5 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St override def getPreferredLocations(split: Split) = locations_(split.asInstanceOf[BlockRDDSplit].blockId) - override def clearDependencies() { - splits_ = null - } } diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 0f9ca0653198f..2f572a1941dc2 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -35,7 +35,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( val numSplitsInRdd2 = rdd2.splits.size - override def getSplits: Array[Split] = { + override def getSplits = { // create the cross product split val array = new Array[Split](rdd1.splits.size * rdd2.splits.size) for (s1 <- rdd1.splits; s2 <- rdd2.splits) { @@ -66,6 +66,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( ) override def clearDependencies() { + super.clearDependencies() rdd1 = null rdd2 = null } diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 96b593ba7ca6d..7cde523f113f5 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -20,7 +20,7 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) - @transient val splits_ : Array[Split] = { + override def getSplits = { val dirContents = fs.listStatus(new Path(checkpointPath)) val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted val numSplits = splitFiles.size @@ -34,8 +34,6 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri checkpointData = Some(new RDDCheckpointData[T](this)) checkpointData.get.cpFile = Some(checkpointPath) - override def getSplits = splits_ - override def getPreferredLocations(split: Split): Seq[String] = { val status = fs.getFileStatus(new Path(checkpointPath)) val locations = fs.getFileBlockLocations(status, 0, status.getLen) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 021118c8ba5de..d31ce13706d39 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -43,26 +43,22 @@ private[spark] class CoGroupAggregator class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) with Logging { - val aggr = new CoGroupAggregator + private val aggr = new CoGroupAggregator - @transient var deps_ = { - val deps = new ArrayBuffer[Dependency[_]] - for (rdd <- rdds) { + override def getDependencies = { + rdds.map { rdd => if (rdd.partitioner == Some(part)) { logInfo("Adding one-to-one dependency with " + rdd) - deps += new OneToOneDependency(rdd) + new OneToOneDependency(rdd) } else { logInfo("Adding shuffle dependency with " + rdd) val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true) - deps += new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part) + new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part) } } - deps.toList } - override def getDependencies = deps_ - - @transient var splits_ : Array[Split] = { + override def getSplits = { val array = new Array[Split](part.numPartitions) for (i <- 0 until array.size) { // Each CoGroupSplit will have a dependency per contributing RDD @@ -79,8 +75,6 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) array } - override def getSplits = splits_ - override val partitioner = Some(part) override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { @@ -117,8 +111,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) } override def clearDependencies() { - deps_ = null - splits_ = null + super.clearDependencies() rdds = null } } diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 4c57434b65f9a..a1aa7a30b0e4e 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -31,7 +31,7 @@ class CoalescedRDD[T: ClassManifest]( maxPartitions: Int) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies - override def getSplits: Array[Split] = { + override def getSplits = { val prevSplits = prev.splits if (prevSplits.length < maxPartitions) { prevSplits.map(_.index).map{idx => new CoalescedRDDSplit(idx, prev, Array(idx)) } @@ -50,14 +50,13 @@ class CoalescedRDD[T: ClassManifest]( } } - override def getDependencies: Seq[Dependency[_]] = List( - new NarrowDependency(prev) { - def getParents(id: Int): Seq[Int] = - splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices - } - ) + override def getDependencies = Seq(new NarrowDependency(prev) { + def getParents(id: Int): Seq[Int] = + splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices + }) override def clearDependencies() { + super.clearDependencies() prev = null } } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index f547f53812661..cd948de967762 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -45,10 +45,9 @@ class HadoopRDD[K, V]( extends RDD[(K, V)](sc, Nil) { // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it - val confBroadcast = sc.broadcast(new SerializableWritable(conf)) + private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) - @transient - val splits_ : Array[Split] = { + override def getSplits = { val inputFormat = createInputFormat(conf) val inputSplits = inputFormat.getSplits(conf, minSplits) val array = new Array[Split](inputSplits.size) @@ -63,8 +62,6 @@ class HadoopRDD[K, V]( .asInstanceOf[InputFormat[K, V]] } - override def getSplits = splits_ - override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[HadoopSplit] var reader: RecordReader[K, V] = null diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index c3b155fcbddd6..2d000f5c689c3 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -29,7 +29,7 @@ class NewHadoopRDD[K, V]( with HadoopMapReduceUtil { // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it - val confBroadcast = sc.broadcast(new SerializableWritable(conf)) + private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) // private val serializableConf = new SerializableWritable(conf) private val jobtrackerId: String = { @@ -39,7 +39,7 @@ class NewHadoopRDD[K, V]( @transient private val jobId = new JobID(jobtrackerId, id) - @transient private val splits_ : Array[Split] = { + override def getSplits = { val inputFormat = inputFormatClass.newInstance val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray @@ -50,8 +50,6 @@ class NewHadoopRDD[K, V]( result } - override def getSplits = splits_ - override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopSplit] val conf = confBroadcast.value.value diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index e24ad23b21423..81626d5009ac5 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -19,13 +19,11 @@ class SampledRDD[T: ClassManifest]( seed: Int) extends RDD[T](prev) { - @transient var splits_ : Array[Split] = { + override def getSplits = { val rg = new Random(seed) firstParent[T].splits.map(x => new SampledRDDSplit(x, rg.nextInt)) } - override def getSplits = splits_ - override def getPreferredLocations(split: Split) = firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev) @@ -48,8 +46,4 @@ class SampledRDD[T: ClassManifest]( firstParent[T].iterator(split.prev, context).filter(x => (rand.nextDouble <= frac)) } } - - override def clearDependencies() { - splits_ = null - } } diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 26a2d511f2670..5ac24d2ffc7ca 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -28,7 +28,7 @@ class UnionRDD[T: ClassManifest]( @transient var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies - override def getSplits: Array[Split] = { + override def getSplits = { val array = new Array[Split](rdds.map(_.splits.size).sum) var pos = 0 for (rdd <- rdds; split <- rdd.splits) { @@ -38,7 +38,7 @@ class UnionRDD[T: ClassManifest]( array } - override def getDependencies: Seq[Dependency[_]] = { + override def getDependencies = { val deps = new ArrayBuffer[Dependency[_]] var pos = 0 for (rdd <- rdds) { @@ -53,8 +53,4 @@ class UnionRDD[T: ClassManifest]( override def getPreferredLocations(s: Split): Seq[String] = s.asInstanceOf[UnionSplit[T]].preferredLocations() - - override def clearDependencies() { - rdds = null - } } diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index e5df6d8c7239b..a079720a937a0 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -29,10 +29,9 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( sc: SparkContext, var rdd1: RDD[T], var rdd2: RDD[U]) - extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) - with Serializable { + extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) { - override def getSplits: Array[Split] = { + override def getSplits = { if (rdd1.splits.size != rdd2.splits.size) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } @@ -54,6 +53,7 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( } override def clearDependencies() { + super.clearDependencies() rdd1 = null rdd2 = null } From dab81a85116c828231277bcfa3a7230b29bddaba Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 5 Feb 2013 20:53:14 -0800 Subject: [PATCH 173/696] Fixing to match Spark styleguide --- .../examples/{OLAPQuery.scala => LogQuery.scala} | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) rename examples/src/main/scala/spark/examples/{OLAPQuery.scala => LogQuery.scala} (84%) diff --git a/examples/src/main/scala/spark/examples/OLAPQuery.scala b/examples/src/main/scala/spark/examples/LogQuery.scala similarity index 84% rename from examples/src/main/scala/spark/examples/OLAPQuery.scala rename to examples/src/main/scala/spark/examples/LogQuery.scala index ff3af01b1734b..5330b8da9444f 100644 --- a/examples/src/main/scala/spark/examples/OLAPQuery.scala +++ b/examples/src/main/scala/spark/examples/LogQuery.scala @@ -5,7 +5,7 @@ import spark.SparkContext._ /** * Executes a roll up-style query against Apache logs. */ -object OLAPQuery { +object LogQuery { val exampleApacheLogs = List( """10.10.10.10 - "FRED" [18/Jan/2013:17:56:07 +1100] "GET http://images.com/2013/Generic.jpg | HTTP/1.1" 304 315 "http://referall.com/" "Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; @@ -23,16 +23,16 @@ object OLAPQuery { def main(args: Array[String]) { if (args.length == 0) { - System.err.println("Usage: OLAPQuery [logFile]") + System.err.println("Usage: LogQuery [logFile]") System.exit(1) } - val sc = new SparkContext(args(0), "OLAP Query") + val sc = new SparkContext(args(0), "Log Query") val dataSet = if (args.length == 2) sc.textFile(args(1)) else sc.parallelize(exampleApacheLogs) - val apache_log_regex = + val apacheLogRegex = """^([\d.]+) (\S+) (\S+) \[([\w\d:/]+\s[+\-]\d{4})\] "(.+?)" (\d{3}) ([\d\-]+) "([^"]+)" "([^"]+)".*""".r /** Tracks the total query count and number of aggregate bytes for a particular group. */ @@ -42,8 +42,8 @@ object OLAPQuery { } def extractKey(line: String): (String, String, String) = { - apache_log_regex findFirstIn line match { - case Some(apache_log_regex(ip, _, user, dateTime, query, status, bytes, referer, ua)) => + apacheLogRegex.findFirstIn(line) match { + case Some(apacheLogRegex(ip, _, user, dateTime, query, status, bytes, referer, ua)) => if (user != "\"-\"") (ip, user, query) else (null, null, null) case _ => (null, null, null) @@ -51,8 +51,8 @@ object OLAPQuery { } def extractStats(line: String): Stats = { - apache_log_regex findFirstIn line match { - case Some(apache_log_regex(ip, _, user, dateTime, query, status, bytes, referer, ua)) => + apacheLogRegex.findFirstIn(line) match { + case Some(apacheLogRegex(ip, _, user, dateTime, query, status, bytes, referer, ua)) => new Stats(1, bytes.toInt) case _ => new Stats(1, 0) } From 934a53c8b677df524315a75011b4c9396eb4b54e Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 5 Feb 2013 22:19:58 -0800 Subject: [PATCH 174/696] Change docs on 'reduce' since the merging of local reduces no longer preserves ordering, so the reduce function must also be commutative. --- core/src/main/scala/spark/RDD.scala | 2 +- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 2 +- docs/scala-programming-guide.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index d15c6f739663c..3ad3b4d2337ad 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -330,7 +330,7 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial def toArray(): Array[T] = collect() /** - * Reduces the elements of this RDD using the specified associative binary operator. + * Reduces the elements of this RDD using the specified commutative and associative binary operator. */ def reduce(f: (T, T) => T): T = { val cleanF = sc.clean(f) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 81d3a944668d0..6da89f518f511 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -201,7 +201,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Reduces the elements of this RDD using the specified associative binary operator. + * Reduces the elements of this RDD using the specified commutative and associative binary operator. */ def reduce(f: JFunction2[T, T, T]): T = rdd.reduce(f) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 7350eca83796f..95de4f50bb2f2 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -203,7 +203,7 @@ A complete list of transformations is available in the [RDD API doc](api/core/in ActionMeaning reduce(func) - Aggregate the elements of the dataset using a function func (which takes two arguments and returns one). The function should be associative so that it can be computed correctly in parallel. + Aggregate the elements of the dataset using a function func (which takes two arguments and returns one). The function should be commutative and associative so that it can be computed correctly in parallel. collect() From 21d3946d172ded8573e2557a5cf124bb96e75d34 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 7 Feb 2013 16:54:31 +0200 Subject: [PATCH 175/696] Adding IPYTHON environment variable support for launching pyspark using ipython shell --- pyspark | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pyspark b/pyspark index ab7f4f50c01bd..d662e90287edf 100755 --- a/pyspark +++ b/pyspark @@ -36,4 +36,9 @@ if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then export SPARK_LAUNCH_WITH_SCALA=1 fi -exec "$PYSPARK_PYTHON" "$@" +if [[ "$IPYTHON" = "1" ]] ; then + export PYSPARK_PYTHON="ipython" + exec "$PYSPARK_PYTHON" -i -c "%run $PYTHONSTARTUP" +else + exec "$PYSPARK_PYTHON" "$@" +fi From d55e3aa467ab7d406739255bd8dc3dfc60f3cb16 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 7 Feb 2013 13:59:18 -0800 Subject: [PATCH 176/696] Updated JavaStreamingContext with updated kafkaStream API. --- .../api/java/JavaStreamingContext.scala | 26 +++++++------------ 1 file changed, 9 insertions(+), 17 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index f82e6a37ccb95..70d6bd2b1b5ea 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -45,27 +45,24 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create an input stream that pulls messages form a Kafka Broker. - * @param hostname Zookeper hostname. - * @param port Zookeper port. + * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. */ def kafkaStream[T]( - hostname: String, - port: Int, + zkQuorum: String, groupId: String, topics: JMap[String, JInt]) : JavaDStream[T] = { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.kafkaStream[T](hostname, port, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + ssc.kafkaStream[T](zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) } /** * Create an input stream that pulls messages form a Kafka Broker. - * @param hostname Zookeper hostname. - * @param port Zookeper port. + * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. @@ -73,8 +70,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * By default the value is pulled from zookeper. */ def kafkaStream[T]( - hostname: String, - port: Int, + zkQuorum: String, groupId: String, topics: JMap[String, JInt], initialOffsets: JMap[KafkaPartitionKey, JLong]) @@ -82,8 +78,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] ssc.kafkaStream[T]( - hostname, - port, + zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), Map(initialOffsets.mapValues(_.longValue()).toSeq: _*)) @@ -91,8 +86,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create an input stream that pulls messages form a Kafka Broker. - * @param hostname Zookeper hostname. - * @param port Zookeper port. + * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. @@ -101,8 +95,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel RDD storage level. Defaults to memory-only */ def kafkaStream[T]( - hostname: String, - port: Int, + zkQuorum: String, groupId: String, topics: JMap[String, JInt], initialOffsets: JMap[KafkaPartitionKey, JLong], @@ -111,8 +104,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] ssc.kafkaStream[T]( - hostname, - port, + zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), Map(initialOffsets.mapValues(_.longValue()).toSeq: _*), From 291dd47c7f702f1229f82b111126f5f64b29d0c6 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 8 Feb 2013 14:34:07 +0530 Subject: [PATCH 177/696] Taking FeederActor out as seperate program --- .../streaming/examples/ActorWordCount.scala | 66 ++++++++++++------- 1 file changed, 44 insertions(+), 22 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 0ffa45a4c6ac2..71b4e5bf1a80c 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -49,25 +49,24 @@ class FeederActor extends Actor { case SubscribeReceiver(receiverActor: ActorRef) => println("received subscribe from %s".format(receiverActor.toString)) - receivers = LinkedList(receiverActor) ++ receivers + receivers = LinkedList(receiverActor) ++ receivers case UnsubscribeReceiver(receiverActor: ActorRef) => println("received unsubscribe from %s".format(receiverActor.toString)) - receivers = receivers.dropWhile(x => x eq receiverActor) + receivers = receivers.dropWhile(x => x eq receiverActor) } } /** - * A sample actor as receiver is also simplest. This receiver actor + * A sample actor as receiver, is also simplest. This receiver actor * goes and subscribe to a typical publisher/feeder actor and receives - * data, thus it is important to have feeder running before this example - * can be run. + * data. * * @see [[spark.streaming.examples.FeederActor]] */ class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) - extends Actor with Receiver { +extends Actor with Receiver { lazy private val remotePublisher = context.actorFor(urlOfPublisher) @@ -81,13 +80,42 @@ class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) } +/** + * A sample feeder actor + * + * Usage: FeederActor + * and describe the AkkaSystem that Spark Sample feeder would start on. + */ +object FeederActor { + + def main(args: Array[String]) { + if(args.length < 2){ + System.err.println( + "Usage: FeederActor \n" + ) + System.exit(1) + } + val Seq(host, port) = args.toSeq + + + val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1 + val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") + + println("Feeder started as:" + feeder) + + actorSystem.awaitTermination(); + } +} + /** * A sample word count program demonstrating the use of plugging in * Actor as Receiver * Usage: ActorWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. - * and describe the AkkaSystem that Spark Sample feeder would work on. - * + * and describe the AkkaSystem that Spark Sample feeder is running on. + * + * To run this example locally, you may run Feeder Actor as + * `$ ./run spark.streaming.examples.FeederActor 127.0.1.1 9999` * and then run the example * `$ ./run spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ @@ -96,7 +124,7 @@ object ActorWordCount { if (args.length < 3) { System.err.println( "Usage: ActorWordCount " + - "In local mode, should be 'local[n]' with n > 1") + "In local mode, should be 'local[n]' with n > 1") System.exit(1) } @@ -104,21 +132,16 @@ object ActorWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ActorWordCount", - Seconds(10)) + Seconds(10)) - //Start feeder actor on this actor system. - val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1 - - val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") - - /* + /* * Following is the use of actorStream to plug in custom actor as receiver - * + * * An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e type of data received and InputDstream + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e type of data received and InputDstream * should be same. - * + * * For example: Both actorStream and SampleActorReceiver are parameterized * to same type to ensure type safety. */ @@ -127,10 +150,9 @@ object ActorWordCount { Props(new SampleActorReceiver[String]("akka://spark@%s:%s/user/FeederActor".format( host, port.toInt))), "SampleReceiver") - //compute wordcount + //compute wordcount lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print() ssc.start() - } } From 04e828f7c13b83eff84cd2e57a57a821898b5e51 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 8 Feb 2013 19:07:36 -0800 Subject: [PATCH 178/696] general fixes to Distribution, plus some tests --- .../main/scala/spark/util/Distribution.scala | 34 +++++++++++++------ .../scala/spark/util/DistributionSuite.scala | 25 ++++++++++++++ 2 files changed, 48 insertions(+), 11 deletions(-) create mode 100644 core/src/test/scala/spark/util/DistributionSuite.scala diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala index 99bbb43ff1de0..ccd9232c8b98d 100644 --- a/core/src/main/scala/spark/util/Distribution.scala +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -6,8 +6,11 @@ import java.io.PrintStream * util for getting some stats from a small sample of numeric values, with some handy summary functions * * Entirely in memory, not intended as a good way to compute stats over large data sets. + * + * assumes you are giving it a non-empty set of data */ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { + require(startIdx < endIdx) def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) val length = endIdx - startIdx @@ -19,34 +22,43 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) * given from 0 to 1 * @param probabilities */ - def getQuantiles(probabilities: Traversable[Double]) = { - probabilities.map{q =>data((q * length).toInt + startIdx)} + def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) = { + probabilities.toIndexedSeq.map{p:Double => data(closestIndex(p))} } - def showQuantiles(out: PrintStream = System.out, probabilities: Traversable[Double] = defaultProbabilities) = { - out.println("min\t25%\t50%\t75%max") - probabilities.foreach{q => out.print(q + "\t")} - out.println + private def closestIndex(p: Double) = { + math.min((p * length).toInt + startIdx, endIdx - 1) } - def summary : (StatCounter, Traversable[Double]) = { - (StatCounter(data), getQuantiles(defaultProbabilities)) + def showQuantiles(out: PrintStream = System.out) = { + out.println("min\t25%\t50%\t75%\tmax") + getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")} + out.println } + def statCounter = StatCounter(data.slice(startIdx, endIdx)) + /** * print a summary of this distribution to the given PrintStream. * @param out */ def summary(out: PrintStream = System.out) { - val (statCounter, quantiles) = summary out.println(statCounter) - Distribution.showQuantiles(out, quantiles) + showQuantiles(out) } } object Distribution { + + def apply(data: Traversable[Double]): Option[Distribution] = { + if (data.size > 0) + Some(new Distribution(data)) + else + None + } + def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) { - out.println("min\t25%\t50%\t75%max") + out.println("min\t25%\t50%\t75%\tmax") quantiles.foreach{q => out.print(q + "\t")} out.println } diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/spark/util/DistributionSuite.scala new file mode 100644 index 0000000000000..cc6249b1dda8d --- /dev/null +++ b/core/src/test/scala/spark/util/DistributionSuite.scala @@ -0,0 +1,25 @@ +package spark.util + +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers + +/** + * + */ + +class DistributionSuite extends FunSuite with ShouldMatchers { + test("summary") { + val d = new Distribution((1 to 100).toArray.map{_.toDouble}) + val stats = d.statCounter + stats.count should be (100) + stats.mean should be (50.5) + stats.sum should be (50 * 101) + + val quantiles = d.getQuantiles() + quantiles(0) should be (1) + quantiles(1) should be (26) + quantiles(2) should be (51) + quantiles(3) should be (76) + quantiles(4) should be (100) + } +} From da52b16b38a5d6200ef2c6a3b7ba28ddf35a30f8 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 9 Feb 2013 10:11:54 -0600 Subject: [PATCH 179/696] Remove RDD.coalesce default arguments. --- core/src/main/scala/spark/RDD.scala | 3 +-- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 5 ----- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 5f99591fd5458..dea52eb5c6de8 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -235,8 +235,7 @@ abstract class RDD[T: ClassManifest]( /** * Return a new RDD that is reduced into `numSplits` partitions. */ - def coalesce(numSplits: Int = sc.defaultParallelism): RDD[T] = - new CoalescedRDD(this, numSplits) + def coalesce(numSplits: Int): RDD[T] = new CoalescedRDD(this, numSplits) /** * Return a sampled subset of this RDD. diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 295eaa57c0c2e..d3a4b62553c7f 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -130,11 +130,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest, other.classManifest) - /** - * Return a new RDD that is reduced into the default number of partitions. - */ - def coalesce(): RDD[T] = coalesce(rdd.context.defaultParallelism) - /** * Return a new RDD that is reduced into `numSplits` partitions. */ From 2a18cd826c42d7c6b35eaedde1e4c423b6a1b1e5 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 9 Feb 2013 10:12:04 -0600 Subject: [PATCH 180/696] Add back return types. --- core/src/main/scala/spark/rdd/BlockRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/CartesianRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 12 +++++++----- core/src/main/scala/spark/rdd/FilteredRDD.scala | 2 +- core/src/main/scala/spark/rdd/FlatMappedRDD.scala | 2 +- core/src/main/scala/spark/rdd/GlommedRDD.scala | 2 +- core/src/main/scala/spark/rdd/HadoopRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/MapPartitionsRDD.scala | 2 +- .../scala/spark/rdd/MapPartitionsWithSplitRDD.scala | 2 +- core/src/main/scala/spark/rdd/MappedRDD.scala | 2 +- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 4 ++-- .../main/scala/spark/rdd/PartitionPruningRDD.scala | 2 +- core/src/main/scala/spark/rdd/PipedRDD.scala | 2 +- core/src/main/scala/spark/rdd/SampledRDD.scala | 6 +++--- core/src/main/scala/spark/rdd/ShuffledRDD.scala | 4 +++- core/src/main/scala/spark/rdd/UnionRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/ZippedRDD.scala | 2 +- 19 files changed, 35 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 4214817c653a7..17989c5ce5d18 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -18,7 +18,7 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St HashMap(blockIds.zip(locations):_*) } - override def getSplits = (0 until blockIds.size).map(i => { + override def getSplits: Array[Split] = (0 until blockIds.size).map(i => { new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split] }).toArray @@ -33,7 +33,7 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St } } - override def getPreferredLocations(split: Split) = + override def getPreferredLocations(split: Split): Seq[String] = locations_(split.asInstanceOf[BlockRDDSplit].blockId) } diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 2f572a1941dc2..41cbbd0093dea 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -35,7 +35,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( val numSplitsInRdd2 = rdd2.splits.size - override def getSplits = { + override def getSplits: Array[Split] = { // create the cross product split val array = new Array[Split](rdd1.splits.size * rdd2.splits.size) for (s1 <- rdd1.splits; s2 <- rdd2.splits) { @@ -45,7 +45,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( array } - override def getPreferredLocations(split: Split) = { + override def getPreferredLocations(split: Split): Seq[String] = { val currSplit = split.asInstanceOf[CartesianSplit] rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2) } diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 7cde523f113f5..3558d4673f216 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -20,7 +20,7 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) - override def getSplits = { + override def getSplits: Array[Split] = { val dirContents = fs.listStatus(new Path(checkpointPath)) val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted val numSplits = splitFiles.size diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index d31ce13706d39..0a1e2cbee0964 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -45,7 +45,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) private val aggr = new CoGroupAggregator - override def getDependencies = { + override def getDependencies: Seq[Dependency[_]] = { rdds.map { rdd => if (rdd.partitioner == Some(part)) { logInfo("Adding one-to-one dependency with " + rdd) @@ -58,7 +58,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) } } - override def getSplits = { + override def getSplits: Array[Split] = { val array = new Array[Split](part.numPartitions) for (i <- 0 until array.size) { // Each CoGroupSplit will have a dependency per contributing RDD diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index a1aa7a30b0e4e..fcd26da43abe5 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -31,7 +31,7 @@ class CoalescedRDD[T: ClassManifest]( maxPartitions: Int) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies - override def getSplits = { + override def getSplits: Array[Split] = { val prevSplits = prev.splits if (prevSplits.length < maxPartitions) { prevSplits.map(_.index).map{idx => new CoalescedRDDSplit(idx, prev, Array(idx)) } @@ -50,10 +50,12 @@ class CoalescedRDD[T: ClassManifest]( } } - override def getDependencies = Seq(new NarrowDependency(prev) { - def getParents(id: Int): Seq[Int] = - splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices - }) + override def getDependencies: Seq[Dependency[_]] = { + Seq(new NarrowDependency(prev) { + def getParents(id: Int): Seq[Int] = + splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices + }) + } override def clearDependencies() { super.clearDependencies() diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index 6dbe235bd9f15..93e398ea2b34a 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -7,7 +7,7 @@ private[spark] class FilteredRDD[T: ClassManifest]( f: T => Boolean) extends RDD[T](prev) { - override def getSplits = firstParent[T].splits + override def getSplits: Array[Split] = firstParent[T].splits override val partitioner = prev.partitioner // Since filter cannot change a partition's keys diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 1b604c66e2fa5..8c2a610593306 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -9,7 +9,7 @@ class FlatMappedRDD[U: ClassManifest, T: ClassManifest]( f: T => TraversableOnce[U]) extends RDD[U](prev) { - override def getSplits = firstParent[T].splits + override def getSplits: Array[Split] = firstParent[T].splits override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context).flatMap(f) diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index 051bffed192bc..70b9b4e34ed49 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -5,7 +5,7 @@ import spark.{RDD, Split, TaskContext} private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev) { - override def getSplits = firstParent[T].splits + override def getSplits: Array[Split] = firstParent[T].splits override def compute(split: Split, context: TaskContext) = Array(firstParent[T].iterator(split, context).toArray).iterator diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index cd948de967762..854993737bc47 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -47,7 +47,7 @@ class HadoopRDD[K, V]( // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) - override def getSplits = { + override def getSplits: Array[Split] = { val inputFormat = createInputFormat(conf) val inputSplits = inputFormat.getSplits(conf, minSplits) val array = new Array[Split](inputSplits.size) @@ -106,7 +106,7 @@ class HadoopRDD[K, V]( } } - override def getPreferredLocations(split: Split) = { + override def getPreferredLocations(split: Split): Seq[String] = { // TODO: Filtering out "localhost" in case of file:// URLs val hadoopSplit = split.asInstanceOf[HadoopSplit] hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost") diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index 073f7d7d2aad2..7b0b4525c7e62 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -13,7 +13,7 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None - override def getSplits = firstParent[T].splits + override def getSplits: Array[Split] = firstParent[T].splits override def compute(split: Split, context: TaskContext) = f(firstParent[T].iterator(split, context)) diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala index 2ddc3d01b647a..c6dc1080a9089 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala @@ -15,7 +15,7 @@ class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest]( preservesPartitioning: Boolean ) extends RDD[U](prev) { - override def getSplits = firstParent[T].splits + override def getSplits: Array[Split] = firstParent[T].splits override val partitioner = if (preservesPartitioning) prev.partitioner else None diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index 5466c9c657fcb..6074f411e3b48 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -6,7 +6,7 @@ private[spark] class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) extends RDD[U](prev) { - override def getSplits = firstParent[T].splits + override def getSplits: Array[Split] = firstParent[T].splits override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(split, context).map(f) diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 2d000f5c689c3..345ae79d74d0d 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -39,7 +39,7 @@ class NewHadoopRDD[K, V]( @transient private val jobId = new JobID(jobtrackerId, id) - override def getSplits = { + override def getSplits: Array[Split] = { val inputFormat = inputFormatClass.newInstance val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray @@ -83,7 +83,7 @@ class NewHadoopRDD[K, V]( } } - override def getPreferredLocations(split: Split) = { + override def getPreferredLocations(split: Split): Seq[String] = { val theSplit = split.asInstanceOf[NewHadoopSplit] theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") } diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index a50ce751718c4..d1553181c166d 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -37,6 +37,6 @@ class PartitionPruningRDD[T: ClassManifest]( override def compute(split: Split, context: TaskContext) = firstParent[T].iterator( split.asInstanceOf[PartitionPruningRDDSplit].parentSplit, context) - override protected def getSplits = + override protected def getSplits: Array[Split] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions } diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 6631f83510cb6..56032a8659a05 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -27,7 +27,7 @@ class PipedRDD[T: ClassManifest]( // using a standard StringTokenizer (i.e. by spaces) def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command)) - override def getSplits = firstParent[T].splits + override def getSplits: Array[Split] = firstParent[T].splits override def compute(split: Split, context: TaskContext): Iterator[String] = { val pb = new ProcessBuilder(command) diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index 81626d5009ac5..f2a144e2e025f 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -19,15 +19,15 @@ class SampledRDD[T: ClassManifest]( seed: Int) extends RDD[T](prev) { - override def getSplits = { + override def getSplits: Array[Split] = { val rg = new Random(seed) firstParent[T].splits.map(x => new SampledRDDSplit(x, rg.nextInt)) } - override def getPreferredLocations(split: Split) = + override def getPreferredLocations(split: Split): Seq[String] = firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev) - override def compute(splitIn: Split, context: TaskContext) = { + override def compute(splitIn: Split, context: TaskContext): Iterator[T] = { val split = splitIn.asInstanceOf[SampledRDDSplit] if (withReplacement) { // For large datasets, the expected number of occurrences of each element in a sample with diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index d396478673615..bf69b5150bcc4 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -22,7 +22,9 @@ class ShuffledRDD[K, V]( override val partitioner = Some(part) - override def getSplits = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) + override def getSplits: Array[Split] = { + Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) + } override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 5ac24d2ffc7ca..ebc006822804f 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -28,7 +28,7 @@ class UnionRDD[T: ClassManifest]( @transient var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies - override def getSplits = { + override def getSplits: Array[Split] = { val array = new Array[Split](rdds.map(_.splits.size).sum) var pos = 0 for (rdd <- rdds; split <- rdd.splits) { @@ -38,7 +38,7 @@ class UnionRDD[T: ClassManifest]( array } - override def getDependencies = { + override def getDependencies: Seq[Dependency[_]] = { val deps = new ArrayBuffer[Dependency[_]] var pos = 0 for (rdd <- rdds) { diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index a079720a937a0..1ce70268bb491 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -31,7 +31,7 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( var rdd2: RDD[U]) extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) { - override def getSplits = { + override def getSplits: Array[Split] = { if (rdd1.splits.size != rdd2.splits.size) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } From b7a1fb5c5dfe12d18fe9c4b1da1818a143cd247f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sat, 9 Feb 2013 12:14:11 -0800 Subject: [PATCH 181/696] Add commutative requirement for 'reduce' to Python docstring. --- python/pyspark/rdd.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4cda6cf661197..6b6ab6abd97e9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -274,8 +274,8 @@ def _collect_iterator_through_file(self, iterator): def reduce(self, f): """ - Reduces the elements of this RDD using the specified associative binary - operator. + Reduces the elements of this RDD using the specified commutative and + associative binary operator. >>> from operator import add >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add) From fb7599870f4e3ee4e5a1e3c6e74ac2eaa2aaabf0 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 9 Feb 2013 16:10:52 -0600 Subject: [PATCH 182/696] Fix JavaRDDLike.coalesce return type. --- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index d3a4b62553c7f..9e52c224dd796 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -133,7 +133,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround /** * Return a new RDD that is reduced into `numSplits` partitions. */ - def coalesce(numSplits: Int): RDD[T] = rdd.coalesce(numSplits) + def coalesce(numSplits: Int): JavaRDD[T] = rdd.coalesce(numSplits) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements From 921be765339ac6a1b1a12672d73620855984eade Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 9 Feb 2013 10:58:47 -0600 Subject: [PATCH 183/696] Use stubs instead of mocks for DAGSchedulerSuite. --- .../main/scala/spark/MapOutputTracker.scala | 18 +- core/src/main/scala/spark/SparkEnv.scala | 31 +- .../scala/spark/storage/BlockManager.scala | 4 +- .../spark/storage/BlockManagerMaster.scala | 24 +- .../scala/spark/storage/ThreadingTest.scala | 5 +- .../scala/spark/MapOutputTrackerSuite.scala | 62 +- .../spark/scheduler/DAGSchedulerSuite.scala | 633 +++++------------- .../spark/storage/BlockManagerSuite.scala | 3 +- 8 files changed, 253 insertions(+), 527 deletions(-) diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 473520758557d..866d630a6d27b 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -38,9 +38,10 @@ private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Ac } } -private[spark] class MapOutputTracker(actorSystem: ActorSystem, isDriver: Boolean) extends Logging { +private[spark] class MapOutputTracker extends Logging { - val timeout = 10.seconds + // Set to the MapOutputTrackerActor living on the driver + var trackerActor: ActorRef = _ var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] @@ -53,24 +54,13 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isDriver: Boolea var cacheGeneration = generation val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] - val actorName: String = "MapOutputTracker" - var trackerActor: ActorRef = if (isDriver) { - val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName) - logInfo("Registered MapOutputTrackerActor actor") - actor - } else { - val ip = System.getProperty("spark.driver.host", "localhost") - val port = System.getProperty("spark.driver.port", "7077").toInt - val url = "akka://spark@%s:%s/user/%s".format(ip, port, actorName) - actorSystem.actorFor(url) - } - val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup) // Send a message to the trackerActor and get its result within a default timeout, or // throw a SparkException if this fails. def askTracker(message: Any): Any = { try { + val timeout = 10.seconds val future = trackerActor.ask(message)(timeout) return Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index d2193ae72b3d0..7157fd26883d3 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,7 +1,6 @@ package spark -import akka.actor.ActorSystem -import akka.actor.ActorSystemImpl +import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider import serializer.Serializer @@ -83,11 +82,23 @@ object SparkEnv extends Logging { } val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") + + def registerOrLookup(name: String, newActor: => Actor): ActorRef = { + if (isDriver) { + logInfo("Registering " + name) + actorSystem.actorOf(Props(newActor), name = name) + } else { + val driverIp: String = System.getProperty("spark.driver.host", "localhost") + val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt + val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, name) + logInfo("Connecting to " + name + ": " + url) + actorSystem.actorFor(url) + } + } - val driverIp: String = System.getProperty("spark.driver.host", "localhost") - val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt - val blockManagerMaster = new BlockManagerMaster( - actorSystem, isDriver, isLocal, driverIp, driverPort) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( + "BlockManagerMaster", + new spark.storage.BlockManagerMasterActor(isLocal))) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) val connectionManager = blockManager.connectionManager @@ -99,7 +110,12 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - val mapOutputTracker = new MapOutputTracker(actorSystem, isDriver) + // Have to assign trackerActor after initialization as MapOutputTrackerActor + // requires the MapOutputTracker itself + val mapOutputTracker = new MapOutputTracker() + mapOutputTracker.trackerActor = registerOrLookup( + "MapOutputTracker", + new MapOutputTrackerActor(mapOutputTracker)) val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher") @@ -137,4 +153,5 @@ object SparkEnv extends Logging { httpFileServer, sparkFilesDir) } + } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9893e9625d909..d9838f65ab421 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -88,7 +88,7 @@ class BlockManager( val host = System.getProperty("spark.hostname", Utils.localHostName()) - val slaveActor = master.actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), + val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) // Pending reregistration action being executed asynchronously or null if none @@ -946,7 +946,7 @@ class BlockManager( heartBeatTask.cancel() } connectionManager.stop() - master.actorSystem.stop(slaveActor) + actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diskStore.clear() diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 7389bee1509d4..991ae61712bdb 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -15,32 +15,12 @@ import akka.util.duration._ import spark.{Logging, SparkException, Utils} -private[spark] class BlockManagerMaster( - val actorSystem: ActorSystem, - isDriver: Boolean, - isLocal: Boolean, - driverIp: String, - driverPort: Int) - extends Logging { +private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt - val DRIVER_AKKA_ACTOR_NAME = "BlockMasterManager" - val timeout = 10.seconds - var driverActor: ActorRef = { - if (isDriver) { - val driverActor = actorSystem.actorOf(Props(new BlockManagerMasterActor(isLocal)), - name = DRIVER_AKKA_ACTOR_NAME) - logInfo("Registered BlockManagerMaster Actor") - driverActor - } else { - val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, DRIVER_AKKA_ACTOR_NAME) - logInfo("Connecting to BlockManagerMaster: " + url) - actorSystem.actorFor(url) - } - } /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { @@ -59,7 +39,7 @@ private[spark] class BlockManagerMaster( /** Register the BlockManager's id with the driver. */ def registerBlockManager( - blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) logInfo("Registered BlockManager") diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index a70d1c8e78e10..5c406e68cb2ac 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -75,9 +75,8 @@ private[spark] object ThreadingTest { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") val serializer = new KryoSerializer - val driverIp: String = System.getProperty("spark.driver.host", "localhost") - val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt - val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true, driverIp, driverPort) + val blockManagerMaster = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index dd19442dcb4ad..b49cd81e235de 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -31,13 +31,15 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTracker(actorSystem, true) + val tracker = new MapOutputTracker() + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTracker(actorSystem, true) + val tracker = new MapOutputTracker() + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -55,7 +57,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTracker(actorSystem, true) + val tracker = new MapOutputTracker() + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -77,35 +80,34 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { } test("remote fetch") { - try { - System.clearProperty("spark.driver.host") // In case some previous test had set it - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) - System.setProperty("spark.driver.port", boundPort.toString) - val masterTracker = new MapOutputTracker(actorSystem, true) - val slaveTracker = new MapOutputTracker(actorSystem, false) - masterTracker.registerShuffle(10, 1) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) + val masterTracker = new MapOutputTracker() + masterTracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(masterTracker))) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0) + val slaveTracker = new MapOutputTracker() + slaveTracker.trackerActor = slaveSystem.actorFor("akka://spark@localhost:" + boundPort) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val size1000 = MapOutputTracker.decompressSize(compressedSize1000) + masterTracker.registerMapOutput(10, 0, new MapStatus( + BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) - masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + masterTracker.incrementGeneration() + slaveTracker.updateGeneration(masterTracker.getGeneration) + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - // failure should be cached - intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - } finally { - System.clearProperty("spark.driver.port") - } + // failure should be cached + intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } } } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 83663ac702a5b..edc5a7dfbaa86 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -4,16 +4,6 @@ import scala.collection.mutable.{Map, HashMap} import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.TimeLimitedTests -import org.scalatest.mock.EasyMockSugar -import org.scalatest.time.{Span, Seconds} - -import org.easymock.EasyMock._ -import org.easymock.Capture -import org.easymock.EasyMock -import org.easymock.{IAnswer, IArgumentMatcher} - -import akka.actor.ActorSystem import spark.storage.BlockManager import spark.storage.BlockManagerId @@ -42,27 +32,26 @@ import spark.{FetchFailed, Success} * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) * and capturing the resulting TaskSets from the mock TaskScheduler. */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar with TimeLimitedTests { +class DAGSchedulerSuite extends FunSuite with BeforeAndAfter { + + val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") - // impose a time limit on this test in case we don't let the job finish, in which case - // JobWaiter#getResult will hang. - override val timeLimit = Span(5, Seconds) + /** Set of TaskSets the DAGScheduler has requested executed. */ + val taskSets = scala.collection.mutable.Buffer[TaskSet]() + val taskScheduler = new TaskScheduler() { + override def start() = {} + override def stop() = {} + override def submitTasks(taskSet: TaskSet) = { + // normally done by TaskSetManager + taskSet.tasks.foreach(_.generation = mapOutputTracker.getGeneration) + taskSets += taskSet + } + override def setListener(listener: TaskSchedulerListener) = {} + override def defaultParallelism() = 2 + } - val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") - var scheduler: DAGScheduler = null - val taskScheduler = mock[TaskScheduler] - val blockManagerMaster = mock[BlockManagerMaster] var mapOutputTracker: MapOutputTracker = null - var schedulerThread: Thread = null - var schedulerException: Throwable = null - - /** - * Set of EasyMock argument matchers that match a TaskSet for a given RDD. - * We cache these so we do not create duplicate matchers for the same RDD. - * This allows us to easily setup a sequence of expectations for task sets for - * that RDD. - */ - val taskSetMatchers = new HashMap[MyRDD, IArgumentMatcher] + var scheduler: DAGScheduler = null /** * Set of cache locations to return from our mock BlockManagerMaster. @@ -70,68 +59,46 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * list of cache locations silently. */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] - - /** - * JobWaiter for the last JobSubmitted event we pushed. To keep tests (most of which - * will only submit one job) from needing to explicitly track it. - */ - var lastJobWaiter: JobWaiter[Int] = null - - /** - * Array into which we are accumulating the results from the last job asynchronously. - */ - var lastJobResult: Array[Int] = null - - /** - * Tell EasyMockSugar what mock objects we want to be configured by expecting {...} - * and whenExecuting {...} */ - implicit val mocks = MockObjects(taskScheduler, blockManagerMaster) - - /** - * Utility function to reset mocks and set expectations on them. EasyMock wants mock objects - * to be reset after each time their expectations are set, and we tend to check mock object - * calls over a single call to DAGScheduler. - * - * We also set a default expectation here that blockManagerMaster.getLocations can be called - * and will return values from cacheLocations. - */ - def resetExpecting(f: => Unit) { - reset(taskScheduler) - reset(blockManagerMaster) - expecting { - expectGetLocations() - f + // stub out BlockManagerMaster.getLocations to use our cacheLocations + val blockManagerMaster = new BlockManagerMaster(null) { + override def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = { + blockIds.map { name => + val pieces = name.split("_") + if (pieces(0) == "rdd") { + val key = pieces(1).toInt -> pieces(2).toInt + cacheLocations.getOrElse(key, Seq()) + } else { + Seq() + } + }.toSeq + } + override def removeExecutor(execId: String) { + // don't need to propagate to the driver, which we don't have + } } + + /** The list of results that DAGScheduler has collected. */ + val results = new HashMap[Int, Any]() + var failure: Exception = _ + val listener = new JobListener() { + override def taskSucceeded(index: Int, result: Any) = results.put(index, result) + override def jobFailed(exception: Exception) = { failure = exception } } before { - taskSetMatchers.clear() + taskSets.clear() cacheLocations.clear() - val actorSystem = ActorSystem("test") - mapOutputTracker = new MapOutputTracker(actorSystem, true) - resetExpecting { - taskScheduler.setListener(anyObject()) - } - whenExecuting { - scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) - } + results.clear() + mapOutputTracker = new MapOutputTracker() + scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) } after { - assert(scheduler.processEvent(StopDAGScheduler)) - resetExpecting { - taskScheduler.stop() - } - whenExecuting { - scheduler.stop() - } + scheduler.stop() sc.stop() System.clearProperty("spark.master.port") } - def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) - /** * Type of RDD we use for testing. Note that we should never call the real RDD compute methods. * This is a pair RDD type so it can always be used in ShuffleDependencies. @@ -143,7 +110,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * preferredLocations (if any) that are passed to them. They are deliberately not executable * so we can test that DAGScheduler does not try to execute RDDs locally. */ - def makeRdd( + private def makeRdd( numSplits: Int, dependencies: List[Dependency[_]], locations: Seq[Seq[String]] = Nil @@ -164,55 +131,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } - /** - * EasyMock matcher method. For use as an argument matcher for a TaskSet whose first task - * is from a particular RDD. - */ - def taskSetForRdd(rdd: MyRDD): TaskSet = { - val matcher = taskSetMatchers.getOrElseUpdate(rdd, - new IArgumentMatcher { - override def matches(actual: Any): Boolean = { - val taskSet = actual.asInstanceOf[TaskSet] - taskSet.tasks(0) match { - case rt: ResultTask[_, _] => rt.rdd.id == rdd.id - case smt: ShuffleMapTask => smt.rdd.id == rdd.id - case _ => false - } - } - override def appendTo(buf: StringBuffer) { - buf.append("taskSetForRdd(" + rdd + ")") - } - }) - EasyMock.reportMatcher(matcher) - return null - } - - /** - * Setup an EasyMock expectation to repsond to blockManagerMaster.getLocations() called from - * cacheLocations. - */ - def expectGetLocations(): Unit = { - EasyMock.expect(blockManagerMaster.getLocations(anyObject().asInstanceOf[Array[String]])). - andAnswer(new IAnswer[Seq[Seq[BlockManagerId]]] { - override def answer(): Seq[Seq[BlockManagerId]] = { - val blocks = getCurrentArguments()(0).asInstanceOf[Array[String]] - return blocks.map { name => - val pieces = name.split("_") - if (pieces(0) == "rdd") { - val key = pieces(1).toInt -> pieces(2).toInt - if (cacheLocations.contains(key)) { - cacheLocations(key) - } else { - Seq[BlockManagerId]() - } - } else { - Seq[BlockManagerId]() - } - }.toSeq - } - }).anyTimes() - } - /** * Process the supplied event as if it were the top of the DAGScheduler event queue, expecting * the scheduler not to exit. @@ -220,48 +138,21 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * After processing the event, submit waiting stages as is done on most iterations of the * DAGScheduler event loop. */ - def runEvent(event: DAGSchedulerEvent) { + private def runEvent(event: DAGSchedulerEvent) { assert(!scheduler.processEvent(event)) scheduler.submitWaitingStages() } /** - * Expect a TaskSet for the specified RDD to be submitted to the TaskScheduler. Should be - * called from a resetExpecting { ... } block. - * - * Returns a easymock Capture that will contain the task set after the stage is submitted. - * Most tests should use interceptStage() instead of this directly. - */ - def expectStage(rdd: MyRDD): Capture[TaskSet] = { - val taskSetCapture = new Capture[TaskSet] - taskScheduler.submitTasks(and(capture(taskSetCapture), taskSetForRdd(rdd))) - return taskSetCapture - } - - /** - * Expect the supplied code snippet to submit a stage for the specified RDD. - * Return the resulting TaskSet. First marks all the tasks are belonging to the - * current MapOutputTracker generation. + * When we submit dummy Jobs, this is the compute function we supply. Except in a local test + * below, we do not expect this function to ever be executed; instead, we will return results + * directly through CompletionEvents. */ - def interceptStage(rdd: MyRDD)(f: => Unit): TaskSet = { - var capture: Capture[TaskSet] = null - resetExpecting { - capture = expectStage(rdd) - } - whenExecuting { - f - } - val taskSet = capture.getValue - for (task <- taskSet.tasks) { - task.generation = mapOutputTracker.getGeneration - } - return taskSet - } + private val jobComputeFunc = (context: TaskContext, it: Iterator[(_)]) => + it.next.asInstanceOf[Tuple2[_, _]]._1 - /** - * Send the given CompletionEvent messages for the tasks in the TaskSet. - */ - def respondToTaskSet(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { + /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ + private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { @@ -269,108 +160,59 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } } - - /** - * Assert that the supplied TaskSet has exactly the given preferredLocations. - */ - def expectTaskSetLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { - assert(locations.size === taskSet.tasks.size) - for ((expectLocs, taskLocs) <- - taskSet.tasks.map(_.preferredLocations).zip(locations)) { - assert(expectLocs === taskLocs) - } + + /** Sends the rdd to the scheduler for scheduling. */ + private def submit( + rdd: RDD[_], + partitions: Array[Int], + func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, + allowLocal: Boolean = false, + listener: JobListener = listener) { + runEvent(JobSubmitted(rdd, func, partitions, allowLocal, null, listener)) } - - /** - * When we submit dummy Jobs, this is the compute function we supply. Except in a local test - * below, we do not expect this function to ever be executed; instead, we will return results - * directly through CompletionEvents. - */ - def jobComputeFunc(context: TaskContext, it: Iterator[(Int, Int)]): Int = - it.next._1.asInstanceOf[Int] - - - /** - * Start a job to compute the given RDD. Returns the JobWaiter that will - * collect the result of the job via callbacks from DAGScheduler. - */ - def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): (JobWaiter[Int], Array[Int]) = { - val resultArray = new Array[Int](rdd.splits.size) - val (toSubmit, waiter) = scheduler.prepareJob[(Int, Int), Int]( - rdd, - jobComputeFunc, - (0 to (rdd.splits.size - 1)), - "test-site", - allowLocal, - (i: Int, value: Int) => resultArray(i) = value - ) - lastJobWaiter = waiter - lastJobResult = resultArray - runEvent(toSubmit) - return (waiter, resultArray) - } - - /** - * Assert that a job we started has failed. - */ - def expectJobException(waiter: JobWaiter[Int] = lastJobWaiter) { - waiter.awaitResult() match { - case JobSucceeded => fail() - case JobFailed(_) => return - } - } - - /** - * Assert that a job we started has succeeded and has the given result. - */ - def expectJobResult(expected: Array[Int], waiter: JobWaiter[Int] = lastJobWaiter, - result: Array[Int] = lastJobResult) { - waiter.awaitResult match { - case JobSucceeded => - assert(expected === result) - case JobFailed(_) => - fail() - } + + /** Sends TaskSetFailed to the scheduler. */ + private def failed(taskSet: TaskSet, message: String) { + runEvent(TaskSetFailed(taskSet, message)) } - def makeMapStatus(host: String, reduces: Int): MapStatus = - new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) - test("zero split job") { val rdd = makeRdd(0, Nil) var numResults = 0 - def accumulateResult(partition: Int, value: Int) { - numResults += 1 + val fakeListener = new JobListener() { + override def taskSucceeded(partition: Int, value: Any) = numResults += 1 + override def jobFailed(exception: Exception) = throw exception } - scheduler.runJob(rdd, jobComputeFunc, Seq(), "test-site", false, accumulateResult) + submit(rdd, Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { val rdd = makeRdd(1, Nil) - val taskSet = interceptStage(rdd) { submitRdd(rdd) } - respondToTaskSet(taskSet, List( (Success, 42) )) - expectJobResult(Array(42)) + submit(rdd, Array(0)) + complete(taskSets(0), List((Success, 42))) + assert(results === Map(0 -> 42)) } test("local job") { val rdd = new MyRDD(sc, Nil) { - override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] = - Array(42 -> 0).iterator - override def getSplits() = Array( new Split { override def index = 0 } ) + override def compute(split: Split, context: TaskContext) = Array(42 -> 0).iterator + override def getSplits() = Array(new Split { override def index = 0 }) override def getPreferredLocations(split: Split) = Nil override def toString = "DAGSchedulerSuite Local RDD" } - submitRdd(rdd, true) - expectJobResult(Array(42)) + runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener)) + // this shouldn't be needed, but i haven't stubbed out runLocally yet + Thread.sleep(500) + assert(results === Map(0 -> 42)) } - + test("run trivial job w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) - val taskSet = interceptStage(finalRdd) { submitRdd(finalRdd) } - respondToTaskSet(taskSet, List( (Success, 42) )) - expectJobResult(Array(42)) + submit(finalRdd, Array(0)) + complete(taskSets(0), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } test("cache location preferences w/ dependency") { @@ -378,17 +220,17 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) cacheLocations(baseRdd.id -> 0) = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) - val taskSet = interceptStage(finalRdd) { submitRdd(finalRdd) } - expectTaskSetLocations(taskSet, List(Seq("hostA", "hostB"))) - respondToTaskSet(taskSet, List( (Success, 42) )) - expectJobResult(Array(42)) + submit(finalRdd, Array(0)) + val taskSet = taskSets(0) + assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) + complete(taskSet, Seq((Success, 42))) + assert(results === Map(0 -> 42)) } test("trivial job failure") { - val rdd = makeRdd(1, Nil) - val taskSet = interceptStage(rdd) { submitRdd(rdd) } - runEvent(TaskSetFailed(taskSet, "test failure")) - expectJobException() + submit(makeRdd(1, Nil), Array(0)) + failed(taskSets(0), "some failure") + assert(failure.getMessage === "Job failed: some failure") } test("run trivial shuffle") { @@ -396,52 +238,39 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) - - val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } - val secondStage = interceptStage(reduceRdd) { - respondToTaskSet(firstStage, List( + submit(reduceRdd, Array(0)) + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } + (Success, makeMapStatus("hostB", 1)))) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) - respondToTaskSet(secondStage, List( (Success, 42) )) - expectJobResult(Array(42)) + complete(taskSets(1), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } - + test("run trivial shuffle with fetch failure") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) - - val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } - val secondStage = interceptStage(reduceRdd) { - respondToTaskSet(firstStage, List( + submit(reduceRdd, Array(0, 1)) + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(secondStage, List( + (Success, makeMapStatus("hostB", 1)))) + // the 2nd ResultTask failed + complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null) - )) - } - val thirdStage = interceptStage(shuffleMapRdd) { - scheduler.resubmitFailedStages() - } - val fourthStage = interceptStage(reduceRdd) { - respondToTaskSet(thirdStage, List( (Success, makeMapStatus("hostA", 1)) )) - } - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) - respondToTaskSet(fourthStage, List( (Success, 43) )) - expectJobResult(Array(42, 43)) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null))) + // this will get called + // blockManagerMaster.removeExecutor("exec-hostA") + // ask the scheduler to try it again + scheduler.resubmitFailedStages() + // have the 2nd attempt pass + complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + // we can see both result blocks now + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.ip) === Array("hostA", "hostB")) + complete(taskSets(3), Seq((Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) } test("ignore late map task completions") { @@ -449,33 +278,27 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) - - val taskSet = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } + submit(reduceRdd, Array(0, 1)) + // pretend we were told hostA went away val oldGeneration = mapOutputTracker.getGeneration - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - runEvent(ExecutorLost("exec-hostA")) - } + runEvent(ExecutorLost("exec-hostA")) val newGeneration = mapOutputTracker.getGeneration assert(newGeneration > oldGeneration) val noAccum = Map[Long, Any]() - // We rely on the event queue being ordered and increasing the generation number by 1 + val taskSet = taskSets(0) // should be ignored for being too old runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) // should work because it's a non-failed host runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum)) // should be ignored for being too old runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum)) + // should work because it's a new generation taskSet.tasks(1).generation = newGeneration - val secondStage = interceptStage(reduceRdd) { - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum)) - } + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) - respondToTaskSet(secondStage, List( (Success, 42), (Success, 43) )) - expectJobResult(Array(42, 43)) + complete(taskSets(1), Seq((Success, 42), (Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) } test("run trivial shuffle with out-of-band failure and retry") { @@ -483,76 +306,49 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) - - val firstStage = interceptStage(shuffleMapRdd) { submitRdd(reduceRdd) } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - runEvent(ExecutorLost("exec-hostA")) - } + submit(reduceRdd, Array(0)) + // blockManagerMaster.removeExecutor("exec-hostA") + // pretend we were told hostA went away + runEvent(ExecutorLost("exec-hostA")) // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks // rather than marking it is as failed and waiting. - val secondStage = interceptStage(shuffleMapRdd) { - respondToTaskSet(firstStage, List( + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - val thirdStage = interceptStage(reduceRdd) { - respondToTaskSet(secondStage, List( - (Success, makeMapStatus("hostC", 1)) - )) - } - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) - respondToTaskSet(thirdStage, List( (Success, 42) )) - expectJobResult(Array(42)) - } - - test("recursive shuffle failures") { + (Success, makeMapStatus("hostB", 1)))) + // have hostC complete the resubmitted task + complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + complete(taskSets(2), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + } + + test("recursive shuffle failures") { val shuffleOneRdd = makeRdd(2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) val finalRdd = makeRdd(1, List(shuffleDepTwo)) - - val firstStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } - val secondStage = interceptStage(shuffleTwoRdd) { - respondToTaskSet(firstStage, List( + submit(finalRdd, Array(0)) + // have the first stage complete normally + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)) - )) - } - val thirdStage = interceptStage(finalRdd) { - respondToTaskSet(secondStage, List( + (Success, makeMapStatus("hostB", 2)))) + // have the second stage complete normally + complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostC", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(thirdStage, List( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) - )) - } - val recomputeOne = interceptStage(shuffleOneRdd) { - scheduler.resubmitFailedStages() - } - val recomputeTwo = interceptStage(shuffleTwoRdd) { - respondToTaskSet(recomputeOne, List( - (Success, makeMapStatus("hostA", 2)) - )) - } - val finalStage = interceptStage(finalRdd) { - respondToTaskSet(recomputeTwo, List( - (Success, makeMapStatus("hostA", 1)) - )) - } - respondToTaskSet(finalStage, List( (Success, 42) )) - expectJobResult(Array(42)) + (Success, makeMapStatus("hostC", 1)))) + // fail the third stage because hostA went down + complete(taskSets(2), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) + // TODO assert this: + // blockManagerMaster.removeExecutor("exec-hostA") + // have DAGScheduler try again + scheduler.resubmitFailedStages() + complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) + complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(5), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } test("cached post-shuffle") { @@ -561,103 +357,44 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) val finalRdd = makeRdd(1, List(shuffleDepTwo)) - - val firstShuffleStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } + submit(finalRdd, Array(0)) cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) - val secondShuffleStage = interceptStage(shuffleTwoRdd) { - respondToTaskSet(firstShuffleStage, List( + // complete stage 2 + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)) - )) - } - val reduceStage = interceptStage(finalRdd) { - respondToTaskSet(secondShuffleStage, List( + (Success, makeMapStatus("hostB", 2)))) + // complete stage 1 + complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(reduceStage, List( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) - )) - } + (Success, makeMapStatus("hostB", 1)))) + // pretend stage 0 failed because hostA went down + complete(taskSets(2), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) + // TODO assert this: + // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. - val recomputeTwo = interceptStage(shuffleTwoRdd) { - scheduler.resubmitFailedStages() - } - expectTaskSetLocations(recomputeTwo, Seq(Seq("hostD"))) - val finalRetry = interceptStage(finalRdd) { - respondToTaskSet(recomputeTwo, List( - (Success, makeMapStatus("hostD", 1)) - )) - } - respondToTaskSet(finalRetry, List( (Success, 42) )) - expectJobResult(Array(42)) + scheduler.resubmitFailedStages() + assertLocations(taskSets(3), Seq(Seq("hostD"))) + // allow hostD to recover + complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) + complete(taskSets(4), Seq((Success, 42))) + assert(results === Map(0 -> 42)) } - test("cached post-shuffle but fails") { - val shuffleOneRdd = makeRdd(2, Nil) - val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) - val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) - - val firstShuffleStage = interceptStage(shuffleOneRdd) { submitRdd(finalRdd) } - cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) - cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) - val secondShuffleStage = interceptStage(shuffleTwoRdd) { - respondToTaskSet(firstShuffleStage, List( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)) - )) - } - val reduceStage = interceptStage(finalRdd) { - respondToTaskSet(secondShuffleStage, List( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)) - )) - } - resetExpecting { - blockManagerMaster.removeExecutor("exec-hostA") - } - whenExecuting { - respondToTaskSet(reduceStage, List( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null) - )) - } - val recomputeTwoCached = interceptStage(shuffleTwoRdd) { - scheduler.resubmitFailedStages() - } - expectTaskSetLocations(recomputeTwoCached, Seq(Seq("hostD"))) - intercept[FetchFailedException]{ - mapOutputTracker.getServerStatuses(shuffleDepOne.shuffleId, 0) + /** Assert that the supplied TaskSet has exactly the given preferredLocations. */ + private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { + assert(locations.size === taskSet.tasks.size) + for ((expectLocs, taskLocs) <- + taskSet.tasks.map(_.preferredLocations).zip(locations)) { + assert(expectLocs === taskLocs) } + } - // Simulate the shuffle input data failing to be cached. - cacheLocations.remove(shuffleTwoRdd.id -> 0) - respondToTaskSet(recomputeTwoCached, List( - (FetchFailed(null, shuffleDepOne.shuffleId, 0, 0), null) - )) + private def makeMapStatus(host: String, reduces: Int): MapStatus = + new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) - // After the fetch failure, DAGScheduler should recheck the cache and decide to resubmit - // everything. - val recomputeOne = interceptStage(shuffleOneRdd) { - scheduler.resubmitFailedStages() - } - // We use hostA here to make sure DAGScheduler doesn't think it's still dead. - val recomputeTwoUncached = interceptStage(shuffleTwoRdd) { - respondToTaskSet(recomputeOne, List( (Success, makeMapStatus("hostA", 1)) )) - } - expectTaskSetLocations(recomputeTwoUncached, Seq(Seq[String]())) - val finalRetry = interceptStage(finalRdd) { - respondToTaskSet(recomputeTwoUncached, List( (Success, makeMapStatus("hostA", 1)) )) + private def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) - } - respondToTaskSet(finalRetry, List( (Success, 42) )) - expectJobResult(Array(42)) - } -} +} \ No newline at end of file diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 2d177bbf67458..53450609a1f91 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -31,7 +31,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT before { actorSystem = ActorSystem("test") - master = new BlockManagerMaster(actorSystem, true, true, "localhost", 7077) + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true)))) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") From 99a5fc498acf3de14d754f8dda0df6bb81dd9595 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 9 Feb 2013 15:18:05 -0800 Subject: [PATCH 184/696] Added an initial spark job to ensure worker nodes are initialized. --- .../main/scala/spark/streaming/NetworkInputTracker.scala | 7 ++++++- .../src/test/scala/spark/streaming/InputStreamsSuite.scala | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index e4152f3a61bb5..b54f53b2033b8 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -4,6 +4,7 @@ import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} import spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} import spark.Logging import spark.SparkEnv +import spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.Queue @@ -138,8 +139,12 @@ class NetworkInputTracker( } iterator.next().start() } + // Run the dummy Spark job to ensure that all slaves have registered. + // This avoids all the receivers to be scheduled on the same node. + //ssc.sparkContext.makeRDD(1 to 100, 100).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() + // Distribute the receivers and start them - ssc.sc.runJob(tempRDD, startReceiver) + ssc.sparkContext.runJob(tempRDD, startReceiver) } /** Stops the receivers. */ diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index c442210004573..0eb9c7b81e29b 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -95,7 +95,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val input = Seq(1, 2, 3, 4, 5) - + Thread.sleep(1000) val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", 33333)); val client = SpecificRequestor.getClient( classOf[AvroSourceProtocol], transceiver); From 4619ee0787066da15628970bd55cb8cec31a372c Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 9 Feb 2013 20:05:42 -0600 Subject: [PATCH 185/696] Move JavaRDDLike.coalesce into the right places. --- core/src/main/scala/spark/api/java/JavaDoubleRDD.scala | 5 +++++ core/src/main/scala/spark/api/java/JavaPairRDD.scala | 5 +++++ core/src/main/scala/spark/api/java/JavaRDD.scala | 5 +++++ core/src/main/scala/spark/api/java/JavaRDDLike.scala | 5 ----- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 843e1bd18bdbf..2810631b4158a 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -52,6 +52,11 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav def filter(f: JFunction[Double, java.lang.Boolean]): JavaDoubleRDD = fromRDD(srdd.filter(x => f(x).booleanValue())) + /** + * Return a new RDD that is reduced into `numSplits` partitions. + */ + def coalesce(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numSplits)) + /** * Return a sampled subset of this RDD. */ diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 8ce32e0e2fd21..8a123bdb47297 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -62,6 +62,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif def filter(f: Function[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue())) + /** + * Return a new RDD that is reduced into `numSplits` partitions. + */ + def coalesce(numSplits: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.coalesce(numSplits)) + /** * Return a sampled subset of this RDD. */ diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index ac31350ec3374..23e7ae2726f64 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -38,6 +38,11 @@ JavaRDDLike[T, JavaRDD[T]] { def filter(f: JFunction[T, java.lang.Boolean]): JavaRDD[T] = wrapRDD(rdd.filter((x => f(x).booleanValue()))) + /** + * Return a new RDD that is reduced into `numSplits` partitions. + */ + def coalesce(numSplits: Int): JavaRDD[T] = rdd.coalesce(numSplits) + /** * Return a sampled subset of this RDD. */ diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 9e52c224dd796..60025b459c383 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -130,11 +130,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest, other.classManifest) - /** - * Return a new RDD that is reduced into `numSplits` partitions. - */ - def coalesce(numSplits: Int): JavaRDD[T] = rdd.coalesce(numSplits) - /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. From 1859c9f93c409a355b404d24b5632b3822ad42c1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 9 Feb 2013 21:55:17 -0800 Subject: [PATCH 186/696] Changing to use Timer based on code review --- .../scheduler/cluster/ClusterScheduler.scala | 51 ++++++++----------- .../scheduler/cluster/TaskSetManager.scala | 8 --- 2 files changed, 20 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index aed9826377ddf..04d01e9ce8d36 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -11,6 +11,7 @@ import spark.TaskState.TaskState import spark.scheduler._ import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong +import java.util.{TimerTask, Timer} /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call @@ -22,8 +23,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // How often to check for speculative tasks val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong - // How often to check for starved TaskSets - val STARVATION_CHECK_INTERVAL = System.getProperty("spark.starvation_check.interval", "5000").toLong + // Threshold above which we warn user initial TaskSet may be starved + val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "5000").toLong val activeTaskSets = new HashMap[String, TaskSetManager] var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -32,6 +33,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val taskIdToExecutorId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] + var hasReceivedTask = false + var hasLaunchedTask = false + val starvationTimer = new Timer(true) + // Incrementing Mesos task IDs val nextTaskId = new AtomicLong(0) @@ -86,21 +91,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } }.start() } - - new Thread("ClusterScheduler starvation check") { - setDaemon(true) - - override def run() { - while (true) { - try { - Thread.sleep(STARVATION_CHECK_INTERVAL) - } catch { - case e: InterruptedException => {} - } - detectStarvedTaskSets() - } - } - }.start() } override def submitTasks(taskSet: TaskSet) { @@ -111,6 +101,18 @@ private[spark] class ClusterScheduler(val sc: SparkContext) activeTaskSets(taskSet.id) = manager activeTaskSetsQueue += manager taskSetTaskIds(taskSet.id) = new HashSet[Long]() + + if (hasReceivedTask == false) { + starvationTimer.scheduleAtFixedRate(new TimerTask() { + override def run() { + if (!hasLaunchedTask) { + logWarning("Initial TaskSet has not accepted any offers. " + + "Check the scheduler UI to ensure slaves are registered.") + } + } + }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + } + hasReceivedTask = true; } backend.reviveOffers() } @@ -167,6 +169,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } while (launchedTask) } + if (tasks.size > 0) hasLaunchedTask = true return tasks } } @@ -266,20 +269,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - // Find and resource-starved TaskSets and alert the user - def detectStarvedTaskSets() { - val noOfferThresholdSeconds = 5 - synchronized { - for (ts <- activeTaskSetsQueue) { - if (ts == TaskSetManager.firstTaskSet.get && - (System.currentTimeMillis - ts.creationTime > noOfferThresholdSeconds * 1000) && - ts.receivedOffers == 0) { - logWarning("No offers received. Check the scheduler UI to ensure slaves are registered.") - } - } - } - } - def executorLost(executorId: String, reason: ExecutorLossReason) { var failedExecutor: Option[String] = None synchronized { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 58c5d4553e7ce..584cfdff45a1d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -44,7 +44,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 val creationTime = System.currentTimeMillis - var receivedOffers = 0 // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -98,8 +97,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } - if (!TaskSetManager.firstTaskSet.isDefined) TaskSetManager.firstTaskSet = Some(this) - // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive @@ -192,7 +189,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Respond to an offer of a single slave from the scheduler by finding a task def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { - receivedOffers += 1 if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -432,7 +428,3 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return foundTasks } } - -object TaskSetManager { - var firstTaskSet: Option[TaskSetManager] = None -} From 2ed791fd7fa193ea7e10d70e1c1b0787d584b0fd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 9 Feb 2013 21:59:01 -0800 Subject: [PATCH 187/696] Minor fixes --- core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 584cfdff45a1d..3dabdd76b1aa5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -43,7 +43,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val numFailures = new Array[Int](numTasks) val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 - val creationTime = System.currentTimeMillis // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis From 680f42e6cd1ee8593136323a539dc5117b165377 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sun, 10 Feb 2013 02:27:03 -0600 Subject: [PATCH 188/696] Change defaultPartitioner to use upstream split size. Previously it used the SparkContext.defaultParallelism, which occassionally ended up being a very bad guess. Looking at upstream RDDs seems to make better use of the context. Also sorted the upstream RDDs by partition size first, as if we have a hugely-partitioned RDD and tiny-partitioned RDD, it is unlikely we want the resulting RDD to be tiny-partitioned. --- .../main/scala/spark/PairRDDFunctions.scala | 8 ++++++-- .../test/scala/spark/PartitioningSuite.scala | 8 ++++---- core/src/test/scala/spark/ShuffleSuite.scala | 19 +++++++++++++++++++ 3 files changed, 29 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index cc3cca2571359..18b4a1eca486c 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -439,12 +439,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. If any of * the RDDs already has a partitioner, choose that one, otherwise use a default HashPartitioner. + * + * The number of partitions will be the same as the number of partitions in the largest upstream + * RDD, as this should be least likely to cause out-of-memory errors. */ def defaultPartitioner(rdds: RDD[_]*): Partitioner = { - for (r <- rdds if r.partitioner != None) { + val bySize = rdds.sortBy(_.splits.size).reverse + for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - return new HashPartitioner(self.context.defaultParallelism) + return new HashPartitioner(bySize.head.splits.size) } /** diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index af1107cd197b4..60db759c25f3b 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -84,10 +84,10 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { assert(grouped4.groupByKey(3).partitioner != grouped4.partitioner) assert(grouped4.groupByKey(4).partitioner === grouped4.partitioner) - assert(grouped2.join(grouped4).partitioner === grouped2.partitioner) - assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped2.partitioner) - assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped2.partitioner) - assert(grouped2.cogroup(grouped4).partitioner === grouped2.partitioner) + assert(grouped2.join(grouped4).partitioner === grouped4.partitioner) + assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped4.partitioner) + assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped4.partitioner) + assert(grouped2.cogroup(grouped4).partitioner === grouped4.partitioner) assert(grouped2.join(reduced2).partitioner === grouped2.partitioner) assert(grouped2.leftOuterJoin(reduced2).partitioner === grouped2.partitioner) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 3493b9511f6c2..ab7060a1ac2df 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -211,6 +211,25 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(rdd.keys.collect().toList === List(1, 2)) assert(rdd.values.collect().toList === List("a", "b")) } + + test("default partition size uses split size") { + sc = new SparkContext("local", "test") + // specify 2000 splits + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 splits + val c = b.groupByKey() + assert(c.splits.size === 2000) + } + + test("default partition uses largest partitioner") { + sc = new SparkContext("local", "test") + val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) + val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) + val c = a.join(b) + assert(c.splits.size === 2000) + } } object ShuffleSuite { From 4975dcdafc0e4b1ab8c6e91525e01a84a0a934aa Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sun, 10 Feb 2013 12:55:47 -0800 Subject: [PATCH 189/696] Fixed a 404 -- missing '.html' --- docs/tuning.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tuning.md b/docs/tuning.md index 9aaa53cd65205..738c530458b59 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -233,7 +233,7 @@ number of cores in your clusters. ## Broadcasting Large Variables -Using the [broadcast functionality](scala-programming-guide#broadcast-variables) +Using the [broadcast functionality](scala-programming-guide.html#broadcast-variables) available in `SparkContext` can greatly reduce the size of each serialized task, and the cost of launching a job over a cluster. If your tasks use any large object from the driver program inside of them (e.g. a static lookup table), consider turning it into a broadcast variable. From 131b56afd0ec20b92502e11acda77c6594380471 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 10 Feb 2013 13:28:31 -0800 Subject: [PATCH 190/696] Update issue tracker link in contributing guide. --- docs/contributing-to-spark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md index c6e01c62d8080..14d0dc856b2f7 100644 --- a/docs/contributing-to-spark.md +++ b/docs/contributing-to-spark.md @@ -15,7 +15,7 @@ The Spark team welcomes contributions in the form of GitHub pull requests. Here But first, make sure that you have [configured a spark-env.sh](configuration.html) with at least `SCALA_HOME`, as some of the tests try to spawn subprocesses using this. - Add new unit tests for your code. We use [ScalaTest](http://www.scalatest.org/) for testing. Just add a new Suite in `core/src/test`, or methods to an existing Suite. -- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issues page](https://github.com/mesos/spark/issues), or email the [mailing list](http://www.spark-project.org/mailing-lists.html). +- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issue tracker](https://spark-project.atlassian.net), or email the [mailing list](http://www.spark-project.org/mailing-lists.html). # Licensing of Contributions From b7d9e2439445da9b1ca8709f4ad8fcac9927dd76 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Feb 2013 14:18:52 -0800 Subject: [PATCH 191/696] use TaskMetrics to gather all stats; lots of plumbing to get it all the way back to driver --- .../scala/spark/executor/TaskMetrics.scala | 9 ++++++--- .../main/scala/spark/rdd/CoGroupedRDD.scala | 3 +++ .../scala/spark/scheduler/DAGScheduler.scala | 15 +++++++------- .../spark/scheduler/DAGSchedulerEvent.scala | 4 +++- .../spark/scheduler/ShuffleMapTask.scala | 5 +---- .../scala/spark/scheduler/StageInfo.scala | 20 +++++++++++++++---- .../src/main/scala/spark/scheduler/Task.scala | 6 ++++++ .../scala/spark/scheduler/TaskResult.scala | 4 +++- .../scheduler/TaskSchedulerListener.scala | 6 ++++-- .../spark/scheduler/cluster/TaskInfo.scala | 2 ++ .../scheduler/cluster/TaskSetManager.scala | 6 +++--- .../scheduler/local/LocalScheduler.scala | 4 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 10 +++++----- 13 files changed, 61 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 6cdc6d0b7e292..b005b9ff27a94 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -7,10 +7,13 @@ case class TaskMetrics( val remoteBlocksFetched: Option[Int], val localBlocksFetched: Option[Int], val remoteFetchWaitTime: Option[Long], - val remoteBytesRead: Option[Long] + val remoteBytesRead: Option[Long], + val shuffleBytesWritten: Option[Long] ) object TaskMetrics { - private[spark] def apply(task: Task[_]) : TaskMetrics = - TaskMetrics(None, None, None, task.remoteFetchWaitTime, task.remoteReadBytes) + private[spark] def apply(task: Task[_]) : TaskMetrics = { + TaskMetrics(task.totalBlocksFetched, task.remoteBlocksFetched, task.localBlocksFetched, + task.remoteFetchWaitTime, task.remoteReadBytes, task.shuffleBytesWritten) + } } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 90b822ec857e4..4aa97f6ad9e4e 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -116,6 +116,9 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) context.task.remoteFetchTime = Some(fetchItr.remoteFetchTime) context.task.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) context.task.remoteReadBytes = Some(fetchItr.remoteBytesRead) + context.task.totalBlocksFetched = Some(fetchItr.totalBlocks) + context.task.localBlocksFetched = Some(fetchItr.numLocalBlocks) + context.task.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f07ee50670408..c0651e69d5800 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -7,9 +7,10 @@ import java.util.concurrent.Future import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.TimeUnit -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ +import executor.TaskMetrics import spark.partial.ApproximateActionListener import spark.partial.ApproximateEvaluator import spark.partial.PartialResult @@ -42,8 +43,9 @@ class DAGScheduler( reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], - taskInfo: TaskInfo) { - eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo)) + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) { + eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) } // Called by TaskScheduler when an executor fails. @@ -77,7 +79,7 @@ class DAGScheduler( private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] - private val sparkListeners = Traversable[SparkListener]() + private[spark] var sparkListeners = ArrayBuffer[SparkListener]() var cacheLocs = new HashMap[Int, Array[List[String]]] @@ -491,6 +493,7 @@ class DAGScheduler( } pendingTasks(stage) -= task stageToInfos(stage).taskInfos += event.taskInfo + stageToInfos(stage).taskMetrics += event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { @@ -512,10 +515,6 @@ class DAGScheduler( case smt: ShuffleMapTask => val status = event.result.asInstanceOf[MapStatus] - smt.totalBytesWritten match { - case Some(b) => stageToInfos(stage).shuffleBytesWritten += b - case None => throw new RuntimeException("shuffle stask completed without tracking bytes written") - } val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) { diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 9f19e137e4131..2a5cad7006b75 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -4,6 +4,7 @@ import cluster.TaskInfo import scala.collection.mutable.Map import spark._ +import executor.TaskMetrics /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue @@ -27,7 +28,8 @@ private[spark] case class CompletionEvent( reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], - taskInfo: TaskInfo) + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) extends DAGSchedulerEvent private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index e97dd0ad3ab0b..744f046b0e87a 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -81,9 +81,6 @@ private[spark] class ShuffleMapTask( with Externalizable with Logging { - - var totalBytesWritten : Option[Long] = None - protected def this() = this(0, null, null, 0, null) var split = if (rdd == null) { @@ -144,7 +141,7 @@ private[spark] class ShuffleMapTask( totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) } - totalBytesWritten = Some(totalBytes) + shuffleBytesWritten = Some(totalBytes) return new MapStatus(blockManager.blockManagerId, compressedSizes) } finally { diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 489b9744dead2..c70b346e4005c 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -3,17 +3,29 @@ package spark.scheduler import cluster.TaskInfo import collection._ import spark.util.Distribution +import spark.executor.TaskMetrics case class StageInfo( val stage: Stage, val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), - val shuffleBytesWritten : mutable.Buffer[Long] = mutable.Buffer[Long](), - val shuffleBytesRead : mutable.Buffer[Long] = mutable.Buffer[Long]() + val taskMetrics: mutable.Buffer[TaskMetrics] = mutable.Buffer[TaskMetrics]() ) { - def name = stage.rdd.name + "(" + stage.origin + ")" + override def toString = stage.rdd.toString def getTaskRuntimeDistribution = { - new Distribution(taskInfos.map{_.duration.toDouble}) + Distribution(taskInfos.map{_.duration.toDouble}) + } + + def getShuffleBytesWrittenDistribution = { + Distribution(taskMetrics.flatMap{_.shuffleBytesWritten.map{_.toDouble}}) + } + + def getRemoteFetchWaitTimeDistribution = { + Distribution(taskMetrics.flatMap{_.remoteFetchWaitTime.map{_.toDouble}}) + } + + def getRemoteBytesReadDistribution = { + Distribution(taskMetrics.flatMap{_.remoteBytesRead.map{_.toDouble}}) } } diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index 9c1d5869be6b4..84fc3bc9889b8 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -21,6 +21,12 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { var remoteReadBytes : Option[Long] = None var remoteFetchWaitTime : Option[Long] = None var remoteFetchTime : Option[Long] = None + var totalBlocksFetched : Option[Int] = None + var remoteBlocksFetched: Option[Int] = None + var localBlocksFetched: Option[Int] = None + + var shuffleBytesWritten : Option[Long] = None + } /** diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 003ee2bb2ee22..6de0aa7adf8ac 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -9,7 +9,7 @@ import spark.executor.TaskMetrics // TODO: Use of distributed cache to return result is a hack to get around // what seems to be a bug with messages over 60KB in libprocess; fix it private[spark] -class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: TaskMetrics) extends Externalizable { +class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) extends Externalizable { def this() = this(null.asInstanceOf[T], null, null) override def writeExternal(out: ObjectOutput) { @@ -19,6 +19,7 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: out.writeLong(key) out.writeObject(value) } + out.writeObject(metrics) } override def readExternal(in: ObjectInput) { @@ -32,5 +33,6 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], val metrics: accumUpdates(in.readLong()) = in.readObject() } } + metrics = in.readObject().asInstanceOf[TaskMetrics] } } diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 4ede03cc2a1f0..771518dddfaca 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -1,16 +1,18 @@ package spark.scheduler -import cluster.TaskInfo +import spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import spark.TaskEndReason +import spark.executor.TaskMetrics /** * Interface for getting events back from the TaskScheduler. */ private[spark] trait TaskSchedulerListener { // A task has finished or failed. - def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo): Unit + def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], + taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit // A node was lost from the cluster. def executorLost(execId: String): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 0f975ce1eb419..a7e14094fb70a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,5 +1,7 @@ package spark.scheduler.cluster +import spark.executor.TaskMetrics + /** * Information about a running task attempt inside a TaskSet. */ diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 25aa6ab94a7ca..9ed098e501c7f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -259,7 +259,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info) + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { @@ -290,7 +290,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe reason match { case fetchFailed: FetchFailed => logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info) + sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) finished(index) = true tasksFinished += 1 sched.taskSetFinished(this) @@ -378,7 +378,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. - sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info) + sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null) } } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index e0075e9e8b232..1221f2957c585 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -87,7 +87,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon // If the threadpool has not already been shutdown, notify DAGScheduler if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, Success, resultToReturn, accumUpdates, info) + listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, null) } catch { case t: Throwable => { logError("Exception in task " + idInJob, t) @@ -98,7 +98,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } else { // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, new ExceptionFailure(t), null, null, info) + listener.taskEnded(task, new ExceptionFailure(t), null, null, info, null) } } } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index f74b425dd57a5..dc2c51ed4a7e5 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -265,7 +265,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null)) + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null, null)) } } } @@ -463,14 +463,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val noAccum = Map[Long, Any]() // We rely on the event queue being ordered and increasing the generation number by 1 // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null, null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null)) taskSet.tasks(1).generation = newGeneration val secondStage = interceptStage(reduceRdd) { - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null)) + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null)) } assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) From 383af599bb9e4935994be811415d1ee84f95ce56 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Feb 2013 14:19:37 -0800 Subject: [PATCH 192/696] SparkContext.addSparkListener; "std" listener in StatsReportListener --- core/src/main/scala/spark/SparkContext.scala | 4 +++ .../scala/spark/scheduler/SparkListener.scala | 35 +++++++++++++++++++ 2 files changed, 39 insertions(+) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 00b0b8270e353..6ac2f7c7e9133 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -469,6 +469,10 @@ class SparkContext( logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) } + def addSparkListener(listener: SparkListener) { + dagScheduler.sparkListeners += listener + } + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 54afc714b3b57..92d9c0330ae69 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,5 +1,8 @@ package spark.scheduler +import spark.util.Distribution +import spark.{Utils, Logging} + trait SparkListener { def onStageCompleted(stageCompleted: StageCompleted) } @@ -7,3 +10,35 @@ trait SparkListener { sealed trait SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents + + + +class StatsReportListener extends SparkListener with Logging { + def onStageCompleted(stageCompleted: StageCompleted) { + println("Finished stage: " + stageCompleted.stageInfo) + showDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution, "%4.0f") + showDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution, d => Utils.memoryBytesToString(d.toLong)) + showDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution, "%4.0f") + showDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution, d => Utils.memoryBytesToString(d.toLong)) + } + + //for profiling, the extremes are more interesting + val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) + val probabilities = percentiles.map{_ / 100.0} + val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" + + def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { + def f(d:Double) = format.format(d) + showDistribution(heading, dOpt, f _) + } + + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { + dOpt.foreach { d => + val stats = d.statCounter + logInfo(heading + stats) + val quantiles = d.getQuantiles(probabilities).map{formatNumber} + logInfo(percentilesHeader) + logInfo("\t" + quantiles.mkString("\t")) + } + } +} \ No newline at end of file From 16baea62bce62987158acce0595a0916c25b32b2 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 10 Feb 2013 19:14:49 -0800 Subject: [PATCH 193/696] Fixed bug in CheckpointRDD to prevent exception when the original RDD had zero splits. --- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 4 ++-- core/src/test/scala/spark/CheckpointSuite.scala | 10 ++++++++++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 96b593ba7ca6d..a21338f85f385 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -24,8 +24,8 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri val dirContents = fs.listStatus(new Path(checkpointPath)) val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted val numSplits = splitFiles.size - if (!splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || - !splitFiles(numSplits-1).endsWith(CheckpointRDD.splitIdToFile(numSplits-1))) { + if (numSplits > 0 && (!splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || + !splitFiles(numSplits-1).endsWith(CheckpointRDD.splitIdToFile(numSplits-1)))) { throw new SparkException("Invalid checkpoint directory: " + checkpointPath) } Array.tabulate(numSplits)(i => new CheckpointRDDSplit(i)) diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 0b74607fb85a6..4425949f46c70 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -162,6 +162,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false) } + test("CheckpointRDD with zero partitions") { + val rdd = new BlockRDD[Int](sc, Array[String]()) + assert(rdd.splits.size === 0) + assert(rdd.isCheckpointed === false) + rdd.checkpoint() + assert(rdd.count() === 0) + assert(rdd.isCheckpointed === true) + assert(rdd.splits.size === 0) + } + /** * Test checkpointing of the final RDD generated by the given operation. By default, * this method tests whether the size of serialized RDD has reduced after checkpointing or not. From fd90daf850a922fe33c3638b18304d827953e2cb Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 10 Feb 2013 19:48:42 -0800 Subject: [PATCH 194/696] Fixed bugs in FileInputDStream and Scheduler that occasionally failed to reprocess old files after recovering from master failure. Completely modified spark.streaming.FailureTest to test multiple master failures using file input stream. --- .../scala/spark/streaming/DStreamGraph.scala | 2 + .../scala/spark/streaming/JobManager.scala | 4 +- .../scala/spark/streaming/Scheduler.scala | 8 +- .../src/main/scala/spark/streaming/Time.scala | 4 + .../streaming/dstream/FileInputDStream.scala | 13 +- .../scala/spark/streaming/FailureSuite.scala | 281 +++++++++++++----- 6 files changed, 221 insertions(+), 91 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index d5a5496839407..7aa9d20004630 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -81,12 +81,14 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { private[streaming] def generateRDDs(time: Time): Seq[Job] = { this.synchronized { + logInfo("Generating RDDs for time " + time) outputStreams.flatMap(outputStream => outputStream.generateJob(time)) } } private[streaming] def forgetOldRDDs(time: Time) { this.synchronized { + logInfo("Forgetting old RDDs for time " + time) outputStreams.foreach(_.forgetOldMetadata(time)) } } diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala index 5acdd01e5816c..8b18c7bc6a8c2 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/spark/streaming/JobManager.scala @@ -15,8 +15,8 @@ class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging { SparkEnv.set(ssc.env) try { val timeTaken = job.run() - logInfo("Total delay: %.5f s for job %s (execution: %.5f s)".format( - (System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, timeTaken / 1000.0)) + logInfo("Total delay: %.5f s for job %s of time %s (execution: %.5f s)".format( + (System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, job.time.milliseconds, timeTaken / 1000.0)) } catch { case e: Exception => logError("Running " + job + " failed", e) diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala index b77986a3ba420..23a0f0974d3d4 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala @@ -40,7 +40,11 @@ class Scheduler(ssc: StreamingContext) extends Logging { clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } // Reschedule the batches that were received but not processed before failure - ssc.initialCheckpoint.pendingTimes.foreach(time => generateRDDs(time)) + //ssc.initialCheckpoint.pendingTimes.foreach(time => generateRDDs(time)) + val pendingTimes = ssc.initialCheckpoint.pendingTimes.sorted(Time.ordering) + println(pendingTimes.mkString(", ")) + pendingTimes.foreach(time => + graph.generateRDDs(time).foreach(jobManager.runJob)) // Restart the timer timer.restart(graph.zeroTime.milliseconds) logInfo("Scheduler's timer restarted") @@ -64,11 +68,11 @@ class Scheduler(ssc: StreamingContext) extends Logging { graph.generateRDDs(time).foreach(jobManager.runJob) graph.forgetOldRDDs(time) doCheckpoint(time) - logInfo("Generated RDDs for time " + time) } private def doCheckpoint(time: Time) { if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { + logInfo("Checkpointing graph for time " + time) val startTime = System.currentTimeMillis() ssc.graph.updateCheckpointData(time) checkpointWriter.write(new Checkpoint(ssc, time)) diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala index 5daeb761ddd67..8a6c9a5cb5ff6 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/spark/streaming/Time.scala @@ -39,4 +39,8 @@ case class Time(private val millis: Long) { override def toString: String = (millis.toString + " ms") +} + +object Time { + val ordering = Ordering.by((time: Time) => time.millis) } \ No newline at end of file diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index c6ffb252ce887..10ccb4318dfd6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -128,7 +128,7 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K private[streaming] class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]] + def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]] override def update() { hadoopFiles.clear() @@ -139,11 +139,12 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K override def restore() { hadoopFiles.foreach { - case (time, files) => { - logInfo("Restoring Hadoop RDD for time " + time + " from files " + - files.mkString("[", ",", "]") ) - files - generatedRDDs += ((time, filesToRDD(files))) + case (t, f) => { + // Restore the metadata in both files and generatedRDDs + logInfo("Restoring files for time " + t + " - " + + f.mkString("[", ", ", "]") ) + files += ((t, f)) + generatedRDDs += ((t, filesToRDD(f))) } } } diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index c4cfffbfc1ef3..efaa098d2e193 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -1,58 +1,58 @@ package spark.streaming -import org.scalatest.BeforeAndAfter +import org.scalatest.{FunSuite, BeforeAndAfter} import org.apache.commons.io.FileUtils import java.io.File import scala.runtime.RichInt import scala.util.Random import spark.streaming.StreamingContext._ -import collection.mutable.ArrayBuffer +import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import spark.Logging +import com.google.common.io.Files /** * This testsuite tests master failures at random times while the stream is running using * the real clock. */ -class FailureSuite extends TestSuiteBase with BeforeAndAfter { +class FailureSuite extends FunSuite with BeforeAndAfter with Logging { + + var testDir: File = null + var checkpointDir: File = null + val batchDuration = Milliseconds(500) before { - FileUtils.deleteDirectory(new File(checkpointDir)) + testDir = Files.createTempDir() + checkpointDir = Files.createTempDir() } after { FailureSuite.reset() - FileUtils.deleteDirectory(new File(checkpointDir)) + FileUtils.deleteDirectory(checkpointDir) + FileUtils.deleteDirectory(testDir) // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") } - override def framework = "CheckpointSuite" - - override def batchDuration = Milliseconds(500) - - override def checkpointDir = "checkpoint" - - override def checkpointInterval = batchDuration - test("multiple failures with updateStateByKey") { val n = 30 // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ... - val input = (1 to n).map(i => (1 to i).map(_ =>"a").toSeq).toSeq - // Last output: [ (a, 465) ] for n=30 - val lastOutput = Seq( ("a", (1 to n).reduce(_ + _)) ) + val input = (1 to n).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq + // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ... + val expectedOutput = (1 to n).map(i => (1 to i).reduce(_ + _)).map(j => ("a", j)) val operation = (st: DStream[String]) => { val updateFunc = (values: Seq[Int], state: Option[RichInt]) => { Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0))) } - st.map(x => (x, 1)) - .updateStateByKey[RichInt](updateFunc) - .checkpoint(Seconds(2)) - .map(t => (t._1, t._2.self)) + st.flatMap(_.split(" ")) + .map(x => (x, 1)) + .updateStateByKey[RichInt](updateFunc) + .checkpoint(Seconds(2)) + .map(t => (t._1, t._2.self)) } - testOperationWithMultipleFailures(input, operation, lastOutput, n, n) + testOperationWithMultipleFailures(input, operation, expectedOutput) } test("multiple failures with reduceByKeyAndWindow") { @@ -60,17 +60,18 @@ class FailureSuite extends TestSuiteBase with BeforeAndAfter { val w = 100 assert(w > n, "Window should be much larger than the number of input sets in this test") // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ... - val input = (1 to n).map(i => (1 to i).map(_ =>"a").toSeq).toSeq - // Last output: [ (a, 465) ] - val lastOutput = Seq( ("a", (1 to n).reduce(_ + _)) ) + val input = (1 to n).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq + // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ... + val expectedOutput = (1 to n).map(i => (1 to i).reduce(_ + _)).map(j => ("a", j)) val operation = (st: DStream[String]) => { - st.map(x => (x, 1)) + st.flatMap(_.split(" ")) + .map(x => (x, 1)) .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration) .checkpoint(Seconds(2)) } - testOperationWithMultipleFailures(input, operation, lastOutput, n, n) + testOperationWithMultipleFailures(input, operation, expectedOutput) } @@ -79,113 +80,231 @@ class FailureSuite extends TestSuiteBase with BeforeAndAfter { * final set of output values is as expected or not. Checking the final value is * proof that no intermediate data was lost due to master failures. */ - def testOperationWithMultipleFailures[U: ClassManifest, V: ClassManifest]( - input: Seq[Seq[U]], - operation: DStream[U] => DStream[V], - lastExpectedOutput: Seq[V], - numBatches: Int, - numExpectedOutput: Int + def testOperationWithMultipleFailures( + input: Seq[String], + operation: DStream[String] => DStream[(String, Int)], + expectedOutput: Seq[(String, Int)] ) { - var ssc = setupStreams[U, V](input, operation) - val mergedOutput = new ArrayBuffer[Seq[V]]() + var ssc = setupStreamsWithFileStream(operation) + + val mergedOutput = new ArrayBuffer[(String, Int)]() + val lastExpectedOutput = expectedOutput.last + val maxTimeToRun = expectedOutput.size * batchDuration.milliseconds * 2 var totalTimeRan = 0L - while(totalTimeRan <= numBatches * batchDuration.milliseconds * 2) { - new KillingThread(ssc, numBatches * batchDuration.milliseconds.toInt / 4).start() - val (output, timeRan) = runStreamsWithRealClock[V](ssc, numBatches, numExpectedOutput) + // Start generating files in the a different thread + val fileGeneratingThread = new FileGeneratingThread(input, testDir.getPath, batchDuration.milliseconds) + fileGeneratingThread.start() + + // Repeatedly start and kill the streaming context until timed out or + // all expected output is generated + while(!FailureSuite.outputGenerated && !FailureSuite.timedOut) { + + // Start the thread to kill the streaming after some time + FailureSuite.failed = false + val killingThread = new KillingThread(ssc, batchDuration.milliseconds * 10) + killingThread.start() + + // Run the streams with real clock until last expected output is seen or timed out + val (output, timeRan) = runStreamsWithRealClock(ssc, lastExpectedOutput, maxTimeToRun - totalTimeRan) + if (killingThread.isAlive) killingThread.interrupt() + + // Merge output and time ran and see whether already timed out or not mergedOutput ++= output totalTimeRan += timeRan logInfo("New output = " + output) logInfo("Merged output = " + mergedOutput) logInfo("Total time spent = " + totalTimeRan) - val sleepTime = Random.nextInt(numBatches * batchDuration.milliseconds.toInt / 8) - logInfo( - "\n-------------------------------------------\n" + - " Restarting stream computation in " + sleepTime + " ms " + - "\n-------------------------------------------\n" - ) - Thread.sleep(sleepTime) - FailureSuite.failed = false - ssc = new StreamingContext(checkpointDir) + if (totalTimeRan > maxTimeToRun) { + FailureSuite.timedOut = true + } + + if (!FailureSuite.outputGenerated && !FailureSuite.timedOut) { + val sleepTime = Random.nextInt(batchDuration.milliseconds.toInt * 2) + logInfo( + "\n-------------------------------------------\n" + + " Restarting stream computation in " + sleepTime + " ms " + + "\n-------------------------------------------\n" + ) + Thread.sleep(sleepTime) + } + + // Recreate the streaming context from checkpoint + ssc = new StreamingContext(checkpointDir.getPath) } ssc.stop() ssc = null + logInfo("Finished test after " + FailureSuite.failureCount + " failures") + + if (FailureSuite.timedOut) { + logWarning("Timed out with run time of "+ maxTimeToRun + " ms for " + + expectedOutput.size + " batches of " + batchDuration) + } + + // Verify whether the output is as expected + verifyOutput(mergedOutput, expectedOutput) + if (fileGeneratingThread.isAlive) fileGeneratingThread.interrupt() + } - // Verify whether the last output is the expected one - val lastOutput = mergedOutput(mergedOutput.lastIndexWhere(!_.isEmpty)) - assert(lastOutput.toSet === lastExpectedOutput.toSet) - logInfo("Finished computation after " + FailureSuite.failureCount + " failures") + /** Sets up the stream operations with file input stream */ + def setupStreamsWithFileStream( + operation: DStream[String] => DStream[(String, Int)] + ): StreamingContext = { + val ssc = new StreamingContext("local[4]", "FailureSuite", batchDuration) + ssc.checkpoint(checkpointDir.getPath) + val inputStream = ssc.textFileStream(testDir.getPath) + val operatedStream = operation(inputStream) + val outputBuffer = new ArrayBuffer[Seq[(String, Int)]] with SynchronizedBuffer[Seq[(String, Int)]] + val outputStream = new TestOutputStream(operatedStream, outputBuffer) + ssc.registerOutputStream(outputStream) + ssc } /** - * Runs the streams set up in `ssc` on real clock until the expected max number of + * Runs the streams set up in `ssc` on real clock. */ - def runStreamsWithRealClock[V: ClassManifest]( - ssc: StreamingContext, - numBatches: Int, - maxExpectedOutput: Int - ): (Seq[Seq[V]], Long) = { + def runStreamsWithRealClock( + ssc: StreamingContext, + lastExpectedOutput: (String, Int), + timeout: Long + ): (Seq[(String, Int)], Long) = { System.clearProperty("spark.streaming.clock") - assert(numBatches > 0, "Number of batches to run stream computation is zero") - assert(maxExpectedOutput > 0, "Max expected outputs after " + numBatches + " is zero") - logInfo("numBatches = " + numBatches + ", maxExpectedOutput = " + maxExpectedOutput) - // Get the output buffer - val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]] + val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[(String, Int)]] val output = outputStream.output - val waitTime = (batchDuration.milliseconds * (numBatches.toDouble + 0.5)).toLong val startTime = System.currentTimeMillis() + // Functions to detect various conditions + def hasFailed = FailureSuite.failed + def isLastOutputGenerated = !output.flatMap(x => x).isEmpty && output(output.lastIndexWhere(!_.isEmpty)).head == lastExpectedOutput + def isTimedOut = System.currentTimeMillis() - startTime > timeout + + // Start the streaming computation and let it run while ... + // (i) StreamingContext has not been shut down yet + // (ii) The last expected output has not been generated yet + // (iii) Its not timed out yet try { - // Start computation ssc.start() - - // Wait until expected number of output items have been generated - while (output.size < maxExpectedOutput && System.currentTimeMillis() - startTime < waitTime && !FailureSuite.failed) { - logInfo("output.size = " + output.size + ", maxExpectedOutput = " + maxExpectedOutput) + while (!hasFailed && !isLastOutputGenerated && !isTimedOut) { Thread.sleep(100) } + logInfo("Has failed = " + hasFailed) + logInfo("Is last output generated = " + isLastOutputGenerated) + logInfo("Is timed out = " + isTimedOut) } catch { case e: Exception => logInfo("Exception while running streams: " + e) } finally { ssc.stop() } + + // Verify whether the output of each batch has only one element + assert(output.forall(_.size <= 1), "output of each batch should have only one element") + + // Set appropriate flags is timed out or output has been generated + if (isTimedOut) FailureSuite.timedOut = true + if (isLastOutputGenerated) FailureSuite.outputGenerated = true + val timeTaken = System.currentTimeMillis() - startTime logInfo("" + output.size + " sets of output generated in " + timeTaken + " ms") - (output, timeTaken) + (output.flatMap(_.headOption), timeTaken) } + /** + * Verifies the output value are the same as expected. Since failures can lead to + * a batch being processed twice, a batches output may appear more than once + * consecutively. To avoid getting confused with those, we eliminate consecutive + * duplicate batch outputs of values from the `output`. As a result, the + * expected output should not have consecutive batches with the same values as output. + */ + def verifyOutput(output: Seq[(String, Int)], expectedOutput: Seq[(String, Int)]) { + // Verify whether expected outputs do not consecutive batches with same output + for (i <- 0 until expectedOutput.size - 1) { + assert(expectedOutput(i) != expectedOutput(i+1), + "Expected output has consecutive duplicate sequence of values") + } + // Match the output with the expected output + logInfo( + "\n-------------------------------------------\n" + + " Verifying output " + + "\n-------------------------------------------\n" + ) + logInfo("Expected output, size = " + expectedOutput.size) + logInfo(expectedOutput.mkString("[", ",", "]")) + logInfo("Output, size = " + output.size) + logInfo(output.mkString("[", ",", "]")) + output.foreach(o => + assert(expectedOutput.contains(o), "Expected value " + o + " not found") + ) + } } object FailureSuite { var failed = false + var outputGenerated = false + var timedOut = false var failureCount = 0 def reset() { failed = false + outputGenerated = false + timedOut = false failureCount = 0 } } -class KillingThread(ssc: StreamingContext, maxKillWaitTime: Int) extends Thread with Logging { +/** + * Thread to kill streaming context after some time. + */ +class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging { initLogging() override def run() { - var minKillWaitTime = if (FailureSuite.failureCount == 0) 3000 else 1000 // to allow the first checkpoint - val killWaitTime = minKillWaitTime + Random.nextInt(maxKillWaitTime) - logInfo("Kill wait time = " + killWaitTime) - Thread.sleep(killWaitTime.toLong) - logInfo( - "\n---------------------------------------\n" + - "Killing streaming context after " + killWaitTime + " ms" + - "\n---------------------------------------\n" - ) - if (ssc != null) ssc.stop() - FailureSuite.failed = true - FailureSuite.failureCount += 1 + try { + var minKillWaitTime = if (FailureSuite.failureCount == 0) 5000 else 1000 // to allow the first checkpoint + val killWaitTime = minKillWaitTime + math.abs(Random.nextLong % maxKillWaitTime) + logInfo("Kill wait time = " + killWaitTime) + Thread.sleep(killWaitTime) + logInfo( + "\n---------------------------------------\n" + + "Killing streaming context after " + killWaitTime + " ms" + + "\n---------------------------------------\n" + ) + if (ssc != null) { + ssc.stop() + FailureSuite.failed = true + FailureSuite.failureCount += 1 + } + logInfo("Killing thread exited") + } catch { + case ie: InterruptedException => logInfo("Killing thread interrupted") + case e: Exception => logWarning("Exception in killing thread", e) + } } } + +/** + * Thread to generate input files periodically with the desired text + */ +class FileGeneratingThread(input: Seq[String], testDir: String, interval: Long) + extends Thread with Logging { + initLogging() + + override def run() { + try { + Thread.sleep(5000) // To make sure that all the streaming context has been set up + for (i <- 0 until input.size) { + FileUtils.writeStringToFile(new File(testDir, i.toString), input(i).toString + "\n") + Thread.sleep(interval) + } + logInfo("File generating thread exited") + } catch { + case ie: InterruptedException => logInfo("File generating thread interrupted") + case e: Exception => logWarning("File generating in killing thread", e) + } + } +} + From d9461b15d35ed76ffaff23a4c07a84981428602a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 10 Feb 2013 21:41:40 -0800 Subject: [PATCH 195/696] cleanup a bunch of imports --- .../spark/BlockStoreShuffleFetcher.scala | 4 +-- .../src/main/scala/spark/ShuffleFetcher.scala | 4 +-- core/src/main/scala/spark/SparkContext.scala | 26 +++++++------------ core/src/main/scala/spark/TaskContext.scala | 2 +- .../scala/spark/scheduler/DAGScheduler.scala | 7 ++--- .../spark/scheduler/DAGSchedulerEvent.scala | 4 +-- .../scala/spark/scheduler/StageInfo.scala | 4 +-- .../scheduler/local/LocalScheduler.scala | 6 ++--- 8 files changed, 22 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index bb4e3de63ccb8..25532dae28b14 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -3,8 +3,8 @@ package spark import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import storage.{DelegateBlockFetchTracker, BlockManagerId} -import util.TimedIterator +import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} +import spark.util.TimedIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int) = { diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index d64b2611b7866..828cd943f4366 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,7 +1,7 @@ package spark -import storage.BlockFetchTracker -import util.TimedIterator +import spark.storage.BlockFetchTracker +import spark.util.TimedIterator private[spark] abstract class ShuffleFetcher { /** diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 6ac2f7c7e9133..e8da703d53603 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,19 +1,15 @@ package spark import java.io._ -import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger -import java.net.{URI, URLClassLoader} -import java.lang.ref.WeakReference +import java.net.URI import scala.collection.Map import scala.collection.generic.Growable -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ -import akka.actor.Actor -import akka.actor.Actor._ -import org.apache.hadoop.fs.{FileUtil, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.SequenceFileInputFormat @@ -33,23 +29,19 @@ import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} -import org.apache.mesos.{Scheduler, MesosNativeLibrary} +import org.apache.mesos.MesosNativeLibrary -import spark.broadcast._ import spark.deploy.LocalSparkCluster import spark.partial.ApproximateEvaluator import spark.partial.PartialResult -import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD} -import scheduler._ +import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD} +import spark.scheduler._ import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import storage.BlockManagerUI -import storage.RDDInfo -import storage.StorageStatus -import util.{MetadataCleaner, TimeStampedHashMap} -import storage.{StorageStatus, StorageUtils, RDDInfo} -import scala.Some +import spark.storage.BlockManagerUI +import spark.util.{MetadataCleaner, TimeStampedHashMap} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index 11819db3a3814..d62229f0ce9ff 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,7 +1,7 @@ package spark import scala.collection.mutable.ArrayBuffer -import scheduler.Task +import spark.scheduler.Task class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val task: Task[_]) extends Serializable { //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c0651e69d5800..608962b5f174f 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -1,22 +1,19 @@ package spark.scheduler import cluster.TaskInfo -import java.net.URI import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.Future import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.TimeUnit import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ -import executor.TaskMetrics +import spark.executor.TaskMetrics import spark.partial.ApproximateActionListener import spark.partial.ApproximateEvaluator import spark.partial.PartialResult import spark.storage.BlockManagerMaster -import spark.storage.BlockManagerId -import util.{MetadataCleaner, TimeStampedHashMap} +import spark.util.{MetadataCleaner, TimeStampedHashMap} /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 2a5cad7006b75..ed0b9bf178a89 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,10 +1,10 @@ package spark.scheduler -import cluster.TaskInfo +import spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import spark._ -import executor.TaskMetrics +import spark.executor.TaskMetrics /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index c70b346e4005c..df3d1ec4dc062 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -1,7 +1,7 @@ package spark.scheduler -import cluster.TaskInfo -import collection._ +import spark.scheduler.cluster.TaskInfo +import scala.collection._ import spark.util.Distribution import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 1221f2957c585..6b91728b74f2f 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -1,15 +1,13 @@ package spark.scheduler.local import java.io.File -import java.net.URLClassLoader -import java.util.concurrent.Executors import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.HashMap import spark._ -import executor.ExecutorURLClassLoader +import spark.executor.ExecutorURLClassLoader import spark.scheduler._ -import cluster.TaskInfo +import spark.scheduler.cluster.TaskInfo /** * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally From 0b788b760bc3b2d4d986acb9f6f04592aca9be26 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 6 Feb 2013 21:56:00 -0800 Subject: [PATCH 196/696] Update Windows scripts to launch daemons with less RAM and fix a few other issues Conflicts: run2.cmd --- run2.cmd | 23 ++++++++++++++++++----- sbt/sbt.cmd | 2 +- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/run2.cmd b/run2.cmd index 67f1e465e47b3..c913a5195ef95 100644 --- a/run2.cmd +++ b/run2.cmd @@ -11,9 +11,22 @@ set SPARK_HOME=%FWDIR% rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" +rem Test that an argument was given +if not "x%1"=="x" goto arg_given + echo Usage: run ^ [^] + goto exit +:arg_given + +set RUNNING_DAEMON=0 +if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1 +if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1 +if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m +if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% +if "%RUNNING_DAEMON%"=="1" set SPARK_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% + rem Check that SCALA_HOME has been specified if not "x%SCALA_HOME%"=="x" goto scala_exists - echo "SCALA_HOME is not set" + echo SCALA_HOME is not set goto exit :scala_exists @@ -40,10 +53,10 @@ rem Build up classpath set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes -for /R "%FWDIR%\lib_managed\jars" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j -for /R "%FWDIR%\lib_managed\bundles" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j -for /R "%REPL_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j -for /R "%PYSPARK_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j +set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\* +set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* +set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* +set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes rem Figure out whether to run our class with java or with the scala launcher. diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd index 6b289ab44761c..ce3ae7017458b 100644 --- a/sbt/sbt.cmd +++ b/sbt/sbt.cmd @@ -2,4 +2,4 @@ set EXTRA_ARGS= if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java set SPARK_HOME=%~dp0.. -java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-*.jar "%*" +java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*" From 8c66c4996220e7ea77aa9e307a744635b9576e5e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 6 Feb 2013 14:14:14 -0800 Subject: [PATCH 197/696] Tweak web UI so that people don't get confused about master URL format Conflicts: core/src/main/twirl/spark/deploy/master/index.scala.html core/src/main/twirl/spark/deploy/worker/index.scala.html --- core/src/main/scala/spark/deploy/DeployMessage.scala | 9 ++++++--- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- core/src/main/scala/spark/deploy/worker/Worker.scala | 2 +- core/src/main/twirl/spark/deploy/master/index.scala.html | 6 +++--- core/src/main/twirl/spark/deploy/worker/index.scala.html | 6 +++--- 5 files changed, 14 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 35f40c6e91fcf..9f05cb4f353d6 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -76,8 +76,11 @@ private[spark] case object RequestMasterState // Master to MasterWebUI private[spark] -case class MasterState(uri: String, workers: Array[WorkerInfo], activeJobs: Array[JobInfo], - completedJobs: Array[JobInfo]) +case class MasterState(host: String, port: Int, workers: Array[WorkerInfo], + activeJobs: Array[JobInfo], completedJobs: Array[JobInfo]) { + + def uri = "spark://" + host + ":" + port +} // WorkerWebUI to Worker private[spark] case object RequestWorkerState @@ -85,6 +88,6 @@ private[spark] case object RequestWorkerState // Worker to WorkerWebUI private[spark] -case class WorkerState(uri: String, workerId: String, executors: List[ExecutorRunner], +case class WorkerState(host: String, port: Int, workerId: String, executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 92e7914b1b931..5986281d97e2d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -131,7 +131,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } case RequestMasterState => { - sender ! MasterState(ip + ":" + port, workers.toArray, jobs.toArray, completedJobs.toArray) + sender ! MasterState(ip, port, workers.toArray, jobs.toArray, completedJobs.toArray) } } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 38547ec4f1dff..62f01776a9a64 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -140,7 +140,7 @@ private[spark] class Worker( masterDisconnected() case RequestWorkerState => { - sender ! WorkerState(ip + ":" + port, workerId, executors.values.toList, + sender ! WorkerState(ip, port, workerId, executors.values.toList, finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html index 285645c389895..cb1651c7e1b05 100644 --- a/core/src/main/twirl/spark/deploy/master/index.scala.html +++ b/core/src/main/twirl/spark/deploy/master/index.scala.html @@ -2,13 +2,13 @@ @import spark.deploy.master._ @import spark.Utils -@spark.common.html.layout(title = "Spark Master on " + state.uri) { - +@spark.common.html.layout(title = "Spark Master on " + state.host) { +
      -
    • URL: spark://@(state.uri)
    • +
    • URL: @(state.uri)
    • Workers: @state.workers.size
    • Cores: @{state.workers.map(_.cores).sum} Total, @{state.workers.map(_.coresUsed).sum} Used
    • diff --git a/core/src/main/twirl/spark/deploy/worker/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html index 1d703dae58ccc..c39f769a7387f 100644 --- a/core/src/main/twirl/spark/deploy/worker/index.scala.html +++ b/core/src/main/twirl/spark/deploy/worker/index.scala.html @@ -1,8 +1,8 @@ @(worker: spark.deploy.WorkerState) @import spark.Utils -@spark.common.html.layout(title = "Spark Worker on " + worker.uri) { - +@spark.common.html.layout(title = "Spark Worker on " + worker.host) { +
      @@ -10,12 +10,12 @@
    • ID: @worker.workerId
    • Master URL: @worker.masterUrl - (WebUI at @worker.masterWebUiUrl)
    • Cores: @worker.cores (@worker.coresUsed Used)
    • Memory: @{Utils.memoryMegabytesToString(worker.memory)} (@{Utils.memoryMegabytesToString(worker.memoryUsed)} Used)
    +

    Back to Master

    From 05d2e94838d5b728df203d87708beaf3f4aa4c81 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 6 Feb 2013 14:34:46 -0800 Subject: [PATCH 198/696] Use a separate memory setting for standalone cluster daemons Conflicts: docs/_config.yml --- docs/configuration.md | 10 +++++++++- docs/spark-standalone.md | 8 ++++++++ run | 12 ++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index a7054b4321bcf..f1ca77aa7826a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -197,6 +197,14 @@ Apart from these, the following properties are also available, and may be useful poor data locality, but the default generally works well. + + spark.worker.timeout + 60 + + Number of seconds after which the standalone deploy master considers a worker lost if it + receives no heartbeats. + + spark.akka.frameSize 10 @@ -218,7 +226,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout 20 - Communication timeout between Spark nodes. + Communication timeout between Spark nodes, in seconds. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index bf296221b8447..3986c0c79dcea 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -115,6 +115,14 @@ You can optionally configure the cluster further by setting environment variable SPARK_WORKER_WEBUI_PORT Port for the worker web UI (default: 8081) + + SPARK_DAEMON_MEMORY + Memory to allocate to the Spark master and worker daemons themselves (default: 512m) + + + SPARK_DAEMON_JAVA_OPTS + JVM options for the Spark master and worker daemons themselves (default: none) + diff --git a/run b/run index a0946294497a8..82b1da005a447 100755 --- a/run +++ b/run @@ -13,6 +13,18 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then . $FWDIR/conf/spark-env.sh fi +if [ -z "$1" ]; then + echo "Usage: run []" >&2 + exit 1 +fi + +# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable +# values for that; it doesn't need a lot +if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then + SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} + SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default +fi + if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then if [ `command -v scala` ]; then RUNNER="scala" From 1b47fa275236657bea358f5c95d89f568c439395 Mon Sep 17 00:00:00 2001 From: root Date: Wed, 6 Feb 2013 09:11:17 +0000 Subject: [PATCH 199/696] Detect hard crashes of workers using a heartbeat mechanism. Also fixes some issues in the rest of the code with detecting workers this way. Conflicts: core/src/main/scala/spark/deploy/master/Master.scala core/src/main/scala/spark/deploy/worker/Worker.scala core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala --- .../scala/spark/deploy/DeployMessage.scala | 3 +- .../scala/spark/deploy/master/Master.scala | 29 +++++++++++++++++-- .../spark/deploy/master/WorkerInfo.scala | 2 ++ .../scala/spark/deploy/worker/Worker.scala | 7 +++++ .../cluster/ExecutorLossReason.scala | 4 +-- .../cluster/SparkDeploySchedulerBackend.scala | 1 + .../cluster/StandaloneClusterMessage.scala | 3 ++ .../cluster/StandaloneSchedulerBackend.scala | 20 +++++++++++-- 8 files changed, 62 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 9f05cb4f353d6..1d88d4bc84926 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -30,6 +30,8 @@ case class ExecutorStateChanged( exitStatus: Option[Int]) extends DeployMessage +private[spark] case class Heartbeat(workerId: String) extends DeployMessage + // Master to Worker private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage @@ -45,7 +47,6 @@ private[spark] case class LaunchExecutor( sparkHome: String) extends DeployMessage - // Client to Master private[spark] case class RegisterJob(jobDescription: JobDescription) extends DeployMessage diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 5986281d97e2d..d9852616003af 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -3,6 +3,7 @@ package spark.deploy.master import akka.actor._ import akka.actor.Terminated import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.util.duration._ import java.text.SimpleDateFormat import java.util.Date @@ -16,6 +17,7 @@ import spark.util.AkkaUtils private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For job IDs + val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 var nextJobNumber = 0 val workers = new HashSet[WorkerInfo] @@ -46,6 +48,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) startWebUi() + context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) } def startWebUi() { @@ -111,6 +114,15 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } } + case Heartbeat(workerId) => { + idToWorker.get(workerId) match { + case Some(workerInfo) => + workerInfo.lastHeartbeat = System.currentTimeMillis() + case None => + logWarning("Got heartbeat from unregistered worker " + workerId) + } + } + case Terminated(actor) => { // The disconnected actor could've been either a worker or a job; remove whichever of // those we have an entry for in the corresponding actor hashmap @@ -219,8 +231,9 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor actorToWorker -= worker.actor addressToWorker -= worker.actor.path.address for (exec <- worker.executors.values) { - exec.job.driver ! ExecutorStateChanged(exec.job.id, exec.id, ExecutorState.LOST, None, None) - exec.job.executors -= exec.id + logInfo("Telling job of lost executor: " + exec.id) + exec.job.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None) + exec.job.removeExecutor(exec) } } @@ -259,6 +272,18 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor nextJobNumber += 1 jobId } + + /** Check for, and remove, any timed-out workers */ + def timeOutDeadWorkers() { + // Copy the workers into an array so we don't modify the hashset while iterating through it + val expirationTime = System.currentTimeMillis() - WORKER_TIMEOUT + val toRemove = workers.filter(_.lastHeartbeat < expirationTime).toArray + for (worker <- toRemove) { + logWarning("Removing %s because we got no heartbeat in %d seconds".format( + worker.id, WORKER_TIMEOUT)) + removeWorker(worker) + } + } } private[spark] object Master { diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 5a7f5fef8a546..2e467007a0165 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -18,6 +18,8 @@ private[spark] class WorkerInfo( var coresUsed = 0 var memoryUsed = 0 + var lastHeartbeat = System.currentTimeMillis() + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 62f01776a9a64..924935a5fdb2c 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -2,6 +2,7 @@ package spark.deploy.worker import scala.collection.mutable.{ArrayBuffer, HashMap} import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.util.duration._ import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ @@ -26,6 +27,9 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs + // Send a heartbeat every (heartbeat timeout) / 4 milliseconds + val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4 + var master: ActorRef = null var masterWebUiUrl : String = "" val workerId = generateWorkerId() @@ -97,6 +101,9 @@ private[spark] class Worker( case RegisteredWorker(url) => masterWebUiUrl = url logInfo("Successfully registered with master") + context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) { + master ! Heartbeat(workerId) + } case RegisterWorkerFailed(message) => logError("Worker registration failed: " + message) diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala index bba7de6a65c3d..8bf838209f3d8 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala @@ -12,10 +12,10 @@ class ExecutorLossReason(val message: String) { private[spark] case class ExecutorExited(val exitCode: Int) - extends ExecutorLossReason(ExecutorExitCode.explainExitCode(exitCode)) { + extends ExecutorLossReason(ExecutorExitCode.explainExitCode(exitCode)) { } private[spark] case class SlaveLost(_message: String = "Slave lost") - extends ExecutorLossReason(_message) { + extends ExecutorLossReason(_message) { } diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 59ff8bcb90fc2..3c3e83b138557 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -67,6 +67,7 @@ private[spark] class SparkDeploySchedulerBackend( case None => SlaveLost(message) } logInfo("Executor %s removed: %s".format(executorId, message)) + removeExecutor(executorId, reason.toString) scheduler.executorLost(executorId, reason) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index da7dcf4b6b48e..d7660678248b2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -37,3 +37,6 @@ object StatusUpdate { // Internal messages in driver private[spark] case object ReviveOffers extends StandaloneClusterMessage private[spark] case object StopDriver extends StandaloneClusterMessage + +private[spark] case class RemoveExecutor(executorId: String, reason: String) + extends StandaloneClusterMessage diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 082022be1c9da..4213eb87190c6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -68,6 +68,10 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor sender ! true context.stop(self) + case RemoveExecutor(executorId, reason) => + removeExecutor(executorId, reason) + sender ! true + case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) @@ -100,7 +104,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor // Remove a disconnected slave from the cluster def removeExecutor(executorId: String, reason: String) { - logInfo("Slave " + executorId + " disconnected, so removing it") + logInfo("Executor " + executorId + " disconnected, so removing it") val numCores = freeCores(executorId) actorToExecutorId -= executorActor(executorId) addressToExecutorId -= executorAddress(executorId) @@ -139,7 +143,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } } catch { case e: Exception => - throw new SparkException("Error stopping standalone scheduler's master actor", e) + throw new SparkException("Error stopping standalone scheduler's driver actor", e) } } @@ -148,6 +152,18 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } override def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2) + + // Called by backends + def removeExecutor(executorId: String, reason: String) { + try { + val timeout = 5.seconds + val future = driverActor.ask(RemoveExecutor(executorId, reason))(timeout) + Await.result(future, timeout) + } catch { + case e: Exception => + throw new SparkException("Error notifying standalone scheduler's driver actor", e) + } + } } private[spark] object StandaloneSchedulerBackend { From da8afbc77e5796d45686034db5560f18c057d3c9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 6 Feb 2013 13:30:35 -0800 Subject: [PATCH 200/696] Some bug and formatting fixes to FT Conflicts: core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala --- .../scala/spark/deploy/client/Client.scala | 2 +- .../scala/spark/deploy/master/Master.scala | 4 ++-- .../cluster/SparkDeploySchedulerBackend.scala | 1 - .../cluster/StandaloneSchedulerBackend.scala | 24 ++++++++++--------- .../mesos/CoarseMesosSchedulerBackend.scala | 6 ++++- 5 files changed, 21 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index a63eee12339d1..e01181d1b2807 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -107,7 +107,7 @@ private[spark] class Client( def stop() { if (actor != null) { try { - val timeout = 1.seconds + val timeout = 5.seconds val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index d9852616003af..a5de23261c61a 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -117,8 +117,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor case Heartbeat(workerId) => { idToWorker.get(workerId) match { case Some(workerInfo) => - workerInfo.lastHeartbeat = System.currentTimeMillis() - case None => + workerInfo.lastHeartbeat = System.currentTimeMillis() + case None => logWarning("Got heartbeat from unregistered worker " + workerId) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 3c3e83b138557..e77355c6cd41a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -68,6 +68,5 @@ private[spark] class SparkDeploySchedulerBackend( } logInfo("Executor %s removed: %s".format(executorId, message)) removeExecutor(executorId, reason.toString) - scheduler.executorLost(executorId, reason) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 4213eb87190c6..d6064325724df 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -104,16 +104,18 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor // Remove a disconnected slave from the cluster def removeExecutor(executorId: String, reason: String) { - logInfo("Executor " + executorId + " disconnected, so removing it") - val numCores = freeCores(executorId) - actorToExecutorId -= executorActor(executorId) - addressToExecutorId -= executorAddress(executorId) - executorActor -= executorId - executorHost -= executorId - freeCores -= executorId - executorHost -= executorId - totalCoreCount.addAndGet(-numCores) - scheduler.executorLost(executorId, SlaveLost(reason)) + if (executorActor.contains(executorId)) { + logInfo("Executor " + executorId + " disconnected, so removing it") + val numCores = freeCores(executorId) + actorToExecutorId -= executorActor(executorId) + addressToExecutorId -= executorAddress(executorId) + executorActor -= executorId + executorHost -= executorId + freeCores -= executorId + executorHost -= executorId + totalCoreCount.addAndGet(-numCores) + scheduler.executorLost(executorId, SlaveLost(reason)) + } } } @@ -153,7 +155,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor override def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2) - // Called by backends + // Called by subclasses when notified of a lost worker def removeExecutor(executorId: String, reason: String) { try { val timeout = 5.seconds diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index b481ec0a72dce..7caf06e917125 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -239,7 +239,11 @@ private[spark] class CoarseMesosSchedulerBackend( override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { logInfo("Mesos slave lost: " + slaveId.getValue) synchronized { - slaveIdsWithExecutors -= slaveId.getValue + if (slaveIdsWithExecutors.contains(slaveId.getValue)) { + // Note that the slave ID corresponds to the executor ID on that slave + slaveIdsWithExecutors -= slaveId.getValue + removeExecutor(slaveId.getValue, "Mesos slave lost") + } } } From e9f53ec0eadadfc6faa2bbf6ead4216ca3417073 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 09:36:49 -0800 Subject: [PATCH 201/696] undo chnage to onCompleteCallbacks --- core/src/main/scala/spark/TaskContext.scala | 6 +++--- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- core/src/main/scala/spark/rdd/HadoopRDD.scala | 2 +- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index d62229f0ce9ff..6b67b17b8ac48 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -7,15 +7,15 @@ class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val t //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to // be separate - @transient val onCompleteCallbacks = new ArrayBuffer[TaskContext => Unit] + @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit] // Add a callback function to be executed on task completion. An example use // is for HadoopRDD to register a callback to close the input stream. - def addOnCompleteCallback(f: TaskContext => Unit) { + def addOnCompleteCallback(f: () => Unit) { onCompleteCallbacks += f } def executeOnCompleteCallbacks() { - onCompleteCallbacks.foreach{_.apply(this)} + onCompleteCallbacks.foreach{_()} } } diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index ad085585904ec..96b593ba7ca6d 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -104,7 +104,7 @@ private[spark] object CheckpointRDD extends Logging { val deserializeStream = serializer.deserializeStream(fileInputStream) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(_ => deserializeStream.close()) + context.addOnCompleteCallback(() => deserializeStream.close()) deserializeStream.asIterator.asInstanceOf[Iterator[T]] } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 3f0b4ce23e9d5..f547f53812661 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -74,7 +74,7 @@ class HadoopRDD[K, V]( reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(_ => reader.close()) + context.addOnCompleteCallback(() => reader.close()) val key: K = reader.createKey() val value: V = reader.createValue() diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index c80d30e12542c..c3b155fcbddd6 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -63,7 +63,7 @@ class NewHadoopRDD[K, V]( reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(_ => reader.close()) + context.addOnCompleteCallback(() => reader.close()) var havePair = false var finished = false From f0b68c623c116540470e06967c1554855d16a500 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 5 Feb 2013 19:02:46 -0800 Subject: [PATCH 202/696] Initial cut at replacing K, V in Java files --- core/src/test/scala/spark/JavaAPISuite.java | 24 ++++++++ .../streaming/api/java/JavaDStreamLike.scala | 4 +- .../java/spark/streaming/JavaAPISuite.java | 56 +++++++++++++++++++ 3 files changed, 82 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 934e4c2f6793b..9ffe7c5f992b6 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -696,4 +696,28 @@ public void checkpointAndRestore() { JavaRDD recovered = sc.checkpointFile(rdd.getCheckpointFile().get()); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); } + + @Test + public void mapOnPairRDD() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); + JavaPairRDD rdd2 = rdd1.map(new PairFunction() { + @Override + public Tuple2 call(Integer i) throws Exception { + return new Tuple2(i, i % 2); + } + }); + JavaPairRDD rdd3 = rdd2.map( + new PairFunction, Integer, Integer>() { + @Override + public Tuple2 call(Tuple2 in) throws Exception { + return new Tuple2(in._2(), in._1()); + } + }); + Assert.assertEquals(Arrays.asList( + new Tuple2(1, 1), + new Tuple2(0, 2), + new Tuple2(1, 3), + new Tuple2(0, 4)), rdd3.collect()); + + } } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index b93cb7865aef0..39fe0d0cccc47 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -59,8 +59,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable } /** Return a new DStream by applying a function to all elements of this DStream. */ - def map[K, V](f: PairFunction[T, K, V]): JavaPairDStream[K, V] = { - def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] + def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { + def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]] new JavaPairDStream(dstream.map(f)(cm))(f.keyType(), f.valueType()) } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 79d60934297f3..26ac82b71ace5 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -506,6 +506,62 @@ public Boolean call(Tuple2 in) throws Exception { new Tuple2("new york", 3), new Tuple2("new york", 1))); + @Test + public void testPairMap() { // Maps pair -> pair + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, "california"), + new Tuple2(3, "california"), + new Tuple2(4, "new york"), + new Tuple2(1, "new york")), + Arrays.asList( + new Tuple2(5, "california"), + new Tuple2(5, "california"), + new Tuple2(3, "new york"), + new Tuple2(1, "new york"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.map( + new PairFunction, Integer, String>() { + @Override + public Tuple2 call(Tuple2 in) throws Exception { + return new Tuple2(in._2(), in._1()); + } + }); + + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairMap2() { // Maps pair -> single + List>> inputData = stringIntKVStream; + + List> expected = Arrays.asList( + Arrays.asList(1, 3, 4, 1), + Arrays.asList(5, 5, 3, 1)); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaDStream reversed = pairStream.map( + new Function, Integer>() { + @Override + public Integer call(Tuple2 in) throws Exception { + return in._2(); + } + }); + + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + @Test public void testPairGroupByKey() { List>> inputData = stringStringKVStream; From 314d87a038d84c4ae9a6471ea19a5431153ea604 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 11 Feb 2013 09:20:37 -0800 Subject: [PATCH 203/696] Indentation fix --- .../java/spark/streaming/JavaAPISuite.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 26ac82b71ace5..4cf9d115ae9aa 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -511,16 +511,16 @@ public void testPairMap() { // Maps pair -> pair List>> inputData = stringIntKVStream; List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2(1, "california"), - new Tuple2(3, "california"), - new Tuple2(4, "new york"), - new Tuple2(1, "new york")), - Arrays.asList( - new Tuple2(5, "california"), - new Tuple2(5, "california"), - new Tuple2(3, "new york"), - new Tuple2(1, "new york"))); + Arrays.asList( + new Tuple2(1, "california"), + new Tuple2(3, "california"), + new Tuple2(4, "new york"), + new Tuple2(1, "new york")), + Arrays.asList( + new Tuple2(5, "california"), + new Tuple2(5, "california"), + new Tuple2(3, "new york"), + new Tuple2(1, "new york"))); JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); From 20cf77054536acd9c064d6e7ffedce23a87fb6a5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 11 Feb 2013 09:21:06 -0800 Subject: [PATCH 204/696] Fix for flatmap --- .../streaming/api/java/JavaDStreamLike.scala | 4 +- .../java/spark/streaming/JavaAPISuite.java | 42 +++++++++++++++++++ 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 39fe0d0cccc47..9cc263930e8b9 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -78,10 +78,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable * Return a new DStream by applying a function to all elements of this DStream, * and then flattening the results */ - def flatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairDStream[K, V] = { + def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { import scala.collection.JavaConverters._ def fn = (x: T) => f.apply(x).asScala - def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] + def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]] new JavaPairDStream(dstream.flatMap(fn)(cm))(f.keyType(), f.valueType()) } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 4cf9d115ae9aa..ec4e5ae18b257 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -562,6 +562,48 @@ public Integer call(Tuple2 in) throws Exception { Assert.assertEquals(expected, result); } + @Test + public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2("hi", 1), + new Tuple2("ho", 2)), + Arrays.asList( + new Tuple2("hi", 1), + new Tuple2("ho", 2))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, "h"), + new Tuple2(1, "i"), + new Tuple2(2, "h"), + new Tuple2(2, "o")), + Arrays.asList( + new Tuple2(1, "h"), + new Tuple2(1, "i"), + new Tuple2(2, "h"), + new Tuple2(2, "o"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream flatMapped = pairStream.flatMap( + new PairFlatMapFunction, Integer, String>() { + @Override + public Iterable> call(Tuple2 in) throws Exception { + List> out = new LinkedList>(); + for (Character s: in._1().toCharArray()) { + out.add(new Tuple2(in._2(), s.toString())); + } + return out; + } + }); + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + @Test public void testPairGroupByKey() { List>> inputData = stringStringKVStream; From c65988bdc1b75e88e6df77df0b84fc3a34c5b028 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 11 Feb 2013 09:51:57 -0800 Subject: [PATCH 205/696] Fix for MapPartitions --- .../streaming/api/java/JavaDStreamLike.scala | 4 +- .../java/spark/streaming/JavaAPISuite.java | 67 ++++++++++++++----- 2 files changed, 54 insertions(+), 17 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 9cc263930e8b9..ec546c81907c1 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -100,8 +100,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition * of the RDD. */ - def mapPartitions[K, V](f: PairFlatMapFunction[java.util.Iterator[T], K, V]) - : JavaPairDStream[K, V] = { + def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]) + : JavaPairDStream[K2, V2] = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) new JavaPairDStream(dstream.mapPartitions(fn))(f.keyType(), f.valueType()) } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index ec4e5ae18b257..67d82d546fb6d 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -507,7 +507,7 @@ public Boolean call(Tuple2 in) throws Exception { new Tuple2("new york", 1))); @Test - public void testPairMap() { // Maps pair -> pair + public void testPairMap() { // Maps pair -> pair of different type List>> inputData = stringIntKVStream; List>> expected = Arrays.asList( @@ -538,6 +538,43 @@ public Tuple2 call(Tuple2 in) throws Exception Assert.assertEquals(expected, result); } + @Test + public void testPairMapPartitions() { // Maps pair -> pair of different type + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, "california"), + new Tuple2(3, "california"), + new Tuple2(4, "new york"), + new Tuple2(1, "new york")), + Arrays.asList( + new Tuple2(5, "california"), + new Tuple2(5, "california"), + new Tuple2(3, "new york"), + new Tuple2(1, "new york"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.mapPartitions( + new PairFlatMapFunction>, Integer, String>() { + @Override + public Iterable> call(Iterator> in) throws Exception { + LinkedList> out = new LinkedList>(); + while (in.hasNext()) { + Tuple2 next = in.next(); + out.add(new Tuple2(next._2(), next._1())); + } + return out; + } + }); + + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + @Test public void testPairMap2() { // Maps pair -> single List>> inputData = stringIntKVStream; @@ -588,16 +625,16 @@ public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream flatMapped = pairStream.flatMap( - new PairFlatMapFunction, Integer, String>() { - @Override - public Iterable> call(Tuple2 in) throws Exception { - List> out = new LinkedList>(); - for (Character s: in._1().toCharArray()) { - out.add(new Tuple2(in._2(), s.toString())); - } - return out; - } - }); + new PairFlatMapFunction, Integer, String>() { + @Override + public Iterable> call(Tuple2 in) throws Exception { + List> out = new LinkedList>(); + for (Character s : in._1().toCharArray()) { + out.add(new Tuple2(in._2(), s.toString())); + } + return out; + } + }); JavaTestUtils.attachTestOutputStream(flatMapped); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -668,7 +705,7 @@ public void testCombineByKey() { JavaPairDStream combined = pairStream.combineByKey( new Function() { - @Override + @Override public Integer call(Integer i) throws Exception { return i; } @@ -766,19 +803,19 @@ public void testUpdateStateByKey() { JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream updated = pairStream.updateStateByKey( - new Function2, Optional, Optional>(){ + new Function2, Optional, Optional>() { @Override public Optional call(List values, Optional state) { int out = 0; if (state.isPresent()) { out = out + state.get(); } - for (Integer v: values) { + for (Integer v : values) { out = out + v; } return Optional.of(out); } - }); + }); JavaTestUtils.attachTestOutputStream(updated); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); From 04786d07391c4052d6dc42ff0828a79a37bbbfdf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 11 Feb 2013 10:05:49 -0800 Subject: [PATCH 206/696] small fix --- streaming/src/test/java/spark/streaming/JavaAPISuite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 67d82d546fb6d..551d4f15e40e7 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -810,12 +810,12 @@ public Optional call(List values, Optional state) { if (state.isPresent()) { out = out + state.get(); } - for (Integer v : values) { + for (Integer v: values) { out = out + v; } return Optional.of(out); } - }); + }); JavaTestUtils.attachTestOutputStream(updated); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); From d09c36065ca040044530a50f0392c92866b6d301 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 11 Feb 2013 10:45:45 -0800 Subject: [PATCH 207/696] Using tuple swap() --- streaming/src/test/java/spark/streaming/JavaAPISuite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 551d4f15e40e7..9bfcd83e4d497 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -528,7 +528,7 @@ public void testPairMap() { // Maps pair -> pair of different type new PairFunction, Integer, String>() { @Override public Tuple2 call(Tuple2 in) throws Exception { - return new Tuple2(in._2(), in._1()); + return in.swap(); } }); @@ -563,7 +563,7 @@ public Iterable> call(Iterator> LinkedList> out = new LinkedList>(); while (in.hasNext()) { Tuple2 next = in.next(); - out.add(new Tuple2(next._2(), next._1())); + out.add(next.swap()); } return out; } From e9fb25426ea0b6dbe4c946243a2ac0836b031c58 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 11 Feb 2013 11:15:58 -0800 Subject: [PATCH 208/696] Remove hack workaround for SPARK-668. Renaming the type paramters solves this problem (see SPARK-694). I tried this fix earlier, but it didn't work because I didn't run `sbt/sbt clean` first. --- .../scala/spark/api/java/JavaRDDLike.scala | 13 ++++++------ .../spark/api/java/PairFlatMapWorkaround.java | 20 ------------------- 2 files changed, 7 insertions(+), 26 deletions(-) delete mode 100644 core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index d34d56d169f5f..e18f28d3265f1 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -12,7 +12,7 @@ import spark.storage.StorageLevel import com.google.common.base.Optional -trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround[T] { +trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This implicit val classManifest: ClassManifest[T] @@ -82,12 +82,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround } /** - * Part of the workaround for SPARK-668; called in PairFlatMapWorkaround.java. + * Return a new RDD by first applying a function to all elements of this + * RDD, and then flattening the results. */ - private[spark] def doFlatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairRDD[K, V] = { + def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { import scala.collection.JavaConverters._ def fn = (x: T) => f.apply(x).asScala - def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] + def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]] JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType()) } @@ -110,8 +111,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[K, V](f: PairFlatMapFunction[java.util.Iterator[T], K, V]): - JavaPairRDD[K, V] = { + def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]): + JavaPairRDD[K2, V2] = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(f.keyType(), f.valueType()) } diff --git a/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java b/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java deleted file mode 100644 index 68b6fd6622742..0000000000000 --- a/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java +++ /dev/null @@ -1,20 +0,0 @@ -package spark.api.java; - -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDDLike; -import spark.api.java.function.PairFlatMapFunction; - -import java.io.Serializable; - -/** - * Workaround for SPARK-668. - */ -class PairFlatMapWorkaround implements Serializable { - /** - * Return a new RDD by first applying a function to all elements of this - * RDD, and then flattening the results. - */ - public JavaPairRDD flatMap(PairFlatMapFunction f) { - return ((JavaRDDLike ) this).doFlatMap(f); - } -} From ea08537143d58b79b3ae5d083e9b3a5647257da8 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 11 Feb 2013 13:23:50 -0800 Subject: [PATCH 209/696] Fixed an exponential recursion that could happen with doCheckpoint due to lack of memoization --- bagel/src/test/scala/bagel/BagelSuite.scala | 35 ++++++++++++++++----- core/src/main/scala/spark/RDD.scala | 14 ++++++--- 2 files changed, 37 insertions(+), 12 deletions(-) diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala index 3c2f9c4616fe2..47829a431e871 100644 --- a/bagel/src/test/scala/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/bagel/BagelSuite.scala @@ -1,10 +1,8 @@ package spark.bagel import org.scalatest.{FunSuite, Assertions, BeforeAndAfter} -import org.scalatest.prop.Checkers -import org.scalacheck.Arbitrary._ -import org.scalacheck.Gen -import org.scalacheck.Prop._ +import org.scalatest.concurrent.Timeouts +import org.scalatest.time.SpanSugar._ import scala.collection.mutable.ArrayBuffer @@ -13,7 +11,7 @@ import spark._ class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable -class BagelSuite extends FunSuite with Assertions with BeforeAndAfter { +class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeouts { var sc: SparkContext = _ @@ -25,7 +23,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") } - + test("halting by voting") { sc = new SparkContext("local", "test") val verts = sc.parallelize(Array("a", "b", "c", "d").map(id => (id, new TestVertex(true, 0)))) @@ -36,8 +34,9 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter { (self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) => (new TestVertex(superstep < numSupersteps - 1, self.age + 1), Array[TestMessage]()) } - for ((id, vert) <- result.collect) + for ((id, vert) <- result.collect) { assert(vert.age === numSupersteps) + } } test("halting by message silence") { @@ -57,7 +56,27 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter { } (new TestVertex(self.active, self.age + 1), msgsOut) } - for ((id, vert) <- result.collect) + for ((id, vert) <- result.collect) { assert(vert.age === numSupersteps) + } + } + + test("large number of iterations") { + // This tests whether jobs with a large number of iterations finish in a reasonable time, + // because non-memoized recursion in RDD or DAGScheduler used to cause them to hang + failAfter(10 seconds) { + sc = new SparkContext("local", "test") + val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0)))) + val msgs = sc.parallelize(Array[(String, TestMessage)]()) + val numSupersteps = 50 + val result = + Bagel.run(sc, verts, msgs, sc.defaultParallelism) { + (self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) => + (new TestVertex(superstep < numSupersteps - 1, self.age + 1), Array[TestMessage]()) + } + for ((id, vert) <- result.collect) { + assert(vert.age === numSupersteps) + } + } } } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 6abb5c4792cc2..f6e927a989c12 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -636,16 +636,22 @@ abstract class RDD[T: ClassManifest]( /** The [[spark.SparkContext]] that this RDD was created on. */ def context = sc + // Avoid handling doCheckpoint multiple times to prevent excessive recursion + private var doCheckpointCalled = false + /** * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler * after a job using this RDD has completed (therefore the RDD has been materialized and * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. */ private[spark] def doCheckpoint() { - if (checkpointData.isDefined) { - checkpointData.get.doCheckpoint() - } else { - dependencies.foreach(_.rdd.doCheckpoint()) + if (!doCheckpointCalled) { + doCheckpointCalled = true + if (checkpointData.isDefined) { + checkpointData.get.doCheckpoint() + } else { + dependencies.foreach(_.rdd.doCheckpoint()) + } } } From 582d31dff99c161a51e15497db983a4b5a6d4cdb Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 11 Feb 2013 13:24:54 -0800 Subject: [PATCH 210/696] Formatting fixes --- bagel/src/main/scala/spark/bagel/Bagel.scala | 22 ++++++++------------ 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala index 996ca2a8771e5..fa0ba4a57319e 100644 --- a/bagel/src/main/scala/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/spark/bagel/Bagel.scala @@ -6,8 +6,8 @@ import spark.SparkContext._ import scala.collection.mutable.ArrayBuffer object Bagel extends Logging { - def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, - C : Manifest, A : Manifest]( + def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, + C: Manifest, A: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], messages: RDD[(K, M)], @@ -50,8 +50,7 @@ object Bagel extends Logging { verts } - def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, - C : Manifest]( + def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], messages: RDD[(K, M)], @@ -66,8 +65,7 @@ object Bagel extends Logging { addAggregatorArg[K, V, M, C](compute)) } - def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, - C : Manifest]( + def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], messages: RDD[(K, M)], @@ -82,7 +80,7 @@ object Bagel extends Logging { addAggregatorArg[K, V, M, C](compute)) } - def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( + def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( sc: SparkContext, vertices: RDD[(K, V)], messages: RDD[(K, M)], @@ -100,7 +98,7 @@ object Bagel extends Logging { * Aggregates the given vertices using the given aggregator, if it * is specified. */ - private def agg[K, V <: Vertex, A : Manifest]( + private def agg[K, V <: Vertex, A: Manifest]( verts: RDD[(K, V)], aggregator: Option[Aggregator[V, A]] ): Option[A] = aggregator match { @@ -116,7 +114,7 @@ object Bagel extends Logging { * function. Returns the processed RDD, the number of messages * created, and the number of active vertices. */ - private def comp[K : Manifest, V <: Vertex, M <: Message[K], C]( + private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( sc: SparkContext, grouped: RDD[(K, (Seq[C], Seq[V]))], compute: (V, Option[C]) => (V, Array[M]) @@ -149,9 +147,7 @@ object Bagel extends Logging { * Converts a compute function that doesn't take an aggregator to * one that does, so it can be passed to Bagel.run. */ - private def addAggregatorArg[ - K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C - ]( + private def addAggregatorArg[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C]( compute: (V, Option[C], Int) => (V, Array[M]) ): (V, Option[C], Option[Nothing], Int) => (V, Array[M]) = { (vert: V, msgs: Option[C], aggregated: Option[Nothing], superstep: Int) => @@ -170,7 +166,7 @@ trait Aggregator[V, A] { def mergeAggregators(a: A, b: A): A } -class DefaultCombiner[M : Manifest] extends Combiner[M, Array[M]] with Serializable { +class DefaultCombiner[M: Manifest] extends Combiner[M, Array[M]] with Serializable { def createCombiner(msg: M): Array[M] = Array(msg) def mergeMsg(combiner: Array[M], msg: M): Array[M] = From 21df6ffc13c7ad5cc3158675560c8364735d376e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 11 Feb 2013 17:43:18 -0800 Subject: [PATCH 211/696] SPARK-696: sortByKey should use 'ascending' parameter --- core/src/main/scala/spark/api/java/JavaPairRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 8a123bdb47297..55dc755358ffa 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -452,7 +452,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def sortByKey(ascending: Boolean): JavaPairRDD[K, V] = { val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]] - sortByKey(comp, true) + sortByKey(comp, ascending) } /** From 3f3e77f28b08fc1db110c3b14b2c90eaa6dca8ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 12 Feb 2013 13:57:57 -0800 Subject: [PATCH 212/696] STREAMING-50: Support transform workaround in JavaPairDStream This ports a useful workaround (the `transform` function) to JavaPairDStream. It is necessary to do things like sorting which are not supported yet in the core streaming API. --- .../streaming/api/java/JavaPairDStream.scala | 34 +++++++++++++- .../java/spark/streaming/JavaAPISuite.java | 45 +++++++++++++++++++ 2 files changed, 77 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index ef10c091caaae..eb2495e3acc27 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -8,11 +8,11 @@ import scala.collection.JavaConversions._ import spark.streaming._ import spark.streaming.StreamingContext._ import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import spark.Partitioner +import spark.{RDD, Partitioner} import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.conf.Configuration -import spark.api.java.JavaPairRDD +import spark.api.java.{JavaRDD, JavaPairRDD} import spark.storage.StorageLevel import com.google.common.base.Optional @@ -81,6 +81,36 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def union(that: JavaPairDStream[K, V]): JavaPairDStream[K, V] = dstream.union(that.dstream) + /** + * Return a new DStream in which each RDD is generated by applying a function + * on each RDD of this DStream. + */ + def transform[K2, V2](transformFunc: JFunction[JavaPairRDD[K, V], JavaPairRDD[K2, V2]]): + JavaPairDStream[K2, V2] = { + implicit val cmk: ClassManifest[K2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]] + implicit val cmv: ClassManifest[V2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]] + def scalaTransform (in: RDD[(K, V)]): RDD[(K2, V2)] = + transformFunc.call(new JavaPairRDD[K, V](in)).rdd + dstream.transform(scalaTransform(_)) + } + + /** + * Return a new DStream in which each RDD is generated by applying a function + * on each RDD of this DStream. + */ + def transform[K2, V2](transformFunc: JFunction2[JavaPairRDD[K, V], Time, JavaPairRDD[K2, V2]]): + JavaPairDStream[K2, V2] = { + implicit val cmk: ClassManifest[K2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]] + implicit val cmv: ClassManifest[V2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]] + def scalaTransform (in: RDD[(K, V)], time: Time): RDD[(K2, V2)] = + transformFunc.call(new JavaPairRDD[K, V](in), time).rdd + dstream.transform(scalaTransform(_, _)) + } + // ======================================================================= // Methods only for PairDStream's // ======================================================================= diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 9bfcd83e4d497..7b385f609d302 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -11,6 +11,7 @@ import org.junit.Test; import scala.Tuple2; import spark.HashPartitioner; +import spark.api.java.JavaPairRDD; import spark.api.java.JavaRDD; import spark.api.java.JavaSparkContext; import spark.api.java.function.*; @@ -872,6 +873,50 @@ public void testCountByKeyAndWindow() { Assert.assertEquals(expected, result); } + @Test + public void testPairTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2(3, 5), + new Tuple2(1, 5), + new Tuple2(4, 5), + new Tuple2(2, 5)), + Arrays.asList( + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5), + new Tuple2(1, 5))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, 5), + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5)), + Arrays.asList( + new Tuple2(1, 5), + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream sorted = pairStream.transform( + new Function, JavaPairRDD>() { + @Override + public JavaPairRDD call(JavaPairRDD in) throws Exception { + return in.sortByKey(); + } + }); + + JavaTestUtils.attachTestOutputStream(sorted); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + @Test public void testMapValues() { List>> inputData = stringStringKVStream; From 39addd380363c0371e935fae50983fe87158c1ac Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 13 Feb 2013 12:17:45 -0800 Subject: [PATCH 213/696] Changed scheduler and file input stream to fix bugs in the driver fault tolerance. Added MasterFailureTest to rigorously test master fault tolerance with file input stream. --- .../main/scala/spark/streaming/DStream.scala | 23 +- .../streaming/DStreamCheckpointData.scala | 2 +- .../scala/spark/streaming/DStreamGraph.scala | 49 ++- .../scala/spark/streaming/JobManager.scala | 10 +- .../scala/spark/streaming/Scheduler.scala | 92 +++-- .../src/main/scala/spark/streaming/Time.scala | 10 + .../streaming/dstream/FileInputDStream.scala | 59 ++- .../dstream/NetworkInputDStream.scala | 11 +- .../streaming/util/MasterFailureTest.scala | 375 ++++++++++++++++++ .../spark/streaming/util/RecurringTimer.scala | 30 +- .../java/spark/streaming/JavaAPISuite.java | 21 +- streaming/src/test/resources/log4j.properties | 7 +- .../streaming/BasicOperationsSuite.scala | 2 + .../spark/streaming/CheckpointSuite.scala | 107 +++-- .../scala/spark/streaming/FailureSuite.scala | 304 +------------- .../spark/streaming/InputStreamsSuite.scala | 29 +- .../scala/spark/streaming/TestSuiteBase.scala | 12 +- .../streaming/WindowOperationsSuite.scala | 2 + 18 files changed, 693 insertions(+), 452 deletions(-) create mode 100644 streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 0eb6aad18721a..0c1b667c0ab42 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -292,7 +292,7 @@ abstract class DStream[T: ClassManifest] ( * Generate a SparkStreaming job for the given time. This is an internal method that * should not be called directly. This default implementation creates a job * that materializes the corresponding RDD. Subclasses of DStream may override this - * (eg. ForEachDStream). + * to generate their own jobs. */ protected[streaming] def generateJob(time: Time): Option[Job] = { getOrCompute(time) match { @@ -308,19 +308,18 @@ abstract class DStream[T: ClassManifest] ( } /** - * Dereference RDDs that are older than rememberDuration. + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This default + * implementation clears the old generated RDDs. Subclasses of DStream may override + * this to clear their own metadata along with the generated RDDs. */ - protected[streaming] def forgetOldMetadata(time: Time) { + protected[streaming] def clearOldMetadata(time: Time) { var numForgotten = 0 - generatedRDDs.keys.foreach(t => { - if (t <= (time - rememberDuration)) { - generatedRDDs.remove(t) - numForgotten += 1 - logInfo("Forgot RDD of time " + t + " from " + this) - } - }) - logInfo("Forgot " + numForgotten + " RDDs from " + this) - dependencies.foreach(_.forgetOldMetadata(time)) + val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) + generatedRDDs --= oldRDDs.keys + logInfo("Cleared " + oldRDDs.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) + dependencies.foreach(_.clearOldMetadata(time)) } /* Adds metadata to the Stream while it is running. diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala index a375980b84a5c..6b0fade7c64d7 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -87,7 +87,7 @@ class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) } override def toString() = { - "[\n" + checkpointFiles.size + "\n" + checkpointFiles.mkString("\n") + "\n]" + "[\n" + checkpointFiles.size + " checkpoint files \n" + checkpointFiles.mkString("\n") + "\n]" } } diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index 7aa9d20004630..22d9e24f05b99 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -11,17 +11,20 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { private val inputStreams = new ArrayBuffer[InputDStream[_]]() private val outputStreams = new ArrayBuffer[DStream[_]]() - private[streaming] var zeroTime: Time = null - private[streaming] var batchDuration: Duration = null - private[streaming] var rememberDuration: Duration = null - private[streaming] var checkpointInProgress = false + var rememberDuration: Duration = null + var checkpointInProgress = false - private[streaming] def start(time: Time) { + var zeroTime: Time = null + var startTime: Time = null + var batchDuration: Duration = null + + def start(time: Time) { this.synchronized { if (zeroTime != null) { throw new Exception("DStream graph computation already started") } zeroTime = time + startTime = time outputStreams.foreach(_.initialize(zeroTime)) outputStreams.foreach(_.remember(rememberDuration)) outputStreams.foreach(_.validate) @@ -29,19 +32,23 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } } - private[streaming] def stop() { + def restart(time: Time) { + this.synchronized { startTime = time } + } + + def stop() { this.synchronized { inputStreams.par.foreach(_.stop()) } } - private[streaming] def setContext(ssc: StreamingContext) { + def setContext(ssc: StreamingContext) { this.synchronized { outputStreams.foreach(_.setContext(ssc)) } } - private[streaming] def setBatchDuration(duration: Duration) { + def setBatchDuration(duration: Duration) { this.synchronized { if (batchDuration != null) { throw new Exception("Batch duration already set as " + batchDuration + @@ -51,61 +58,61 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { batchDuration = duration } - private[streaming] def remember(duration: Duration) { + def remember(duration: Duration) { this.synchronized { if (rememberDuration != null) { throw new Exception("Batch duration already set as " + batchDuration + ". cannot set it again.") } + rememberDuration = duration } - rememberDuration = duration } - private[streaming] def addInputStream(inputStream: InputDStream[_]) { + def addInputStream(inputStream: InputDStream[_]) { this.synchronized { inputStream.setGraph(this) inputStreams += inputStream } } - private[streaming] def addOutputStream(outputStream: DStream[_]) { + def addOutputStream(outputStream: DStream[_]) { this.synchronized { outputStream.setGraph(this) outputStreams += outputStream } } - private[streaming] def getInputStreams() = this.synchronized { inputStreams.toArray } + def getInputStreams() = this.synchronized { inputStreams.toArray } - private[streaming] def getOutputStreams() = this.synchronized { outputStreams.toArray } + def getOutputStreams() = this.synchronized { outputStreams.toArray } - private[streaming] def generateRDDs(time: Time): Seq[Job] = { + def generateRDDs(time: Time): Seq[Job] = { this.synchronized { logInfo("Generating RDDs for time " + time) outputStreams.flatMap(outputStream => outputStream.generateJob(time)) } } - private[streaming] def forgetOldRDDs(time: Time) { + def clearOldMetadata(time: Time) { this.synchronized { - logInfo("Forgetting old RDDs for time " + time) - outputStreams.foreach(_.forgetOldMetadata(time)) + logInfo("Clearing old metadata for time " + time) + outputStreams.foreach(_.clearOldMetadata(time)) } } - private[streaming] def updateCheckpointData(time: Time) { + def updateCheckpointData(time: Time) { this.synchronized { outputStreams.foreach(_.updateCheckpointData(time)) } } - private[streaming] def restoreCheckpointData() { + def restoreCheckpointData() { this.synchronized { outputStreams.foreach(_.restoreCheckpointData()) } } - private[streaming] def validate() { + def validate() { this.synchronized { assert(batchDuration != null, "Batch duration has not been set") //assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + " is very low") diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala index 8b18c7bc6a8c2..649494ff4a8d6 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/spark/streaming/JobManager.scala @@ -38,13 +38,19 @@ class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging { logInfo("Added " + job + " to queue") } + def stop() { + jobExecutor.shutdown() + } + private def clearJob(job: Job) { jobs.synchronized { - val jobsOfTime = jobs.get(job.time) + val time = job.time + val jobsOfTime = jobs.get(time) if (jobsOfTime.isDefined) { jobsOfTime.get -= job if (jobsOfTime.get.isEmpty) { - jobs -= job.time + ssc.scheduler.clearOldMetadata(time) + jobs -= time } } else { throw new Exception("Job finished for time " + job.time + diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala index 23a0f0974d3d4..57d494da8378e 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala @@ -9,11 +9,8 @@ class Scheduler(ssc: StreamingContext) extends Logging { initLogging() - val graph = ssc.graph - val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt val jobManager = new JobManager(ssc, concurrentJobs) - val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { new CheckpointWriter(ssc.checkpointDir) } else { @@ -24,53 +21,80 @@ class Scheduler(ssc: StreamingContext) extends Logging { val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock] val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, longTime => generateRDDs(new Time(longTime))) + val graph = ssc.graph - def start() { - // If context was started from checkpoint, then restart timer such that - // this timer's triggers occur at the same time as the original timer. - // Otherwise just start the timer from scratch, and initialize graph based - // on this first trigger time of the timer. + def start() = synchronized { if (ssc.isCheckpointPresent) { - // If manual clock is being used for testing, then - // either set the manual clock to the last checkpointed time, - // or if the property is defined set it to that time - if (clock.isInstanceOf[ManualClock]) { - val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong - clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) - } - // Reschedule the batches that were received but not processed before failure - //ssc.initialCheckpoint.pendingTimes.foreach(time => generateRDDs(time)) - val pendingTimes = ssc.initialCheckpoint.pendingTimes.sorted(Time.ordering) - println(pendingTimes.mkString(", ")) - pendingTimes.foreach(time => - graph.generateRDDs(time).foreach(jobManager.runJob)) - // Restart the timer - timer.restart(graph.zeroTime.milliseconds) - logInfo("Scheduler's timer restarted") + restart() } else { - val firstTime = new Time(timer.start()) - graph.start(firstTime - ssc.graph.batchDuration) - logInfo("Scheduler's timer started") + startFirstTime() } logInfo("Scheduler started") } - def stop() { + def stop() = synchronized { timer.stop() - graph.stop() + jobManager.stop() + ssc.graph.stop() logInfo("Scheduler stopped") } - - private def generateRDDs(time: Time) { + + private def startFirstTime() { + val startTime = new Time(timer.getStartTime()) + graph.start(startTime - graph.batchDuration) + timer.start(startTime.milliseconds) + logInfo("Scheduler's timer started at " + startTime) + } + + private def restart() { + + // If manual clock is being used for testing, then + // either set the manual clock to the last checkpointed time, + // or if the property is defined set it to that time + if (clock.isInstanceOf[ManualClock]) { + val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds + val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong + clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) + } + + val batchDuration = ssc.graph.batchDuration + + // Batches when the master was down, that is, + // between the checkpoint and current restart time + val checkpointTime = ssc.initialCheckpoint.checkpointTime + val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds)) + val downTimes = checkpointTime.until(restartTime, batchDuration) + logInfo("Batches during down time: " + downTimes.mkString(", ")) + + // Batches that were unprocessed before failure + val pendingTimes = ssc.initialCheckpoint.pendingTimes + logInfo("Batches pending processing: " + pendingTimes.mkString(", ")) + // Reschedule jobs for these times + val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering) + logInfo("Batches to reschedule: " + timesToReschedule.mkString(", ")) + timesToReschedule.foreach(time => + graph.generateRDDs(time).foreach(jobManager.runJob) + ) + + // Restart the timer + timer.start(restartTime.milliseconds) + logInfo("Scheduler's timer restarted") + } + + /** Generates the RDDs, clears old metadata and does checkpoint for the given time */ + def generateRDDs(time: Time) { SparkEnv.set(ssc.env) logInfo("\n-----------------------------------------------------\n") graph.generateRDDs(time).foreach(jobManager.runJob) - graph.forgetOldRDDs(time) doCheckpoint(time) } - private def doCheckpoint(time: Time) { + + def clearOldMetadata(time: Time) { + ssc.graph.clearOldMetadata(time) + } + + def doCheckpoint(time: Time) { if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { logInfo("Checkpointing graph for time " + time) val startTime = System.currentTimeMillis() diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala index 8a6c9a5cb5ff6..8201e84a203d7 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/spark/streaming/Time.scala @@ -37,6 +37,16 @@ case class Time(private val millis: Long) { def max(that: Time): Time = if (this > that) this else that + def until(that: Time, interval: Duration): Seq[Time] = { + assert(that > this, "Cannot create sequence as " + that + " not more than " + this) + assert( + (that - this).isMultipleOf(interval), + "Cannot create sequence as gap between " + that + " and " + + this + " is not multiple of " + interval + ) + (this.milliseconds) until (that.milliseconds) by (interval.milliseconds) map (new Time(_)) + } + override def toString: String = (millis.toString + " ms") } diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index 10ccb4318dfd6..41b9bd9461d9c 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -21,19 +21,21 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData + // Latest file mod time seen till any point of time private val lastModTimeFiles = new HashSet[String]() private var lastModTime = 0L @transient private var path_ : Path = null @transient private var fs_ : FileSystem = null - @transient private var files = new HashMap[Time, Array[String]] + @transient private[streaming] var files = new HashMap[Time, Array[String]] override def start() { if (newFilesOnly) { - lastModTime = System.currentTimeMillis() + lastModTime = graph.zeroTime.milliseconds } else { lastModTime = 0 } + logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly) } override def stop() { } @@ -43,38 +45,50 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K * a union RDD out of them. Note that this maintains the list of files that were processed * in the latest modification time in the previous call to this method. This is because the * modification time returned by the FileStatus API seems to return times only at the - * granularity of seconds. Hence, new files may have the same modification time as the - * latest modification time in the previous call to this method and the list of files - * maintained is used to filter the one that have been processed. + * granularity of seconds. And new files may have the same modification time as the + * latest modification time in the previous call to this method yet was not reported in + * the previous call. */ override def compute(validTime: Time): Option[RDD[(K, V)]] = { + assert(validTime.milliseconds >= lastModTime, "Trying to get new files for really old time [" + validTime + " < " + lastModTime) + // Create the filter for selecting new files val newFilter = new PathFilter() { + // Latest file mod time seen in this round of fetching files and its corresponding files var latestModTime = 0L val latestModTimeFiles = new HashSet[String]() def accept(path: Path): Boolean = { - if (!filter(path)) { + if (!filter(path)) { // Reject file if it does not satisfy filter + logDebug("Rejected by filter " + path) return false - } else { + } else { // Accept file only if val modTime = fs.getFileStatus(path).getModificationTime() - if (modTime < lastModTime){ - return false + logDebug("Mod time for " + path + " is " + modTime) + if (modTime < lastModTime) { + logDebug("Mod time less than last mod time") + return false // If the file was created before the last time it was called } else if (modTime == lastModTime && lastModTimeFiles.contains(path.toString)) { - return false + logDebug("Mod time equal to last mod time, but file considered already") + return false // If the file was created exactly as lastModTime but not reported yet + } else if (modTime > validTime.milliseconds) { + logDebug("Mod time more than valid time") + return false // If the file was created after the time this function call requires } if (modTime > latestModTime) { latestModTime = modTime latestModTimeFiles.clear() + logDebug("Latest mod time updated to " + latestModTime) } latestModTimeFiles += path.toString + logDebug("Accepted " + path) return true } } } - + logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime) val newFiles = fs.listStatus(path, newFilter).map(_.getPath.toString) - logInfo("New files: " + newFiles.mkString(", ")) + logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n")) if (newFiles.length > 0) { // Update the modification time and the files processed for that modification time if (lastModTime != newFilter.latestModTime) { @@ -82,17 +96,21 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K lastModTimeFiles.clear() } lastModTimeFiles ++= newFilter.latestModTimeFiles + logDebug("Last mod time updated to " + lastModTime) } files += ((validTime, newFiles)) Some(filesToRDD(newFiles)) } - /** Forget the old time-to-files mappings along with old RDDs */ - protected[streaming] override def forgetOldMetadata(time: Time) { - super.forgetOldMetadata(time) - val filesToBeRemoved = files.filter(_._1 <= (time - rememberDuration)) - files --= filesToBeRemoved.keys - logInfo("Forgot " + filesToBeRemoved.size + " files from " + this) + /** Clear the old time-to-files mappings along with old RDDs */ + protected[streaming] override def clearOldMetadata(time: Time) { + super.clearOldMetadata(time) + val oldFiles = files.filter(_._1 <= (time - rememberDuration)) + files --= oldFiles.keys + logInfo("Cleared " + oldFiles.size + " old files that were older than " + + (time - rememberDuration) + ": " + oldFiles.keys.mkString(", ")) + logDebug("Cleared files are:\n" + + oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) } /** Generate one RDD from an array of files */ @@ -148,6 +166,11 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K } } } + + override def toString() = { + "[\n" + hadoopFiles.size + " file sets\n" + + hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]" + } } } diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 8c322dd698099..ecc75ec913af8 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -46,8 +46,15 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming def stop() {} override def compute(validTime: Time): Option[RDD[T]] = { - val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime) - Some(new BlockRDD[T](ssc.sc, blockIds)) + // If this is called for any time before the start time of the context, + // then this returns an empty RDD. This may happen when recovering from a + // master failure forces + if (validTime >= graph.startTime) { + val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime) + Some(new BlockRDD[T](ssc.sc, blockIds)) + } else { + Some(new BlockRDD[T](ssc.sc, Array[String]())) + } } } diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala new file mode 100644 index 0000000000000..3ffe4b64d0873 --- /dev/null +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -0,0 +1,375 @@ +package spark.streaming.util + +import spark.{Logging, RDD} +import spark.streaming._ +import spark.streaming.dstream.ForEachDStream +import StreamingContext._ + +import scala.util.Random +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} + +import java.io.{File, ObjectInputStream, IOException} +import java.util.UUID + +import com.google.common.io.Files + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.fs.{FileUtil, FileSystem, Path} +import org.apache.hadoop.conf.Configuration + + +private[streaming] +object MasterFailureTest extends Logging { + initLogging() + + @volatile var killed = false + @volatile var killCount = 0 + + def main(args: Array[String]) { + if (args.size < 2) { + println( + "Usage: MasterFailureTest <# batches> []") + System.exit(1) + } + val directory = args(0) + val numBatches = args(1).toInt + val batchDuration = if (args.size > 2) Milliseconds(args(2).toInt) else Seconds(1) + + println("\n\n========================= MAP TEST =========================\n\n") + testMap(directory, numBatches, batchDuration) + + println("\n\n================= UPDATE-STATE-BY-KEY TEST =================\n\n") + testUpdateStateByKey(directory, numBatches, batchDuration) + } + + def testMap(directory: String, numBatches: Int, batchDuration: Duration) { + // Input: time=1 ==> [ 1 ] , time=2 ==> [ 2 ] , time=3 ==> [ 3 ] , ... + val input = (1 to numBatches).map(_.toString).toSeq + // Expected output: time=1 ==> [ 1 ] , time=2 ==> [ 2 ] , time=3 ==> [ 3 ] , ... + val expectedOutput = (1 to numBatches) + + val operation = (st: DStream[String]) => st.map(_.toInt) + + // Run streaming operation with multiple master failures + val output = testOperation(directory, batchDuration, input, operation, expectedOutput) + + logInfo("Expected output, size = " + expectedOutput.size) + logInfo(expectedOutput.mkString("[", ",", "]")) + logInfo("Output, size = " + output.size) + logInfo(output.mkString("[", ",", "]")) + + // Verify whether all the values of the expected output is present + // in the output + assert(output.distinct.toSet == expectedOutput.toSet) + } + + + def testUpdateStateByKey(directory: String, numBatches: Int, batchDuration: Duration) { + // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ... + val input = (1 to numBatches).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq + // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ... + val expectedOutput = (1L to numBatches).map(i => (1L to i).reduce(_ + _)).map(j => ("a", j)) + + val operation = (st: DStream[String]) => { + val updateFunc = (values: Seq[Long], state: Option[Long]) => { + Some(values.foldLeft(0L)(_ + _) + state.getOrElse(0L)) + } + st.flatMap(_.split(" ")) + .map(x => (x, 1L)) + .updateStateByKey[Long](updateFunc) + .checkpoint(batchDuration * 5) + } + + // Run streaming operation with multiple master failures + val output = testOperation(directory, batchDuration, input, operation, expectedOutput) + + logInfo("Expected output, size = " + expectedOutput.size + "\n" + expectedOutput) + logInfo("Output, size = " + output.size + "\n" + output) + + // Verify whether all the values in the output are among the expected output values + output.foreach(o => + assert(expectedOutput.contains(o), "Expected value " + o + " not found") + ) + + // Verify whether the last expected output value has been generated, there by + // confirming that none of the inputs have been missed + assert(output.last == expectedOutput.last) + } + + /** + * Tests stream operation with multiple master failures, and verifies whether the + * final set of output values is as expected or not. + */ + def testOperation[T: ClassManifest]( + directory: String, + batchDuration: Duration, + input: Seq[String], + operation: DStream[String] => DStream[T], + expectedOutput: Seq[T] + ): Seq[T] = { + + // Just making sure that the expected output does not have duplicates + assert(expectedOutput.distinct.toSet == expectedOutput.toSet) + + // Setup the stream computation with the given operation + val (ssc, checkpointDir, testDir) = setupStreams(directory, batchDuration, operation) + + // Start generating files in the a different thread + val fileGeneratingThread = new FileGeneratingThread(input, testDir, batchDuration.milliseconds) + fileGeneratingThread.start() + + // Run the streams and repeatedly kill it until the last expected output + // has been generated, or until it has run for twice the expected time + val lastExpectedOutput = expectedOutput.last + val maxTimeToRun = expectedOutput.size * batchDuration.milliseconds * 2 + val mergedOutput = runStreams(ssc, lastExpectedOutput, maxTimeToRun) + + // Delete directories + fileGeneratingThread.join() + val fs = checkpointDir.getFileSystem(new Configuration()) + fs.delete(checkpointDir, true) + fs.delete(testDir, true) + logInfo("Finished test after " + killCount + " failures") + mergedOutput + } + + /** + * Sets up the stream computation with the given operation, directory (local or HDFS), + * and batch duration. Returns the streaming context and the directory to which + * files should be written for testing. + */ + private def setupStreams[T: ClassManifest]( + directory: String, + batchDuration: Duration, + operation: DStream[String] => DStream[T] + ): (StreamingContext, Path, Path) = { + // Reset all state + reset() + + // Create the directories for this test + val uuid = UUID.randomUUID().toString + val rootDir = new Path(directory, uuid) + val fs = rootDir.getFileSystem(new Configuration()) + val checkpointDir = new Path(rootDir, "checkpoint") + val testDir = new Path(rootDir, "test") + fs.mkdirs(checkpointDir) + fs.mkdirs(testDir) + + // Setup the streaming computation with the given operation + System.clearProperty("spark.driver.port") + var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration) + ssc.checkpoint(checkpointDir.toString) + val inputStream = ssc.textFileStream(testDir.toString) + val operatedStream = operation(inputStream) + val outputStream = new TestOutputStream(operatedStream) + ssc.registerOutputStream(outputStream) + (ssc, checkpointDir, testDir) + } + + + /** + * Repeatedly starts and kills the streaming context until timed out or + * the last expected output is generated. Finally, return + */ + private def runStreams[T: ClassManifest]( + ssc_ : StreamingContext, + lastExpectedOutput: T, + maxTimeToRun: Long + ): Seq[T] = { + + var ssc = ssc_ + var totalTimeRan = 0L + var isLastOutputGenerated = false + var isTimedOut = false + val mergedOutput = new ArrayBuffer[T]() + val checkpointDir = ssc.checkpointDir + var batchDuration = ssc.graph.batchDuration + + while(!isLastOutputGenerated && !isTimedOut) { + // Get the output buffer + val outputBuffer = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[T]].output + def output = outputBuffer.flatMap(x => x) + + // Start the thread to kill the streaming after some time + killed = false + val killingThread = new KillingThread(ssc, batchDuration.milliseconds * 10) + killingThread.start() + + var timeRan = 0L + try { + // Start the streaming computation and let it run while ... + // (i) StreamingContext has not been shut down yet + // (ii) The last expected output has not been generated yet + // (iii) Its not timed out yet + System.clearProperty("spark.streaming.clock") + System.clearProperty("spark.driver.port") + ssc.start() + val startTime = System.currentTimeMillis() + while (!killed && !isLastOutputGenerated && !isTimedOut) { + Thread.sleep(100) + timeRan = System.currentTimeMillis() - startTime + isLastOutputGenerated = (!output.isEmpty && output.last == lastExpectedOutput) + isTimedOut = (timeRan + totalTimeRan > maxTimeToRun) + } + } catch { + case e: Exception => logError("Error running streaming context", e) + } + if (killingThread.isAlive) killingThread.interrupt() + ssc.stop() + + logInfo("Has been killed = " + killed) + logInfo("Is last output generated = " + isLastOutputGenerated) + logInfo("Is timed out = " + isTimedOut) + + // Verify whether the output of each batch has only one element or no element + // and then merge the new output with all the earlier output + mergedOutput ++= output + totalTimeRan += timeRan + logInfo("New output = " + output) + logInfo("Merged output = " + mergedOutput) + logInfo("Time ran = " + timeRan) + logInfo("Total time ran = " + totalTimeRan) + + if (!isLastOutputGenerated && !isTimedOut) { + val sleepTime = Random.nextInt(batchDuration.milliseconds.toInt * 10) + logInfo( + "\n-------------------------------------------\n" + + " Restarting stream computation in " + sleepTime + " ms " + + "\n-------------------------------------------\n" + ) + Thread.sleep(sleepTime) + // Recreate the streaming context from checkpoint + ssc = new StreamingContext(checkpointDir) + } + } + mergedOutput + } + + /** + * Verifies the output value are the same as expected. Since failures can lead to + * a batch being processed twice, a batches output may appear more than once + * consecutively. To avoid getting confused with those, we eliminate consecutive + * duplicate batch outputs of values from the `output`. As a result, the + * expected output should not have consecutive batches with the same values as output. + */ + private def verifyOutput[T: ClassManifest](output: Seq[T], expectedOutput: Seq[T]) { + // Verify whether expected outputs do not consecutive batches with same output + for (i <- 0 until expectedOutput.size - 1) { + assert(expectedOutput(i) != expectedOutput(i+1), + "Expected output has consecutive duplicate sequence of values") + } + + // Log the output + println("Expected output, size = " + expectedOutput.size) + println(expectedOutput.mkString("[", ",", "]")) + println("Output, size = " + output.size) + println(output.mkString("[", ",", "]")) + + // Match the output with the expected output + output.foreach(o => + assert(expectedOutput.contains(o), "Expected value " + o + " not found") + ) + } + + /** Resets counter to prepare for the test */ + private def reset() { + killed = false + killCount = 0 + } +} + +/** + * This is a output stream just for testing. All the output is collected into a + * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. + */ +private[streaming] +class TestOutputStream[T: ClassManifest]( + parent: DStream[T], + val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]] + ) extends ForEachDStream[T]( + parent, + (rdd: RDD[T], t: Time) => { + val collected = rdd.collect() + output += collected + println(t + ": " + collected.mkString("[", ",", "]")) + } + ) { + + // This is to clear the output buffer every it is read from a checkpoint + @throws(classOf[IOException]) + private def readObject(ois: ObjectInputStream) { + ois.defaultReadObject() + output.clear() + } +} + + +/** + * Thread to kill streaming context after a random period of time. + */ +private[streaming] +class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging { + initLogging() + + override def run() { + try { + // If it is the first killing, then allow the first checkpoint to be created + var minKillWaitTime = if (MasterFailureTest.killCount == 0) 5000 else 1000 + val killWaitTime = minKillWaitTime + math.abs(Random.nextLong % maxKillWaitTime) + logInfo("Kill wait time = " + killWaitTime) + Thread.sleep(killWaitTime) + logInfo( + "\n---------------------------------------\n" + + "Killing streaming context after " + killWaitTime + " ms" + + "\n---------------------------------------\n" + ) + if (ssc != null) { + ssc.stop() + MasterFailureTest.killed = true + MasterFailureTest.killCount += 1 + } + logInfo("Killing thread finished normally") + } catch { + case ie: InterruptedException => logInfo("Killing thread interrupted") + case e: Exception => logWarning("Exception in killing thread", e) + } + + } +} + + +/** + * Thread to generate input files periodically with the desired text. + */ +private[streaming] +class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) + extends Thread with Logging { + initLogging() + + override def run() { + val localTestDir = Files.createTempDir() + val fs = testDir.getFileSystem(new Configuration()) + try { + Thread.sleep(5000) // To make sure that all the streaming context has been set up + for (i <- 0 until input.size) { + // Write the data to a local file and then move it to the target test directory + val localFile = new File(localTestDir, (i+1).toString) + val hadoopFile = new Path(testDir, (i+1).toString) + FileUtils.writeStringToFile(localFile, input(i).toString + "\n") + //fs.moveFromLocalFile(new Path(localFile.toString), new Path(testDir, i.toString)) + fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) + logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis) + Thread.sleep(interval) + localFile.delete() + } + logInfo("File generating thread finished normally") + } catch { + case ie: InterruptedException => logInfo("File generating thread interrupted") + case e: Exception => logWarning("File generating in killing thread", e) + } finally { + fs.close() + } + } +} + + diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala index db715cc295205..8e10276deb905 100644 --- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala @@ -3,9 +3,9 @@ package spark.streaming.util private[streaming] class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) { - val minPollTime = 25L + private val minPollTime = 25L - val pollTime = { + private val pollTime = { if (period / 10.0 > minPollTime) { (period / 10.0).toLong } else { @@ -13,11 +13,20 @@ class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => } } - val thread = new Thread() { + private val thread = new Thread() { override def run() { loop } } - var nextTime = 0L + private var nextTime = 0L + + def getStartTime(): Long = { + (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period + } + + def getRestartTime(originalStartTime: Long): Long = { + val gap = clock.currentTime - originalStartTime + (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime + } def start(startTime: Long): Long = { nextTime = startTime @@ -26,21 +35,14 @@ class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => } def start(): Long = { - val startTime = (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period - start(startTime) + start(getStartTime()) } - def restart(originalStartTime: Long): Long = { - val gap = clock.currentTime - originalStartTime - val newStartTime = (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime - start(newStartTime) - } - - def stop() { + def stop() { thread.interrupt() } - def loop() { + private def loop() { try { while (true) { clock.waitTillTime(nextTime) diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index fbe4af4597e3c..783a393a8ffef 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -33,7 +33,8 @@ public class JavaAPISuite implements Serializable { @Before public void setUp() { - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); ssc.checkpoint("checkpoint", new Duration(1000)); } @@ -45,7 +46,7 @@ public void tearDown() { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port"); } - /* + @Test public void testCount() { List> inputData = Arrays.asList( @@ -434,7 +435,7 @@ public void testUnion() { assertOrderInvariantEquals(expected, result); } - */ + /* * Performs an order-invariant comparison of lists representing two RDD streams. This allows * us to account for ordering variation within individual RDD's which occurs during windowing. @@ -450,7 +451,7 @@ public static void assertOrderInvariantEquals( Assert.assertEquals(expected, actual); } - /* + // PairDStream Functions @Test public void testPairFilter() { @@ -897,7 +898,7 @@ public void testJoin() { Assert.assertEquals(expected, result); } - */ + @Test public void testCheckpointMasterRecovery() throws InterruptedException { List> inputData = Arrays.asList( @@ -964,7 +965,7 @@ public Integer call(String s) throws Exception { assertOrderInvariantEquals(expected, result1); } */ - /* + // Input stream tests. These mostly just test that we can instantiate a given InputStream with // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the // InputStream functionality is deferred to the existing Scala tests. @@ -972,9 +973,9 @@ public Integer call(String s) throws Exception { public void testKafkaStream() { HashMap topics = Maps.newHashMap(); HashMap offsets = Maps.newHashMap(); - JavaDStream test1 = ssc.kafkaStream("localhost", 12345, "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost", 12345, "group", topics, offsets); - JavaDStream test3 = ssc.kafkaStream("localhost", 12345, "group", topics, offsets, + JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, offsets); + JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, offsets, StorageLevel.MEMORY_AND_DISK()); } @@ -1026,5 +1027,5 @@ public void testFlumeStream() { public void testFileStream() { JavaPairDStream foo = ssc.fileStream("/tmp/foo"); - }*/ + } } diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index edfa1243fa2bc..5652596e1e4b9 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -1,6 +1,7 @@ # Set everything to be logged to the file streaming/target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender +log4j.rootCategory=WARN, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.ConsoleAppender log4j.appender.file.append=false log4j.appender.file.file=streaming/target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout @@ -8,4 +9,6 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN +log4j.logger.spark.streaming=INFO +log4j.logger.spark.streaming.dstream.FileInputDStream=DEBUG diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index c031949dd158c..12388b888737b 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -6,6 +6,8 @@ import util.ManualClock class BasicOperationsSuite extends TestSuiteBase { + System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + override def framework() = "BasicOperationsSuite" after { diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index 7126af62d9cad..c89c4a8d43f80 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -1,5 +1,6 @@ package spark.streaming +import dstream.FileInputDStream import spark.streaming.StreamingContext._ import java.io.File import runtime.RichInt @@ -10,8 +11,16 @@ import util.{Clock, ManualClock} import scala.util.Random import com.google.common.io.Files + +/** + * This test suites tests the checkpointing functionality of DStreams - + * the checkpointing of a DStream's RDDs as well as the checkpointing of + * the whole DStream graph. + */ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { + System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + before { FileUtils.deleteDirectory(new File(checkpointDir)) } @@ -64,7 +73,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Run till a time such that at least one RDD in the stream should have been checkpointed, // then check whether some RDD has been checkpointed or not ssc.start() - runStreamsWithRealDelay(ssc, firstNumBatches) + advanceTimeWithRealDelay(ssc, firstNumBatches) logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData) assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure") stateStream.checkpointData.checkpointFiles.foreach { @@ -77,7 +86,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Run till a further time such that previous checkpoint files in the stream would be deleted // and check whether the earlier checkpoint files are deleted val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2)) - runStreamsWithRealDelay(ssc, secondNumBatches) + advanceTimeWithRealDelay(ssc, secondNumBatches) checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted")) ssc.stop() @@ -92,7 +101,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Run one batch to generate a new checkpoint file and check whether some RDD // is present in the checkpoint data or not ssc.start() - runStreamsWithRealDelay(ssc, 1) + advanceTimeWithRealDelay(ssc, 1) assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure") stateStream.checkpointData.checkpointFiles.foreach { case (time, data) => { @@ -113,7 +122,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Adjust manual clock time as if it is being restarted after a delay System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) ssc.start() - runStreamsWithRealDelay(ssc, 4) + advanceTimeWithRealDelay(ssc, 4) ssc.stop() System.clearProperty("spark.streaming.manualClock.jump") ssc = null @@ -168,74 +177,95 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { } // This tests whether file input stream remembers what files were seen before - // the master failure and uses them again to process a large window operatoin. + // the master failure and uses them again to process a large window operation. // It also tests whether batches, whose processing was incomplete due to the // failure, are re-processed or not. test("recovery with file input stream") { + // Disable manual clock as FileInputDStream does not work with manual clock + val clockProperty = System.getProperty("spark.streaming.clock") + System.clearProperty("spark.streaming.clock") + // Set up the streaming context and input streams val testDir = Files.createTempDir() - var ssc = new StreamingContext(master, framework, batchDuration) + var ssc = new StreamingContext(master, framework, Seconds(1)) ssc.checkpoint(checkpointDir, checkpointInterval) val fileStream = ssc.textFileStream(testDir.toString) // Making value 3 take large time to process, to ensure that the master // shuts down in the middle of processing the 3rd batch val mappedStream = fileStream.map(s => { val i = s.toInt - if (i == 3) Thread.sleep(1000) + if (i == 3) Thread.sleep(2000) i }) + // Reducing over a large window to ensure that recovery from master failure // requires reprocessing of all the files seen before the failure - val reducedStream = mappedStream.reduceByWindow(_ + _, batchDuration * 30, batchDuration) + val reducedStream = mappedStream.reduceByWindow(_ + _, Seconds(30), Seconds(1)) val outputBuffer = new ArrayBuffer[Seq[Int]] var outputStream = new TestOutputStream(reducedStream, outputBuffer) ssc.registerOutputStream(outputStream) ssc.start() // Create files and advance manual clock to process them - var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + //var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] Thread.sleep(1000) for (i <- Seq(1, 2, 3)) { FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") // wait to make sure that the file is written such that it gets shown in the file listings - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - // wait to make sure that FileInputDStream picks up this file only and not any other file - Thread.sleep(500) + Thread.sleep(1000) } logInfo("Output = " + outputStream.output.mkString(",")) assert(outputStream.output.size > 0, "No files processed before restart") ssc.stop() + // Verify whether files created have been recorded correctly or not + var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] + def recordedFiles = fileInputDStream.files.values.flatMap(x => x) + assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) + // Create files while the master is down for (i <- Seq(4, 5, 6)) { FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") Thread.sleep(1000) } - // Restart stream computation from checkpoint and create more files to see whether - // they are being processed + // Recover context from checkpoint file and verify whether the files that were + // recorded before failure were saved and successfully recovered logInfo("*********** RESTARTING ************") ssc = new StreamingContext(checkpointDir) + fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] + assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) + + // Restart stream computation ssc.start() - clock = ssc.scheduler.clock.asInstanceOf[ManualClock] for (i <- Seq(7, 8, 9)) { FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - Thread.sleep(500) + Thread.sleep(1000) } Thread.sleep(1000) - logInfo("Output = " + outputStream.output.mkString(",")) + logInfo("Output = " + outputStream.output.mkString("[", ", ", "]")) assert(outputStream.output.size > 0, "No files processed after restart") ssc.stop() + // Verify whether files created while the driver was down have been recorded or not + assert(!recordedFiles.filter(_.endsWith("4")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("5")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("6")).isEmpty) + + // Verify whether new files created after recover have been recorded or not + assert(!recordedFiles.filter(_.endsWith("7")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("8")).isEmpty) + assert(!recordedFiles.filter(_.endsWith("9")).isEmpty) + // Append the new output to the old buffer outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] outputBuffer ++= outputStream.output - // Verify whether data received by Spark Streaming was as expected - val expectedOutput = Seq(1, 3, 6, 28, 36, 45) + val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45) logInfo("--------------------------------") logInfo("output, size = " + outputBuffer.size) outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) @@ -244,11 +274,17 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { logInfo("--------------------------------") // Verify whether all the elements received are as expected - assert(outputBuffer.size === expectedOutput.size) - for (i <- 0 until outputBuffer.size) { - assert(outputBuffer(i).size === 1) - assert(outputBuffer(i).head === expectedOutput(i)) - } + val output = outputBuffer.flatMap(x => x) + assert(output.contains(6)) // To ensure that the 3rd input (i.e., 3) was processed + output.foreach(o => // To ensure all the inputs are correctly added cumulatively + assert(expectedOutput.contains(o), "Expected value " + o + " not found") + ) + // To ensure that all the inputs were received correctly + assert(expectedOutput.last === output.last) + + // Enable manual clock back again for other tests + if (clockProperty != null) + System.setProperty("spark.streaming.clock", clockProperty) } @@ -278,7 +314,9 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Do the computation for initial number of batches, create checkpoint file and quit ssc = setupStreams[U, V](input, operation) - val output = runStreams[V](ssc, initialNumBatches, initialNumExpectedOutputs) + ssc.start() + val output = advanceTimeWithRealDelay[V](ssc, initialNumBatches) + ssc.stop() verifyOutput[V](output, expectedOutput.take(initialNumBatches), true) Thread.sleep(1000) @@ -289,17 +327,20 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { "\n-------------------------------------------\n" ) ssc = new StreamingContext(checkpointDir) - val outputNew = runStreams[V](ssc, nextNumBatches, nextNumExpectedOutputs) + System.clearProperty("spark.driver.port") + ssc.start() + val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches) // the first element will be re-processed data of the last batch before restart verifyOutput[V](outputNew, expectedOutput.takeRight(nextNumExpectedOutputs), true) + ssc.stop() ssc = null } /** * Advances the manual clock on the streaming scheduler by given number of batches. - * It also wait for the expected amount of time for each batch. + * It also waits for the expected amount of time for each batch. */ - def runStreamsWithRealDelay(ssc: StreamingContext, numBatches: Long) { + def advanceTimeWithRealDelay[V: ClassManifest](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] logInfo("Manual clock before advancing = " + clock.time) for (i <- 1 to numBatches.toInt) { @@ -308,6 +349,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { } logInfo("Manual clock after advancing = " + clock.time) Thread.sleep(batchDuration.milliseconds) - } + val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]] + outputStream.output + } } \ No newline at end of file diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index efaa098d2e193..a5fa7ab92dd6b 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -1,14 +1,15 @@ package spark.streaming -import org.scalatest.{FunSuite, BeforeAndAfter} -import org.apache.commons.io.FileUtils -import java.io.File -import scala.runtime.RichInt -import scala.util.Random -import spark.streaming.StreamingContext._ -import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import spark.Logging +import spark.streaming.util.MasterFailureTest +import StreamingContext._ + +import org.scalatest.{FunSuite, BeforeAndAfter} import com.google.common.io.Files +import java.io.File +import org.apache.commons.io.FileUtils +import collection.mutable.ArrayBuffer + /** * This testsuite tests master failures at random times while the stream is running using @@ -16,295 +17,24 @@ import com.google.common.io.Files */ class FailureSuite extends FunSuite with BeforeAndAfter with Logging { - var testDir: File = null - var checkpointDir: File = null - val batchDuration = Milliseconds(500) + var directory = "FailureSuite" + val numBatches = 30 + val batchDuration = Milliseconds(1000) before { - testDir = Files.createTempDir() - checkpointDir = Files.createTempDir() + FileUtils.deleteDirectory(new File(directory)) } after { - FailureSuite.reset() - FileUtils.deleteDirectory(checkpointDir) - FileUtils.deleteDirectory(testDir) - - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") - } - - test("multiple failures with updateStateByKey") { - val n = 30 - // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ... - val input = (1 to n).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq - // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ... - val expectedOutput = (1 to n).map(i => (1 to i).reduce(_ + _)).map(j => ("a", j)) - - val operation = (st: DStream[String]) => { - val updateFunc = (values: Seq[Int], state: Option[RichInt]) => { - Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0))) - } - st.flatMap(_.split(" ")) - .map(x => (x, 1)) - .updateStateByKey[RichInt](updateFunc) - .checkpoint(Seconds(2)) - .map(t => (t._1, t._2.self)) - } - - testOperationWithMultipleFailures(input, operation, expectedOutput) - } - - test("multiple failures with reduceByKeyAndWindow") { - val n = 30 - val w = 100 - assert(w > n, "Window should be much larger than the number of input sets in this test") - // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ... - val input = (1 to n).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq - // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ... - val expectedOutput = (1 to n).map(i => (1 to i).reduce(_ + _)).map(j => ("a", j)) - - val operation = (st: DStream[String]) => { - st.flatMap(_.split(" ")) - .map(x => (x, 1)) - .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration) - .checkpoint(Seconds(2)) - } - - testOperationWithMultipleFailures(input, operation, expectedOutput) + FileUtils.deleteDirectory(new File(directory)) } - - /** - * Tests stream operation with multiple master failures, and verifies whether the - * final set of output values is as expected or not. Checking the final value is - * proof that no intermediate data was lost due to master failures. - */ - def testOperationWithMultipleFailures( - input: Seq[String], - operation: DStream[String] => DStream[(String, Int)], - expectedOutput: Seq[(String, Int)] - ) { - var ssc = setupStreamsWithFileStream(operation) - - val mergedOutput = new ArrayBuffer[(String, Int)]() - val lastExpectedOutput = expectedOutput.last - - val maxTimeToRun = expectedOutput.size * batchDuration.milliseconds * 2 - var totalTimeRan = 0L - - // Start generating files in the a different thread - val fileGeneratingThread = new FileGeneratingThread(input, testDir.getPath, batchDuration.milliseconds) - fileGeneratingThread.start() - - // Repeatedly start and kill the streaming context until timed out or - // all expected output is generated - while(!FailureSuite.outputGenerated && !FailureSuite.timedOut) { - - // Start the thread to kill the streaming after some time - FailureSuite.failed = false - val killingThread = new KillingThread(ssc, batchDuration.milliseconds * 10) - killingThread.start() - - // Run the streams with real clock until last expected output is seen or timed out - val (output, timeRan) = runStreamsWithRealClock(ssc, lastExpectedOutput, maxTimeToRun - totalTimeRan) - if (killingThread.isAlive) killingThread.interrupt() - - // Merge output and time ran and see whether already timed out or not - mergedOutput ++= output - totalTimeRan += timeRan - logInfo("New output = " + output) - logInfo("Merged output = " + mergedOutput) - logInfo("Total time spent = " + totalTimeRan) - if (totalTimeRan > maxTimeToRun) { - FailureSuite.timedOut = true - } - - if (!FailureSuite.outputGenerated && !FailureSuite.timedOut) { - val sleepTime = Random.nextInt(batchDuration.milliseconds.toInt * 2) - logInfo( - "\n-------------------------------------------\n" + - " Restarting stream computation in " + sleepTime + " ms " + - "\n-------------------------------------------\n" - ) - Thread.sleep(sleepTime) - } - - // Recreate the streaming context from checkpoint - ssc = new StreamingContext(checkpointDir.getPath) - } - ssc.stop() - ssc = null - logInfo("Finished test after " + FailureSuite.failureCount + " failures") - - if (FailureSuite.timedOut) { - logWarning("Timed out with run time of "+ maxTimeToRun + " ms for " + - expectedOutput.size + " batches of " + batchDuration) - } - - // Verify whether the output is as expected - verifyOutput(mergedOutput, expectedOutput) - if (fileGeneratingThread.isAlive) fileGeneratingThread.interrupt() + test("multiple failures with map") { + MasterFailureTest.testMap(directory, numBatches, batchDuration) } - /** Sets up the stream operations with file input stream */ - def setupStreamsWithFileStream( - operation: DStream[String] => DStream[(String, Int)] - ): StreamingContext = { - val ssc = new StreamingContext("local[4]", "FailureSuite", batchDuration) - ssc.checkpoint(checkpointDir.getPath) - val inputStream = ssc.textFileStream(testDir.getPath) - val operatedStream = operation(inputStream) - val outputBuffer = new ArrayBuffer[Seq[(String, Int)]] with SynchronizedBuffer[Seq[(String, Int)]] - val outputStream = new TestOutputStream(operatedStream, outputBuffer) - ssc.registerOutputStream(outputStream) - ssc - } - - /** - * Runs the streams set up in `ssc` on real clock. - */ - def runStreamsWithRealClock( - ssc: StreamingContext, - lastExpectedOutput: (String, Int), - timeout: Long - ): (Seq[(String, Int)], Long) = { - - System.clearProperty("spark.streaming.clock") - - // Get the output buffer - val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[(String, Int)]] - val output = outputStream.output - val startTime = System.currentTimeMillis() - - // Functions to detect various conditions - def hasFailed = FailureSuite.failed - def isLastOutputGenerated = !output.flatMap(x => x).isEmpty && output(output.lastIndexWhere(!_.isEmpty)).head == lastExpectedOutput - def isTimedOut = System.currentTimeMillis() - startTime > timeout - - // Start the streaming computation and let it run while ... - // (i) StreamingContext has not been shut down yet - // (ii) The last expected output has not been generated yet - // (iii) Its not timed out yet - try { - ssc.start() - while (!hasFailed && !isLastOutputGenerated && !isTimedOut) { - Thread.sleep(100) - } - logInfo("Has failed = " + hasFailed) - logInfo("Is last output generated = " + isLastOutputGenerated) - logInfo("Is timed out = " + isTimedOut) - } catch { - case e: Exception => logInfo("Exception while running streams: " + e) - } finally { - ssc.stop() - } - - // Verify whether the output of each batch has only one element - assert(output.forall(_.size <= 1), "output of each batch should have only one element") - - // Set appropriate flags is timed out or output has been generated - if (isTimedOut) FailureSuite.timedOut = true - if (isLastOutputGenerated) FailureSuite.outputGenerated = true - - val timeTaken = System.currentTimeMillis() - startTime - logInfo("" + output.size + " sets of output generated in " + timeTaken + " ms") - (output.flatMap(_.headOption), timeTaken) - } - - /** - * Verifies the output value are the same as expected. Since failures can lead to - * a batch being processed twice, a batches output may appear more than once - * consecutively. To avoid getting confused with those, we eliminate consecutive - * duplicate batch outputs of values from the `output`. As a result, the - * expected output should not have consecutive batches with the same values as output. - */ - def verifyOutput(output: Seq[(String, Int)], expectedOutput: Seq[(String, Int)]) { - // Verify whether expected outputs do not consecutive batches with same output - for (i <- 0 until expectedOutput.size - 1) { - assert(expectedOutput(i) != expectedOutput(i+1), - "Expected output has consecutive duplicate sequence of values") - } - - // Match the output with the expected output - logInfo( - "\n-------------------------------------------\n" + - " Verifying output " + - "\n-------------------------------------------\n" - ) - logInfo("Expected output, size = " + expectedOutput.size) - logInfo(expectedOutput.mkString("[", ",", "]")) - logInfo("Output, size = " + output.size) - logInfo(output.mkString("[", ",", "]")) - output.foreach(o => - assert(expectedOutput.contains(o), "Expected value " + o + " not found") - ) - } -} - -object FailureSuite { - var failed = false - var outputGenerated = false - var timedOut = false - var failureCount = 0 - - def reset() { - failed = false - outputGenerated = false - timedOut = false - failureCount = 0 - } -} - -/** - * Thread to kill streaming context after some time. - */ -class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging { - initLogging() - - override def run() { - try { - var minKillWaitTime = if (FailureSuite.failureCount == 0) 5000 else 1000 // to allow the first checkpoint - val killWaitTime = minKillWaitTime + math.abs(Random.nextLong % maxKillWaitTime) - logInfo("Kill wait time = " + killWaitTime) - Thread.sleep(killWaitTime) - logInfo( - "\n---------------------------------------\n" + - "Killing streaming context after " + killWaitTime + " ms" + - "\n---------------------------------------\n" - ) - if (ssc != null) { - ssc.stop() - FailureSuite.failed = true - FailureSuite.failureCount += 1 - } - logInfo("Killing thread exited") - } catch { - case ie: InterruptedException => logInfo("Killing thread interrupted") - case e: Exception => logWarning("Exception in killing thread", e) - } - } -} - -/** - * Thread to generate input files periodically with the desired text - */ -class FileGeneratingThread(input: Seq[String], testDir: String, interval: Long) - extends Thread with Logging { - initLogging() - - override def run() { - try { - Thread.sleep(5000) // To make sure that all the streaming context has been set up - for (i <- 0 until input.size) { - FileUtils.writeStringToFile(new File(testDir, i.toString), input(i).toString + "\n") - Thread.sleep(interval) - } - logInfo("File generating thread exited") - } catch { - case ie: InterruptedException => logInfo("File generating thread interrupted") - case e: Exception => logWarning("File generating in killing thread", e) - } + test("multiple failures with updateStateByKey") { + MasterFailureTest.testUpdateStateByKey(directory, numBatches, batchDuration) } } diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 0eb9c7b81e29b..7c1c2e1040073 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -133,26 +133,29 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("file input stream") { + // Disable manual clock as FileInputDStream does not work with manual clock + System.clearProperty("spark.streaming.clock") + // Set up the streaming context and input streams val testDir = Files.createTempDir() val ssc = new StreamingContext(master, framework, batchDuration) - val filestream = ssc.textFileStream(testDir.toString) + val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] def output = outputBuffer.flatMap(x => x) - val outputStream = new TestOutputStream(filestream, outputBuffer) + val outputStream = new TestOutputStream(fileStream, outputBuffer) ssc.registerOutputStream(outputStream) ssc.start() // Create files in the temporary directory so that Spark Streaming can read data from it - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val input = Seq(1, 2, 3, 4, 5) val expectedOutput = input.map(_.toString) Thread.sleep(1000) for (i <- 0 until input.size) { - FileUtils.writeStringToFile(new File(testDir, i.toString), input(i).toString + "\n") - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - //Thread.sleep(100) + val file = new File(testDir, i.toString) + FileUtils.writeStringToFile(file, input(i).toString + "\n") + logInfo("Created file " + file) + Thread.sleep(batchDuration.milliseconds) + Thread.sleep(1000) } val startTime = System.currentTimeMillis() Thread.sleep(1000) @@ -171,16 +174,16 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Verify whether all the elements received are as expected // (whether the elements were received one in each interval is not verified) - assert(output.size === expectedOutput.size) - for (i <- 0 until output.size) { - assert(output(i).size === 1) - assert(output(i).head.toString === expectedOutput(i)) - } + assert(output.toList === expectedOutput.toList) + FileUtils.deleteDirectory(testDir) + + // Enable manual clock back again for other tests + System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") } } - +/** This is server to test the network input stream */ class TestServer(port: Int) extends Logging { val queue = new ArrayBlockingQueue[String](100) diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index c2733831b23de..2cc31d61376ad 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -63,20 +63,28 @@ class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBu */ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { + // Name of the framework for Spark context def framework = "TestSuiteBase" + // Master for Spark context def master = "local[2]" + // Batch duration def batchDuration = Seconds(1) + // Directory where the checkpoint data will be saved def checkpointDir = "checkpoint" + // Duration after which the graph is checkpointed def checkpointInterval = batchDuration + // Number of partitions of the input parallel collections created for testing def numInputPartitions = 2 + // Maximum time to wait before the test times out def maxWaitTimeMillis = 10000 + // Whether to actually wait in real time before changing manual clock def actuallyWait = false /** @@ -140,9 +148,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { numBatches: Int, numExpectedOutput: Int ): Seq[Seq[V]] = { - - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") - assert(numBatches > 0, "Number of batches to run stream computation is zero") assert(numExpectedOutput > 0, "Number of expected outputs after " + numBatches + " is zero") logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput) @@ -186,7 +191,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { } finally { ssc.stop() } - output } diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index cd9608df530af..10807901475e4 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -5,6 +5,8 @@ import collection.mutable.ArrayBuffer class WindowOperationsSuite extends TestSuiteBase { + System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + override def framework = "WindowOperationsSuite" override def maxWaitTimeMillis = 20000 From 8f18e7e863728734f927edbcb928a37cdccc4d63 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 Feb 2013 13:05:13 -0800 Subject: [PATCH 214/696] include jobid in Executor commandline args --- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 2 +- .../scala/spark/executor/StandaloneExecutorBackend.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 4ef637090c4eb..69f34e604a6fc 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -77,7 +77,7 @@ private[spark] class ExecutorRunner( val command = jobDesc.command val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run"; val runScript = new File(sparkHome, script).getCanonicalPath - Seq(runScript, command.mainClass) ++ command.arguments.map(substituteVariables) + Seq(runScript, command.mainClass) ++ (command.arguments ++ Seq(jobId)).map(substituteVariables) } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 224c126fdd1ee..06229f39d9601 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -68,8 +68,9 @@ private[spark] object StandaloneExecutorBackend { } def main(args: Array[String]) { - if (args.length != 4) { - System.err.println("Usage: StandaloneExecutorBackend ") + if (!(args.length >= 4)) { + //the reason we allow the last frameworkId argument is to make it easy to kill rogue executors + System.err.println("Usage: StandaloneExecutorBackend []") System.exit(1) } run(args(0), args(1), args(2), args(3).toInt) From 893bad90899c5c7edddc5e824f41a975c33571bf Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 Feb 2013 20:30:21 -0800 Subject: [PATCH 215/696] use appid instead of frameworkid; simplify stupid condition --- .../main/scala/spark/executor/StandaloneExecutorBackend.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 06229f39d9601..9a82c3054c0fc 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -68,9 +68,9 @@ private[spark] object StandaloneExecutorBackend { } def main(args: Array[String]) { - if (!(args.length >= 4)) { + if (args.length < 4) { //the reason we allow the last frameworkId argument is to make it easy to kill rogue executors - System.err.println("Usage: StandaloneExecutorBackend []") + System.err.println("Usage: StandaloneExecutorBackend []") System.exit(1) } run(args(0), args(1), args(2), args(3).toInt) From 12b020b6689b8db94df904d9b897a43bce18c971 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 13 Feb 2013 20:53:50 -0800 Subject: [PATCH 216/696] Added filter functionality to reduceByKeyAndWindow with inverse. Consolidated reduceByKeyAndWindow's many functions into smaller number of functions with optional parameters. --- .../main/scala/spark/streaming/DStream.scala | 2 +- .../streaming/PairDStreamFunctions.scala | 71 ++++++++----------- .../streaming/api/java/JavaPairDStream.scala | 28 +++++--- .../dstream/ReducedWindowedDStream.scala | 30 +++++--- .../streaming/util/MasterFailureTest.scala | 1 - streaming/src/test/resources/log4j.properties | 2 +- .../streaming/WindowOperationsSuite.scala | 49 ++++++++----- 7 files changed, 102 insertions(+), 81 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 0c1b667c0ab42..6abec9e6bec0b 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -531,7 +531,7 @@ abstract class DStream[T: ClassManifest] ( windowDuration: Duration, slideDuration: Duration ): DStream[T] = { - this.window(windowDuration, slideDuration).reduce(reduceFunc) + this.reduce(reduceFunc).window(windowDuration, slideDuration).reduce(reduceFunc) } def reduceByWindow( diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index fbcf06112676d..021ff83b3655f 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -137,7 +137,8 @@ extends Serializable { * @param slideDuration sliding interval of the window (i.e., the interval after which * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval - * @param numPartitions Number of partitions of each RDD in the new DStream. + * @param numPartitions number of partitions of each RDD in the new DStream; if not specified + * then Spark's default number of partitions will be used */ def groupByKeyAndWindow( windowDuration: Duration, @@ -155,7 +156,7 @@ extends Serializable { * @param slideDuration sliding interval of the window (i.e., the interval after which * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval - * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. + * @param partitioner partitioner for controlling the partitioning of each RDD in the new DStream. */ def groupByKeyAndWindow( windowDuration: Duration, @@ -213,7 +214,7 @@ extends Serializable { * @param numPartitions Number of partitions of each RDD in the new DStream. */ def reduceByKeyAndWindow( - reduceFunc: (V, V) => V, + reduceFunc: (V, V) => V, windowDuration: Duration, slideDuration: Duration, numPartitions: Int @@ -230,7 +231,8 @@ extends Serializable { * @param slideDuration sliding interval of the window (i.e., the interval after which * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval - * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. + * @param partitioner partitioner for controlling the partitioning of each RDD + * in the new DStream. */ def reduceByKeyAndWindow( reduceFunc: (V, V) => V, @@ -245,7 +247,7 @@ extends Serializable { } /** - * Create a new DStream by reducing over a using incremental computation. + * Create a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) @@ -253,81 +255,64 @@ extends Serializable { * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative reduce function - * @param invReduceFunc inverse function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - */ - def reduceByKeyAndWindow( - reduceFunc: (V, V) => V, - invReduceFunc: (V, V) => V, - windowDuration: Duration, - slideDuration: Duration - ): DStream[(K, V)] = { - - reduceByKeyAndWindow( - reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner()) - } - - /** - * Create a new DStream by reducing over a using incremental computation. - * The reduced value of over a new window is calculated using the old window's reduce value : - * 1. reduce the new values that entered the window (e.g., adding new counts) - * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function. - * However, it is applicable to only "invertible reduce functions". - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - * @param reduceFunc associative reduce function - * @param invReduceFunc inverse function + * @param invReduceFunc inverse reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval - * @param numPartitions Number of partitions of each RDD in the new DStream. + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained */ def reduceByKeyAndWindow( reduceFunc: (V, V) => V, invReduceFunc: (V, V) => V, windowDuration: Duration, - slideDuration: Duration, - numPartitions: Int + slideDuration: Duration = self.slideDuration, + numPartitions: Int = ssc.sc.defaultParallelism, + filterFunc: ((K, V)) => Boolean = null ): DStream[(K, V)] = { reduceByKeyAndWindow( - reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner(numPartitions)) + reduceFunc, invReduceFunc, windowDuration, + slideDuration, defaultPartitioner(numPartitions), filterFunc + ) } /** - * Create a new DStream by reducing over a using incremental computation. + * Create a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". - * @param reduceFunc associative reduce function - * @param invReduceFunc inverse function + * @param reduceFunc associative reduce function + * @param invReduceFunc inverse reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval - * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. + * @param partitioner partitioner for controlling the partitioning of each RDD in the new DStream. + * @param filterFunc Optional function to filter expired key-value pairs; + * only pairs that satisfy the function are retained */ def reduceByKeyAndWindow( reduceFunc: (V, V) => V, invReduceFunc: (V, V) => V, windowDuration: Duration, slideDuration: Duration, - partitioner: Partitioner + partitioner: Partitioner, + filterFunc: ((K, V)) => Boolean ): DStream[(K, V)] = { val cleanedReduceFunc = ssc.sc.clean(reduceFunc) val cleanedInvReduceFunc = ssc.sc.clean(invReduceFunc) + val cleanedFilterFunc = if (filterFunc != null) Some(ssc.sc.clean(filterFunc)) else None new ReducedWindowedDStream[K, V]( - self, cleanedReduceFunc, cleanedInvReduceFunc, windowDuration, slideDuration, partitioner) + self, cleanedReduceFunc, cleanedInvReduceFunc, cleanedFilterFunc, + windowDuration, slideDuration, partitioner + ) } /** diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index ef10c091caaae..4d3e0d03040e1 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -328,7 +328,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by reducing over a using incremental computation. + * Create a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) @@ -342,25 +342,31 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param slideDuration sliding interval of the window (i.e., the interval after which * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval - * @param numPartitions Number of partitions of each RDD in the new DStream. + * @param numPartitions number of partitions of each RDD in the new DStream. + * @param filterFunc function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + * set this to null if you do not want to filter */ def reduceByKeyAndWindow( reduceFunc: Function2[V, V, V], invReduceFunc: Function2[V, V, V], windowDuration: Duration, slideDuration: Duration, - numPartitions: Int + numPartitions: Int, + filterFunc: JFunction[(K, V), java.lang.Boolean] ): JavaPairDStream[K, V] = { dstream.reduceByKeyAndWindow( reduceFunc, invReduceFunc, windowDuration, slideDuration, - numPartitions) + numPartitions, + (p: (K, V)) => filterFunc(p).booleanValue() + ) } /** - * Create a new DStream by reducing over a using incremental computation. + * Create a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) @@ -374,20 +380,26 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. + * @param filterFunc function to filter expired key-value pairs; + * only pairs that satisfy the function are retained + * set this to null if you do not want to filter */ def reduceByKeyAndWindow( reduceFunc: Function2[V, V, V], invReduceFunc: Function2[V, V, V], windowDuration: Duration, slideDuration: Duration, - partitioner: Partitioner - ): JavaPairDStream[K, V] = { + partitioner: Partitioner, + filterFunc: JFunction[(K, V), java.lang.Boolean] + ): JavaPairDStream[K, V] = { dstream.reduceByKeyAndWindow( reduceFunc, invReduceFunc, windowDuration, slideDuration, - partitioner) + partitioner, + (p: (K, V)) => filterFunc(p).booleanValue() + ) } /** diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala index 733d5c4a25271..aa5a71e1ed700 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -3,7 +3,7 @@ package spark.streaming.dstream import spark.streaming.StreamingContext._ import spark.RDD -import spark.rdd.CoGroupedRDD +import spark.rdd.{CoGroupedRDD, MapPartitionsRDD} import spark.Partitioner import spark.SparkContext._ import spark.storage.StorageLevel @@ -15,7 +15,8 @@ private[streaming] class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( parent: DStream[(K, V)], reduceFunc: (V, V) => V, - invReduceFunc: (V, V) => V, + invReduceFunc: (V, V) => V, + filterFunc: Option[((K, V)) => Boolean], _windowDuration: Duration, _slideDuration: Duration, partitioner: Partitioner @@ -87,22 +88,25 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( // // Get the RDDs of the reduced values in "old time steps" - val oldRDDs = reducedStream.slice(previousWindow.beginTime, currentWindow.beginTime - parent.slideDuration) + val oldRDDs = + reducedStream.slice(previousWindow.beginTime, currentWindow.beginTime - parent.slideDuration) logDebug("# old RDDs = " + oldRDDs.size) // Get the RDDs of the reduced values in "new time steps" - val newRDDs = reducedStream.slice(previousWindow.endTime + parent.slideDuration, currentWindow.endTime) + val newRDDs = + reducedStream.slice(previousWindow.endTime + parent.slideDuration, currentWindow.endTime) logDebug("# new RDDs = " + newRDDs.size) // Get the RDD of the reduced value of the previous window - val previousWindowRDD = getOrCompute(previousWindow.endTime).getOrElse(ssc.sc.makeRDD(Seq[(K,V)]())) + val previousWindowRDD = + getOrCompute(previousWindow.endTime).getOrElse(ssc.sc.makeRDD(Seq[(K,V)]())) // Make the list of RDDs that needs to cogrouped together for reducing their reduced values val allRDDs = new ArrayBuffer[RDD[(K, V)]]() += previousWindowRDD ++= oldRDDs ++= newRDDs // Cogroup the reduced RDDs and merge the reduced values - val cogroupedRDD = new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(_, _)]]], partitioner) - //val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _ + val cogroupedRDD = + new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(_, _)]]], partitioner) val numOldValues = oldRDDs.size val numNewValues = newRDDs.size @@ -114,7 +118,9 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( // Getting reduced values "old time steps" that will be removed from current window val oldValues = (1 to numOldValues).map(i => seqOfValues(i)).filter(!_.isEmpty).map(_.head) // Getting reduced values "new time steps" - val newValues = (1 to numNewValues).map(i => seqOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head) + val newValues = + (1 to numNewValues).map(i => seqOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head) + if (seqOfValues(0).isEmpty) { // If previous window's reduce value does not exist, then at least new values should exist if (newValues.isEmpty) { @@ -140,10 +146,12 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K,Seq[Seq[V]])]].mapValues(mergeValues) - Some(mergedValuesRDD) + if (filterFunc.isDefined) { + Some(mergedValuesRDD.filter(filterFunc.get)) + } else { + Some(mergedValuesRDD) + } } - - } diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index 3ffe4b64d0873..83d8591a3a3dc 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -291,7 +291,6 @@ class TestOutputStream[T: ClassManifest]( (rdd: RDD[T], t: Time) => { val collected = rdd.collect() output += collected - println(t + ": " + collected.mkString("[", ",", "]")) } ) { diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 5652596e1e4b9..f0638e0e027f2 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -1,7 +1,7 @@ # Set everything to be logged to the file streaming/target/unit-tests.log log4j.rootCategory=WARN, file # log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file=org.apache.log4j.ConsoleAppender +log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=streaming/target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index 10807901475e4..e6ac7b35aa9a0 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -84,12 +84,9 @@ class WindowOperationsSuite extends TestSuiteBase { ) /* - The output of the reduceByKeyAndWindow with inverse reduce function is - different from the naive reduceByKeyAndWindow. Even if the count of a - particular key is 0, the key does not get eliminated from the RDDs of - ReducedWindowedDStream. This causes the number of keys in these RDDs to - increase forever. A more generalized version that allows elimination of - keys should be considered. + The output of the reduceByKeyAndWindow with inverse function but without a filter + function will be different from the naive reduceByKeyAndWindow, as no keys get + eliminated from the ReducedWindowedDStream even if the value of a key becomes 0. */ val bigReduceInvOutput = Seq( @@ -177,31 +174,31 @@ class WindowOperationsSuite extends TestSuiteBase { // Testing reduceByKeyAndWindow (with invertible reduce function) - testReduceByKeyAndWindowInv( + testReduceByKeyAndWindowWithInverse( "basic reduction", Seq(Seq(("a", 1), ("a", 3)) ), Seq(Seq(("a", 4)) ) ) - testReduceByKeyAndWindowInv( + testReduceByKeyAndWindowWithInverse( "key already in window and new value added into window", Seq( Seq(("a", 1)), Seq(("a", 1)) ), Seq( Seq(("a", 1)), Seq(("a", 2)) ) ) - testReduceByKeyAndWindowInv( + testReduceByKeyAndWindowWithInverse( "new key added into window", Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 1)) ), Seq( Seq(("a", 1)), Seq(("a", 2), ("b", 1)) ) ) - testReduceByKeyAndWindowInv( + testReduceByKeyAndWindowWithInverse( "key removed from window", Seq( Seq(("a", 1)), Seq(("a", 1)), Seq(), Seq() ), Seq( Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 1)), Seq(("a", 0)) ) ) - testReduceByKeyAndWindowInv( + testReduceByKeyAndWindowWithInverse( "larger slide time", largerSlideInput, largerSlideReduceOutput, @@ -209,7 +206,9 @@ class WindowOperationsSuite extends TestSuiteBase { Seconds(2) ) - testReduceByKeyAndWindowInv("big test", bigInput, bigReduceInvOutput) + testReduceByKeyAndWindowWithInverse("big test", bigInput, bigReduceInvOutput) + + testReduceByKeyAndWindowWithFilteredInverse("big test", bigInput, bigReduceOutput) test("groupByKeyAndWindow") { val input = bigInput @@ -276,27 +275,45 @@ class WindowOperationsSuite extends TestSuiteBase { test("reduceByKeyAndWindow - " + name) { val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt val operation = (s: DStream[(String, Int)]) => { - s.reduceByKeyAndWindow(_ + _, windowDuration, slideDuration).persist() + s.reduceByKeyAndWindow((x: Int, y: Int) => x + y, windowDuration, slideDuration) } testOperation(input, operation, expectedOutput, numBatches, true) } } - def testReduceByKeyAndWindowInv( + def testReduceByKeyAndWindowWithInverse( name: String, input: Seq[Seq[(String, Int)]], expectedOutput: Seq[Seq[(String, Int)]], windowDuration: Duration = Seconds(2), slideDuration: Duration = Seconds(1) ) { - test("reduceByKeyAndWindowInv - " + name) { + test("ReduceByKeyAndWindow with inverse function - " + name) { val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt val operation = (s: DStream[(String, Int)]) => { s.reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration) - .persist() .checkpoint(Seconds(100)) // Large value to avoid effect of RDD checkpointing } testOperation(input, operation, expectedOutput, numBatches, true) } } + + def testReduceByKeyAndWindowWithFilteredInverse( + name: String, + input: Seq[Seq[(String, Int)]], + expectedOutput: Seq[Seq[(String, Int)]], + windowDuration: Duration = Seconds(2), + slideDuration: Duration = Seconds(1) + ) { + test("reduceByKeyAndWindow with inverse and filter functions - " + name) { + val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt + val filterFunc = (p: (String, Int)) => p._2 != 0 + val operation = (s: DStream[(String, Int)]) => { + s.reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration, filterFunc = filterFunc) + .persist() + .checkpoint(Seconds(100)) // Large value to avoid effect of RDD checkpointing + } + testOperation(input, operation, expectedOutput, numBatches, true) + } + } } From 03e8dc6861936a0862fba1ca9f830d5ff507718f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 13 Feb 2013 20:59:29 -0800 Subject: [PATCH 217/696] Changes functions comments to make them more consistent. --- .../streaming/PairDStreamFunctions.scala | 42 ++++++++-------- .../streaming/api/java/JavaPairDStream.scala | 48 +++++++++---------- 2 files changed, 45 insertions(+), 45 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 021ff83b3655f..835b20ae08e3a 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -26,7 +26,7 @@ extends Serializable { } /** - * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ def groupByKey(): DStream[(K, Seq[V])] = { @@ -34,7 +34,7 @@ extends Serializable { } /** - * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = { @@ -42,7 +42,7 @@ extends Serializable { } /** - * Create a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]] + * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]] * is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { @@ -54,7 +54,7 @@ extends Serializable { } /** - * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the associative reduce function. Hash partitioning is used to generate the RDDs * with Spark's default number of partitions. */ @@ -63,7 +63,7 @@ extends Serializable { } /** - * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs * with `numPartitions` partitions. */ @@ -72,7 +72,7 @@ extends Serializable { } /** - * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the * partitioning of each RDD. */ @@ -82,7 +82,7 @@ extends Serializable { } /** - * Combine elements of each key in DStream's RDDs using custom function. This is similar to the + * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more * information. */ @@ -95,7 +95,7 @@ extends Serializable { } /** - * Create a new DStream by counting the number of values of each key in each RDD. Hash + * Return a new DStream by counting the number of values of each key in each RDD. Hash * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions. */ def countByKey(numPartitions: Int = self.ssc.sc.defaultParallelism): DStream[(K, Long)] = { @@ -103,7 +103,7 @@ extends Serializable { } /** - * Creates a new DStream by applying `groupByKey` over a sliding window. This is similar to + * Return a new DStream by applying `groupByKey` over a sliding window. This is similar to * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs * with the same interval as this DStream. Hash partitioning is used to generate the RDDs with * Spark's default number of partitions. @@ -115,7 +115,7 @@ extends Serializable { } /** - * Create a new DStream by applying `groupByKey` over a sliding window. Similar to + * Return a new DStream by applying `groupByKey` over a sliding window. Similar to * `DStream.groupByKey()`, but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. * @param windowDuration width of the window; must be a multiple of this DStream's @@ -129,7 +129,7 @@ extends Serializable { } /** - * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. * Similar to `DStream.groupByKey()`, but applies it over a sliding window. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. * @param windowDuration width of the window; must be a multiple of this DStream's @@ -167,7 +167,7 @@ extends Serializable { } /** - * Create a new DStream by applying `reduceByKey` over a sliding window on `this` DStream. + * Return a new DStream by applying `reduceByKey` over a sliding window on `this` DStream. * Similar to `DStream.reduceByKey()`, but applies it over a sliding window. The new DStream * generates RDDs with the same interval as this DStream. Hash partitioning is used to generate * the RDDs with Spark's default number of partitions. @@ -183,7 +183,7 @@ extends Serializable { } /** - * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to + * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative reduce function @@ -202,7 +202,7 @@ extends Serializable { } /** - * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to + * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. * @param reduceFunc associative reduce function @@ -223,7 +223,7 @@ extends Serializable { } /** - * Create a new DStream by applying `reduceByKey` over a sliding window. Similar to + * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to * `DStream.reduceByKey()`, but applies it over a sliding window. * @param reduceFunc associative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's @@ -247,7 +247,7 @@ extends Serializable { } /** - * Create a new DStream by applying incremental `reduceByKey` over a sliding window. + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) @@ -280,7 +280,7 @@ extends Serializable { } /** - * Create a new DStream by applying incremental `reduceByKey` over a sliding window. + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) @@ -316,7 +316,7 @@ extends Serializable { } /** - * Create a new DStream by counting the number of values for each key over a window. + * Return a new DStream by counting the number of values for each key over a window. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval @@ -341,7 +341,7 @@ extends Serializable { } /** - * Create a new "state" DStream where the state for each key is updated by applying + * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param updateFunc State update function. If `this` function returns None, then @@ -355,7 +355,7 @@ extends Serializable { } /** - * Create a new "state" DStream where the state for each key is updated by applying + * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. * @param updateFunc State update function. If `this` function returns None, then @@ -390,7 +390,7 @@ extends Serializable { } /** - * Create a new "state" DStream where the state for each key is updated by applying + * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. * [[spark.Paxrtitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 4d3e0d03040e1..048e10b69c685 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -25,17 +25,17 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( // Methods common to all DStream's // ======================================================================= - /** Returns a new DStream containing only the elements that satisfy a predicate. */ + /** Return a new DStream containing only the elements that satisfy a predicate. */ def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairDStream[K, V] = dstream.filter((x => f(x).booleanValue())) - /** Persists RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ + /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ def cache(): JavaPairDStream[K, V] = dstream.cache() - /** Persists RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ + /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ def persist(): JavaPairDStream[K, V] = dstream.cache() - /** Persists the RDDs of this DStream with the given storage level */ + /** Persist the RDDs of this DStream with the given storage level */ def persist(storageLevel: StorageLevel): JavaPairDStream[K, V] = dstream.persist(storageLevel) /** Method that generates a RDD for the given Duration */ @@ -67,7 +67,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.window(windowDuration, slideDuration) /** - * Returns a new DStream which computed based on tumbling window on this DStream. + * Return a new DStream which computed based on tumbling window on this DStream. * This is equivalent to window(batchDuration, batchDuration). * @param batchDuration tumbling window duration; must be a multiple of this DStream's interval */ @@ -75,7 +75,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.tumble(batchDuration) /** - * Returns a new DStream by unifying data of another DStream with this DStream. + * Return a new DStream by unifying data of another DStream with this DStream. * @param that Another DStream having the same interval (i.e., slideDuration) as this DStream. */ def union(that: JavaPairDStream[K, V]): JavaPairDStream[K, V] = @@ -86,21 +86,21 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( // ======================================================================= /** - * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ def groupByKey(): JavaPairDStream[K, JList[V]] = dstream.groupByKey().mapValues(seqAsJavaList _) /** - * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to + * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ def groupByKey(numPartitions: Int): JavaPairDStream[K, JList[V]] = dstream.groupByKey(numPartitions).mapValues(seqAsJavaList _) /** - * Creates a new DStream by applying `groupByKey` on each RDD of `this` DStream. + * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. * Therefore, the values for each key in `this` DStream's RDDs are grouped into a * single sequence to generate the RDDs of the new DStream. [[spark.Partitioner]] * is used to control the partitioning of each RDD. @@ -109,7 +109,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.groupByKey(partitioner).mapValues(seqAsJavaList _) /** - * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the associative reduce function. Hash partitioning is used to generate the RDDs * with Spark's default number of partitions. */ @@ -117,7 +117,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.reduceByKey(func) /** - * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs * with `numPartitions` partitions. */ @@ -125,7 +125,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.reduceByKey(func, numPartitions) /** - * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are + * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the * partitioning of each RDD. */ @@ -149,7 +149,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by counting the number of values of each key in each RDD. Hash + * Return a new DStream by counting the number of values of each key in each RDD. Hash * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions. */ def countByKey(numPartitions: Int): JavaPairDStream[K, JLong] = { @@ -158,7 +158,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** - * Create a new DStream by counting the number of values of each key in each RDD. Hash + * Return a new DStream by counting the number of values of each key in each RDD. Hash * partitioning is used to generate the RDDs with the default number of partitions. */ def countByKey(): JavaPairDStream[K, JLong] = { @@ -166,7 +166,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Creates a new DStream by applying `groupByKey` over a sliding window. This is similar to + * Return a new DStream by applying `groupByKey` over a sliding window. This is similar to * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs * with the same interval as this DStream. Hash partitioning is used to generate the RDDs with * Spark's default number of partitions. @@ -178,7 +178,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying `groupByKey` over a sliding window. Similar to + * Return a new DStream by applying `groupByKey` over a sliding window. Similar to * `DStream.groupByKey()`, but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. * @param windowDuration width of the window; must be a multiple of this DStream's @@ -193,7 +193,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. * Similar to `DStream.groupByKey()`, but applies it over a sliding window. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. * @param windowDuration width of the window; must be a multiple of this DStream's @@ -210,7 +210,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream. + * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream. * Similar to `DStream.groupByKey()`, but applies it over a sliding window. * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval @@ -243,7 +243,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to + * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative reduce function @@ -262,7 +262,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to + * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. * @param reduceFunc associative reduce function @@ -283,7 +283,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying `reduceByKey` over a sliding window. Similar to + * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to * `DStream.reduceByKey()`, but applies it over a sliding window. * @param reduceFunc associative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's @@ -303,7 +303,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by reducing over a using incremental computation. + * Return a new DStream by reducing over a using incremental computation. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) @@ -328,7 +328,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying incremental `reduceByKey` over a sliding window. + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) @@ -366,7 +366,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Create a new DStream by applying incremental `reduceByKey` over a sliding window. + * Return a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduce value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) From 2eacf22401f75b956036fb0c32eb38baa16b224e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 14 Feb 2013 12:21:47 -0800 Subject: [PATCH 218/696] Removed countByKeyAndWindow on paired DStreams, and added countByValueAndWindow for all DStreams. Updated both scala and java API and testsuites. --- .../examples/clickstream/PageViewStream.scala | 11 ++- .../main/scala/spark/streaming/DStream.scala | 88 +++++++++++++++---- .../streaming/PairDStreamFunctions.scala | 43 ++------- .../streaming/api/java/JavaDStream.scala | 27 +++--- .../streaming/api/java/JavaDStreamLike.scala | 87 +++++++++++++++++- .../streaming/api/java/JavaPairDStream.scala | 56 +----------- .../java/spark/streaming/JavaAPISuite.java | 79 +++++++---------- .../streaming/BasicOperationsSuite.scala | 21 ++++- .../streaming/WindowOperationsSuite.scala | 8 +- 9 files changed, 231 insertions(+), 189 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index a191321d91058..60f228b8adae4 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -28,16 +28,15 @@ object PageViewStream { // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.networkTextStream(host, port) - .flatMap(_.split("\n")) - .map(PageView.fromString(_)) + .flatMap(_.split("\n")) + .map(PageView.fromString(_)) // Return a count of views per URL seen in each batch - val pageCounts = pageViews.map(view => ((view.url, 1))).countByKey() + val pageCounts = pageViews.map(view => view.url).countByValue() // Return a sliding window of page views per URL in the last ten seconds - val slidingPageCounts = pageViews.map(view => ((view.url, 1))) - .window(Seconds(10), Seconds(2)) - .countByKey() + val slidingPageCounts = pageViews.map(view => view.url) + .countByValueAndWindow(Seconds(10), Seconds(2)) // Return the rate of error pages (a non 200 status) in each zip code over the last 30 seconds diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 6abec9e6bec0b..ce42b742d77e4 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -441,6 +441,15 @@ abstract class DStream[T: ClassManifest] ( */ def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _) + /** + * Return a new DStream in which each RDD contains the counts of each distinct value in + * each RDD of this DStream. Hash partitioning is used to generate + * the RDDs with `numPartitions` partitions (Spark's default number of partitions if + * `numPartitions` not specified). + */ + def countByValue(numPartitions: Int = ssc.sc.defaultParallelism): DStream[(T, Long)] = + this.map(x => (x, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) + /** * Apply a function to each RDD in this DStream. This is an output operator, so * this DStream will be registered as an output stream and therefore materialized. @@ -494,14 +503,16 @@ abstract class DStream[T: ClassManifest] ( } /** - * Return a new DStream which is computed based on windowed batches of this DStream. - * The new DStream generates RDDs with the same interval as this DStream. + * Return a new DStream in which each RDD contains all the elements in seen in a + * sliding window of time over this DStream. The new DStream generates RDDs with + * the same interval as this DStream. * @param windowDuration width of the window; must be a multiple of this DStream's interval. */ def window(windowDuration: Duration): DStream[T] = window(windowDuration, this.slideDuration) /** - * Return a new DStream which is computed based on windowed batches of this DStream. + * Return a new DStream in which each RDD contains all the elements in seen in a + * sliding window of time over this DStream. * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -512,19 +523,15 @@ abstract class DStream[T: ClassManifest] ( new WindowedDStream(this, windowDuration, slideDuration) } - /** - * Return a new DStream which computed based on tumbling window on this DStream. - * This is equivalent to window(batchTime, batchTime). - * @param batchDuration tumbling window duration; must be a multiple of this DStream's - * batching interval - */ - def tumble(batchDuration: Duration): DStream[T] = window(batchDuration, batchDuration) - /** * Return a new DStream in which each RDD has a single element generated by reducing all - * elements in a window over this DStream. windowDuration and slideDuration are as defined - * in the window() operation. This is equivalent to - * window(windowDuration, slideDuration).reduce(reduceFunc) + * elements in a sliding window over this DStream. + * @param reduceFunc associative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval */ def reduceByWindow( reduceFunc: (T, T) => T, @@ -534,6 +541,22 @@ abstract class DStream[T: ClassManifest] ( this.reduce(reduceFunc).window(windowDuration, slideDuration).reduce(reduceFunc) } + /** + * Return a new DStream in which each RDD has a single element generated by reducing all + * elements in a sliding window over this DStream. However, the reduction is done incrementally + * using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * This is more efficient than reduceByWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * @param reduceFunc associative reduce function + * @param invReduceFunc inverse reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + */ def reduceByWindow( reduceFunc: (T, T) => T, invReduceFunc: (T, T) => T, @@ -547,13 +570,46 @@ abstract class DStream[T: ClassManifest] ( /** * Return a new DStream in which each RDD has a single element generated by counting the number - * of elements in a window over this DStream. windowDuration and slideDuration are as defined in the - * window() operation. This is equivalent to window(windowDuration, slideDuration).count() + * of elements in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with + * Spark's default number of partitions. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval */ def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Long] = { this.map(_ => 1L).reduceByWindow(_ + _, _ - _, windowDuration, slideDuration) } + /** + * Return a new DStream in which each RDD contains the count of distinct elements in + * RDDs in a sliding window over this DStream. Hash partitioning is used to generate + * the RDDs with `numPartitions` partitions (Spark's default number of partitions if + * `numPartitions` not specified). + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream. + */ + def countByValueAndWindow( + windowDuration: Duration, + slideDuration: Duration, + numPartitions: Int = ssc.sc.defaultParallelism + ): DStream[(T, Long)] = { + + this.map(x => (x, 1L)).reduceByKeyAndWindow( + (x: Long, y: Long) => x + y, + (x: Long, y: Long) => x - y, + windowDuration, + slideDuration, + numPartitions, + (x: (T, Long)) => x._2 != 0L + ) + } + /** * Return a new DStream by unifying data of another DStream with this DStream. * @param that Another DStream having the same slideDuration as this DStream. diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 835b20ae08e3a..5127db3bbcd65 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -94,14 +94,6 @@ extends Serializable { new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner) } - /** - * Return a new DStream by counting the number of values of each key in each RDD. Hash - * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions. - */ - def countByKey(numPartitions: Int = self.ssc.sc.defaultParallelism): DStream[(K, Long)] = { - self.map(x => (x._1, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) - } - /** * Return a new DStream by applying `groupByKey` over a sliding window. This is similar to * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs @@ -211,7 +203,7 @@ extends Serializable { * @param slideDuration sliding interval of the window (i.e., the interval after which * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval - * @param numPartitions Number of partitions of each RDD in the new DStream. + * @param numPartitions number of partitions of each RDD in the new DStream. */ def reduceByKeyAndWindow( reduceFunc: (V, V) => V, @@ -248,10 +240,10 @@ extends Serializable { /** * Return a new DStream by applying incremental `reduceByKey` over a sliding window. - * The reduced value of over a new window is calculated using the old window's reduce value : + * The reduced value of over a new window is calculated using the old window's reduced value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function. + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative reduce function @@ -281,10 +273,10 @@ extends Serializable { /** * Return a new DStream by applying incremental `reduceByKey` over a sliding window. - * The reduced value of over a new window is calculated using the old window's reduce value : + * The reduced value of over a new window is calculated using the old window's reduced value : * 1. reduce the new values that entered the window (e.g., adding new counts) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function. + * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * @param reduceFunc associative reduce function * @param invReduceFunc inverse reduce function @@ -315,31 +307,6 @@ extends Serializable { ) } - /** - * Return a new DStream by counting the number of values for each key over a window. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param numPartitions Number of partitions of each RDD in the new DStream. - */ - def countByKeyAndWindow( - windowDuration: Duration, - slideDuration: Duration, - numPartitions: Int = self.ssc.sc.defaultParallelism - ): DStream[(K, Long)] = { - - self.map(x => (x._1, 1L)).reduceByKeyAndWindow( - (x: Long, y: Long) => x + y, - (x: Long, y: Long) => x - y, - windowDuration, - slideDuration, - numPartitions - ) - } - /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 2e7466b16c93e..30985b4ebc13b 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -36,7 +36,7 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM def cache(): JavaDStream[T] = dstream.cache() /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ - def persist(): JavaDStream[T] = dstream.cache() + def persist(): JavaDStream[T] = dstream.persist() /** Persist the RDDs of this DStream with the given storage level */ def persist(storageLevel: StorageLevel): JavaDStream[T] = dstream.persist(storageLevel) @@ -50,33 +50,26 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM } /** - * Return a new DStream which is computed based on windowed batches of this DStream. - * The new DStream generates RDDs with the same interval as this DStream. + * Return a new DStream in which each RDD contains all the elements in seen in a + * sliding window of time over this DStream. The new DStream generates RDDs with + * the same interval as this DStream. * @param windowDuration width of the window; must be a multiple of this DStream's interval. - * @return */ def window(windowDuration: Duration): JavaDStream[T] = dstream.window(windowDuration) /** - * Return a new DStream which is computed based on windowed batches of this DStream. - * @param windowDuration duration (i.e., width) of the window; - * must be a multiple of this DStream's interval + * Return a new DStream in which each RDD contains all the elements in seen in a + * sliding window of time over this DStream. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's interval + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval */ def window(windowDuration: Duration, slideDuration: Duration): JavaDStream[T] = dstream.window(windowDuration, slideDuration) - /** - * Return a new DStream which computed based on tumbling window on this DStream. - * This is equivalent to window(batchDuration, batchDuration). - * @param batchDuration tumbling window duration; must be a multiple of this DStream's interval - */ - def tumble(batchDuration: Duration): JavaDStream[T] = - dstream.tumble(batchDuration) - /** * Return a new DStream by unifying data of another DStream with this DStream. * @param that Another DStream having the same interval (i.e., slideDuration) as this DStream. diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index b93cb7865aef0..1c1ba05ff98b5 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -33,6 +33,26 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable */ def count(): JavaDStream[JLong] = dstream.count() + /** + * Return a new DStream in which each RDD contains the counts of each distinct value in + * each RDD of this DStream. Hash partitioning is used to generate the RDDs with + * Spark's default number of partitions. + */ + def countByValue(): JavaPairDStream[T, JLong] = { + JavaPairDStream.scalaToJavaLong(dstream.countByValue()) + } + + /** + * Return a new DStream in which each RDD contains the counts of each distinct value in + * each RDD of this DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + * @param numPartitions number of partitions of each RDD in the new DStream. + */ + def countByValue(numPartitions: Int): JavaPairDStream[T, JLong] = { + JavaPairDStream.scalaToJavaLong(dstream.countByValue(numPartitions)) + } + + /** * Return a new DStream in which each RDD has a single element generated by counting the number * of elements in a window over this DStream. windowDuration and slideDuration are as defined in the @@ -42,6 +62,39 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable dstream.countByWindow(windowDuration, slideDuration) } + /** + * Return a new DStream in which each RDD contains the count of distinct elements in + * RDDs in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with + * Spark's default number of partitions. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + */ + def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration) + : JavaPairDStream[T, JLong] = { + JavaPairDStream.scalaToJavaLong( + dstream.countByValueAndWindow(windowDuration, slideDuration)) + } + + /** + * Return a new DStream in which each RDD contains the count of distinct elements in + * RDDs in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + * @param numPartitions number of partitions of each RDD in the new DStream. + */ + def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) + : JavaPairDStream[T, JLong] = { + JavaPairDStream.scalaToJavaLong( + dstream.countByValueAndWindow(windowDuration, slideDuration, numPartitions)) + } + /** * Return a new DStream in which each RDD is generated by applying glom() to each RDD of * this DStream. Applying glom() to an RDD coalesces all elements within each partition into @@ -114,8 +167,38 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable /** * Return a new DStream in which each RDD has a single element generated by reducing all - * elements in a window over this DStream. windowDuration and slideDuration are as defined in the - * window() operation. This is equivalent to window(windowDuration, slideDuration).reduce(reduceFunc) + * elements in a sliding window over this DStream. + * @param reduceFunc associative reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval + */ + def reduceByWindow( + reduceFunc: (T, T) => T, + windowDuration: Duration, + slideDuration: Duration + ): DStream[T] = { + dstream.reduceByWindow(reduceFunc, windowDuration, slideDuration) + } + + + /** + * Return a new DStream in which each RDD has a single element generated by reducing all + * elements in a sliding window over this DStream. However, the reduction is done incrementally + * using the old window's reduced value : + * 1. reduce the new values that entered the window (e.g., adding new counts) + * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * This is more efficient than reduceByWindow without "inverse reduce" function. + * However, it is applicable to only "invertible reduce functions". + * @param reduceFunc associative reduce function + * @param invReduceFunc inverse reduce function + * @param windowDuration width of the window; must be a multiple of this DStream's + * batching interval + * @param slideDuration sliding interval of the window (i.e., the interval after which + * the new DStream will generate RDDs); must be a multiple of this + * DStream's batching interval */ def reduceByWindow( reduceFunc: JFunction2[T, T, T], diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 048e10b69c685..952ca657bf77b 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -33,7 +33,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cache(): JavaPairDStream[K, V] = dstream.cache() /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ - def persist(): JavaPairDStream[K, V] = dstream.cache() + def persist(): JavaPairDStream[K, V] = dstream.persist() /** Persist the RDDs of this DStream with the given storage level */ def persist(storageLevel: StorageLevel): JavaPairDStream[K, V] = dstream.persist(storageLevel) @@ -66,14 +66,6 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def window(windowDuration: Duration, slideDuration: Duration): JavaPairDStream[K, V] = dstream.window(windowDuration, slideDuration) - /** - * Return a new DStream which computed based on tumbling window on this DStream. - * This is equivalent to window(batchDuration, batchDuration). - * @param batchDuration tumbling window duration; must be a multiple of this DStream's interval - */ - def tumble(batchDuration: Duration): JavaPairDStream[K, V] = - dstream.tumble(batchDuration) - /** * Return a new DStream by unifying data of another DStream with this DStream. * @param that Another DStream having the same interval (i.e., slideDuration) as this DStream. @@ -148,23 +140,6 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner) } - /** - * Return a new DStream by counting the number of values of each key in each RDD. Hash - * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions. - */ - def countByKey(numPartitions: Int): JavaPairDStream[K, JLong] = { - JavaPairDStream.scalaToJavaLong(dstream.countByKey(numPartitions)); - } - - - /** - * Return a new DStream by counting the number of values of each key in each RDD. Hash - * partitioning is used to generate the RDDs with the default number of partitions. - */ - def countByKey(): JavaPairDStream[K, JLong] = { - JavaPairDStream.scalaToJavaLong(dstream.countByKey()); - } - /** * Return a new DStream by applying `groupByKey` over a sliding window. This is similar to * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs @@ -402,35 +377,6 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( ) } - /** - * Create a new DStream by counting the number of values for each key over a window. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - */ - def countByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[K, JLong] = { - JavaPairDStream.scalaToJavaLong(dstream.countByKeyAndWindow(windowDuration, slideDuration)) - } - - /** - * Create a new DStream by counting the number of values for each key over a window. - * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @param numPartitions Number of partitions of each RDD in the new DStream. - */ - def countByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - : JavaPairDStream[K, Long] = { - dstream.countByKeyAndWindow(windowDuration, slideDuration, numPartitions) - } - private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]): (Seq[V], Option[S]) => Option[S] = { val scalaFunc: (Seq[V], Option[S]) => Option[S] = (values, state) => { diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 783a393a8ffef..7bea0b1fc4e73 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -134,29 +134,6 @@ public void testWindowWithSlideDuration() { assertOrderInvariantEquals(expected, result); } - @Test - public void testTumble() { - List> inputData = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6), - Arrays.asList(7,8,9), - Arrays.asList(10,11,12), - Arrays.asList(13,14,15), - Arrays.asList(16,17,18)); - - List> expected = Arrays.asList( - Arrays.asList(1,2,3,4,5,6), - Arrays.asList(7,8,9,10,11,12), - Arrays.asList(13,14,15,16,17,18)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream windowed = stream.tumble(new Duration(2000)); - JavaTestUtils.attachTestOutputStream(windowed); - List> result = JavaTestUtils.runStreams(ssc, 6, 3); - - assertOrderInvariantEquals(expected, result); - } - @Test public void testFilter() { List> inputData = Arrays.asList( @@ -584,24 +561,26 @@ public Integer call(Integer i) throws Exception { } @Test - public void testCountByKey() { - List>> inputData = stringStringKVStream; + public void testCountByValue() { + List> inputData = Arrays.asList( + Arrays.asList("hello", "world"), + Arrays.asList("hello", "moon"), + Arrays.asList("hello")); List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2("california", 2L), - new Tuple2("new york", 2L)), - Arrays.asList( - new Tuple2("california", 2L), - new Tuple2("new york", 2L))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + Arrays.asList( + new Tuple2("hello", 1L), + new Tuple2("world", 1L)), + Arrays.asList( + new Tuple2("hello", 1L), + new Tuple2("moon", 1L)), + Arrays.asList( + new Tuple2("hello", 1L))); - JavaPairDStream counted = pairStream.countByKey(); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream counted = stream.countByValue(); JavaTestUtils.attachTestOutputStream(counted); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); Assert.assertEquals(expected, result); } @@ -712,26 +691,28 @@ public void testReduceByKeyAndWindowWithInverse() { } @Test - public void testCountByKeyAndWindow() { - List>> inputData = stringStringKVStream; + public void testCountByValueAndWindow() { + List> inputData = Arrays.asList( + Arrays.asList("hello", "world"), + Arrays.asList("hello", "moon"), + Arrays.asList("hello")); List>> expected = Arrays.asList( Arrays.asList( - new Tuple2("california", 2L), - new Tuple2("new york", 2L)), + new Tuple2("hello", 1L), + new Tuple2("world", 1L)), Arrays.asList( - new Tuple2("california", 4L), - new Tuple2("new york", 4L)), + new Tuple2("hello", 2L), + new Tuple2("world", 1L), + new Tuple2("moon", 1L)), Arrays.asList( - new Tuple2("california", 2L), - new Tuple2("new york", 2L))); + new Tuple2("hello", 2L), + new Tuple2("moon", 1L))); - JavaDStream> stream = JavaTestUtils.attachTestInputStream( + JavaDStream stream = JavaTestUtils.attachTestInputStream( ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream counted = - pairStream.countByKeyAndWindow(new Duration(2000), new Duration(1000)); + stream.countByValueAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(counted); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 12388b888737b..1e86cf49bb75a 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -24,7 +24,7 @@ class BasicOperationsSuite extends TestSuiteBase { ) } - test("flatmap") { + test("flatMap") { val input = Seq(1 to 4, 5 to 8, 9 to 12) testOperation( input, @@ -88,6 +88,23 @@ class BasicOperationsSuite extends TestSuiteBase { ) } + test("count") { + testOperation( + Seq(1 to 1, 1 to 2, 1 to 3, 1 to 4), + (s: DStream[Int]) => s.count(), + Seq(Seq(1L), Seq(2L), Seq(3L), Seq(4L)) + ) + } + + test("countByValue") { + testOperation( + Seq(1 to 1, Seq(1, 1, 1), 1 to 2, Seq(1, 1, 2, 2)), + (s: DStream[Int]) => s.countByValue(), + Seq(Seq((1, 1L)), Seq((1, 3L)), Seq((1, 1L), (2, 1L)), Seq((2, 2L), (1, 2L))), + true + ) + } + test("mapValues") { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), @@ -206,7 +223,7 @@ class BasicOperationsSuite extends TestSuiteBase { case _ => Option(stateObj) } } - s.map(_ -> 1).updateStateByKey[StateObject](updateFunc).mapValues(_.counter) + s.map(x => (x, 1)).updateStateByKey[StateObject](updateFunc).mapValues(_.counter) } testOperation(inputData, updateStateOperation, outputData, true) diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index e6ac7b35aa9a0..f8380af331301 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -236,14 +236,14 @@ class WindowOperationsSuite extends TestSuiteBase { testOperation(input, operation, expectedOutput, numBatches, true) } - test("countByKeyAndWindow") { - val input = Seq(Seq(("a", 1)), Seq(("b", 1), ("b", 2)), Seq(("a", 10), ("b", 20))) + test("countByValueAndWindow") { + val input = Seq(Seq("a"), Seq("b", "b"), Seq("a", "b")) val expectedOutput = Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 2)), Seq(("a", 1), ("b", 3))) val windowDuration = Seconds(2) val slideDuration = Seconds(1) val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt - val operation = (s: DStream[(String, Int)]) => { - s.countByKeyAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toInt)) + val operation = (s: DStream[String]) => { + s.countByValueAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toInt)) } testOperation(input, operation, expectedOutput, numBatches, true) } From def8126d7788a8bd991ac6f9f9403de701a39dc5 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 14 Feb 2013 17:49:43 -0800 Subject: [PATCH 219/696] Added TwitterInputDStream from example to StreamingContext. Renamed example TwitterBasic to TwitterPopularTags. --- ...erBasic.scala => TwitterPopularTags.scala} | 33 +++++------- project/SparkBuild.scala | 8 ++- .../spark/streaming/StreamingContext.scala | 52 +++++++++++++------ .../dstream}/TwitterInputDStream.scala | 5 +- 4 files changed, 53 insertions(+), 45 deletions(-) rename examples/src/main/scala/spark/streaming/examples/{twitter/TwitterBasic.scala => TwitterPopularTags.scala} (55%) rename {examples/src/main/scala/spark/streaming/examples/twitter => streaming/src/main/scala/spark/streaming/dstream}/TwitterInputDStream.scala (94%) diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala similarity index 55% rename from examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala rename to examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 377bc0c98ec1f..fdb3a4c73c6cc 100644 --- a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -1,19 +1,19 @@ -package spark.streaming.examples.twitter +package spark.streaming.examples -import spark.streaming.StreamingContext._ import spark.streaming.{Seconds, StreamingContext} +import StreamingContext._ import spark.SparkContext._ -import spark.storage.StorageLevel /** * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter * stream. The stream is instantiated with credentials and optionally filters supplied by the * command line arguments. + * */ -object TwitterBasic { +object TwitterPopularTags { def main(args: Array[String]) { if (args.length < 3) { - System.err.println("Usage: TwitterBasic " + + System.err.println("Usage: TwitterPopularTags " + " [filter1] [filter2] ... [filter n]") System.exit(1) } @@ -21,10 +21,8 @@ object TwitterBasic { val Array(master, username, password) = args.slice(0, 3) val filters = args.slice(3, args.length) - val ssc = new StreamingContext(master, "TwitterBasic", Seconds(2)) - val stream = new TwitterInputDStream(ssc, username, password, filters, - StorageLevel.MEMORY_ONLY_SER) - ssc.registerInputStream(stream) + val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2)) + val stream = ssc.twitterStream(username, password, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) @@ -39,22 +37,17 @@ object TwitterBasic { // Print popular hashtags topCounts60.foreach(rdd => { - if (rdd.count() != 0) { - val topList = rdd.take(5) - println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count())) - topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} - } + val topList = rdd.take(5) + println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count())) + topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} }) topCounts10.foreach(rdd => { - if (rdd.count() != 0) { - val topList = rdd.take(5) - println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count())) - topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} - } + val topList = rdd.take(5) + println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count())) + topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} }) ssc.start() } - } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index af8b5ba01745b..c6d3cc8b1540f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -154,10 +154,7 @@ object SparkBuild extends Build { ) def examplesSettings = sharedSettings ++ Seq( - name := "spark-examples", - libraryDependencies ++= Seq( - "org.twitter4j" % "twitter4j-stream" % "3.0.3" - ) + name := "spark-examples" ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") @@ -166,7 +163,8 @@ object SparkBuild extends Build { name := "spark-streaming", libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", - "com.github.sgroschupf" % "zkclient" % "0.1" + "com.github.sgroschupf" % "zkclient" % "0.1", + "org.twitter4j" % "twitter4j-stream" % "3.0.3" ) ) ++ assemblySettings ++ extraAssemblySettings diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 8cfbec51d26ac..9be9d884be66c 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -17,6 +17,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import java.util.UUID +import twitter4j.Status /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -30,14 +31,14 @@ class StreamingContext private ( ) extends Logging { /** - * Creates a StreamingContext using an existing SparkContext. + * Create a StreamingContext using an existing SparkContext. * @param sparkContext Existing SparkContext * @param batchDuration The time interval at which streaming data will be divided into batches */ def this(sparkContext: SparkContext, batchDuration: Duration) = this(sparkContext, null, batchDuration) /** - * Creates a StreamingContext by providing the details necessary for creating a new SparkContext. + * Create a StreamingContext by providing the details necessary for creating a new SparkContext. * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param frameworkName A name for your job, to display on the cluster web UI * @param batchDuration The time interval at which streaming data will be divided into batches @@ -46,7 +47,7 @@ class StreamingContext private ( this(StreamingContext.createNewSparkContext(master, frameworkName), null, batchDuration) /** - * Re-creates a StreamingContext from a checkpoint file. + * Re-create a StreamingContext from a checkpoint file. * @param path Path either to the directory that was specified as the checkpoint directory, or * to the checkpoint file 'graph' or 'graph.bk'. */ @@ -101,12 +102,12 @@ class StreamingContext private ( protected[streaming] var scheduler: Scheduler = null /** - * Returns the associated Spark context + * Return the associated Spark context */ def sparkContext = sc /** - * Sets each DStreams in this context to remember RDDs it generated in the last given duration. + * Set each DStreams in this context to remember RDDs it generated in the last given duration. * DStreams remember RDDs only for a limited duration of time and releases them for garbage * collection. This method allows the developer to specify how to long to remember the RDDs ( * if the developer wishes to query old data outside the DStream computation). @@ -117,7 +118,7 @@ class StreamingContext private ( } /** - * Sets the context to periodically checkpoint the DStream operations for master + * Set the context to periodically checkpoint the DStream operations for master * fault-tolerance. By default, the graph will be checkpointed every batch interval. * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored * @param interval checkpoint interval @@ -200,7 +201,7 @@ class StreamingContext private ( } /** - * Creates a input stream from a Flume source. + * Create a input stream from a Flume source. * @param hostname Hostname of the slave machine to which the flume data will be sent * @param port Port of the slave machine to which the flume data will be sent * @param storageLevel Storage level to use for storing the received objects @@ -236,7 +237,7 @@ class StreamingContext private ( } /** - * Creates a input stream that monitors a Hadoop-compatible filesystem + * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * File names starting with . are ignored. * @param directory HDFS directory to monitor for new file @@ -255,7 +256,7 @@ class StreamingContext private ( } /** - * Creates a input stream that monitors a Hadoop-compatible filesystem + * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * @param directory HDFS directory to monitor for new file * @param filter Function to filter paths to process @@ -274,9 +275,8 @@ class StreamingContext private ( inputStream } - /** - * Creates a input stream that monitors a Hadoop-compatible filesystem + * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them as text files (using key as LongWritable, value * as Text and input format as TextInputFormat). File names starting with . are ignored. * @param directory HDFS directory to monitor for new file @@ -286,7 +286,25 @@ class StreamingContext private ( } /** - * Creates an input stream from a queue of RDDs. In each batch, + * Create a input stream that returns tweets received from Twitter. + * @param username Twitter username + * @param password Twitter password + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + username: String, + password: String, + filters: Seq[String], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = { + val inputStream = new TwitterInputDStream(this, username, password, filters, storageLevel) + registerInputStream(inputStream) + inputStream + } + + /** + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * @param queue Queue of RDDs * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval @@ -300,7 +318,7 @@ class StreamingContext private ( } /** - * Creates an input stream from a queue of RDDs. In each batch, + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * @param queue Queue of RDDs * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval @@ -325,7 +343,7 @@ class StreamingContext private ( } /** - * Registers an input stream that will be started (InputDStream.start() called) to get the + * Register an input stream that will be started (InputDStream.start() called) to get the * input data. */ def registerInputStream(inputStream: InputDStream[_]) { @@ -333,7 +351,7 @@ class StreamingContext private ( } /** - * Registers an output stream that will be computed every interval + * Register an output stream that will be computed every interval */ def registerOutputStream(outputStream: DStream[_]) { graph.addOutputStream(outputStream) @@ -351,7 +369,7 @@ class StreamingContext private ( } /** - * Starts the execution of the streams. + * Start the execution of the streams. */ def start() { if (checkpointDir != null && checkpointDuration == null && graph != null) { @@ -379,7 +397,7 @@ class StreamingContext private ( } /** - * Stops the execution of the streams. + * Stop the execution of the streams. */ def stop() { try { diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala similarity index 94% rename from examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala rename to streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index 99ed4cdc1c12d..d733254ddbb91 100644 --- a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -1,4 +1,4 @@ -package spark.streaming.examples.twitter +package spark.streaming.dstream import spark._ import spark.streaming._ @@ -6,7 +6,6 @@ import dstream.{NetworkReceiver, NetworkInputDStream} import storage.StorageLevel import twitter4j._ import twitter4j.auth.BasicAuthorization -import collection.JavaConversions._ /* A stream of Twitter statuses, potentially filtered by one or more keywords. * @@ -50,7 +49,7 @@ class TwitterReceiver( def onTrackLimitationNotice(i: Int) {} def onScrubGeo(l: Long, l1: Long) {} def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) {} + def onException(e: Exception) { stopOnError(e) } }) val query: FilterQuery = new FilterQuery From 4b8402e900c803e64b8a4e2094fd845ccfc9df36 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 14 Feb 2013 18:10:37 -0800 Subject: [PATCH 220/696] Moved Java streaming examples to examples/src/main/java/spark/streaming/... and fixed logging in NetworkInputTracker to highlight errors when receiver deregisters/shuts down. --- .../spark/streaming/examples/JavaFlumeEventCount.java | 0 .../spark/streaming/examples/JavaNetworkWordCount.java | 0 .../spark/streaming/examples/JavaQueueStream.java | 0 .../src/main/scala/spark/streaming/NetworkInputTracker.scala | 2 +- 4 files changed, 1 insertion(+), 1 deletion(-) rename examples/src/main/{scala => java}/spark/streaming/examples/JavaFlumeEventCount.java (100%) rename examples/src/main/{scala => java}/spark/streaming/examples/JavaNetworkWordCount.java (100%) rename examples/src/main/{scala => java}/spark/streaming/examples/JavaQueueStream.java (100%) diff --git a/examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java similarity index 100% rename from examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java rename to examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java diff --git a/examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java similarity index 100% rename from examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java rename to examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java diff --git a/examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java similarity index 100% rename from examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java rename to examples/src/main/java/spark/streaming/examples/JavaQueueStream.java diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index b54f53b2033b8..ca5f11fdbad2c 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -86,7 +86,7 @@ class NetworkInputTracker( } case DeregisterReceiver(streamId, msg) => { receiverInfo -= streamId - logInfo("De-registered receiver for network stream " + streamId + logError("De-registered receiver for network stream " + streamId + " with message " + msg) //TODO: Do something about the corresponding NetworkInputDStream } From ddcb976b0d7ce4a76168da33c0e947a5a6b5a255 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 15 Feb 2013 06:54:47 +0000 Subject: [PATCH 221/696] Made MasterFailureTest more robust. --- .../streaming/util/MasterFailureTest.scala | 26 ++++++++++++++++--- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index 83d8591a3a3dc..776e676063c1c 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -40,6 +40,8 @@ object MasterFailureTest extends Logging { println("\n\n================= UPDATE-STATE-BY-KEY TEST =================\n\n") testUpdateStateByKey(directory, numBatches, batchDuration) + + println("\n\nSUCCESS\n\n") } def testMap(directory: String, numBatches: Int, batchDuration: Duration) { @@ -347,7 +349,8 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) override def run() { val localTestDir = Files.createTempDir() - val fs = testDir.getFileSystem(new Configuration()) + var fs = testDir.getFileSystem(new Configuration()) + val maxTries = 3 try { Thread.sleep(5000) // To make sure that all the streaming context has been set up for (i <- 0 until input.size) { @@ -355,9 +358,24 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) val localFile = new File(localTestDir, (i+1).toString) val hadoopFile = new Path(testDir, (i+1).toString) FileUtils.writeStringToFile(localFile, input(i).toString + "\n") - //fs.moveFromLocalFile(new Path(localFile.toString), new Path(testDir, i.toString)) - fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) - logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis) + var tries = 0 + var done = false + while (!done && tries < maxTries) { + tries += 1 + try { + fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) + done = true + } catch { + case ioe: IOException => { + fs = testDir.getFileSystem(new Configuration()) + logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe) + } + } + } + if (!done) + logError("Could not generate file " + hadoopFile) + else + logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis) Thread.sleep(interval) localFile.delete() } From 6cd68c31cbebb1e3c6b35026f067a3c82ce9fdfb Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 00:29:11 -0600 Subject: [PATCH 222/696] Update default.parallelism docs, have StandaloneSchedulerBackend use it. Only brand new RDDs (e.g. parallelize and makeRDD) now use default parallelism, everything else uses their largest parent's partitioner or partition size. --- core/FileServerSuite.txt | 1 + .../main/scala/spark/PairRDDFunctions.scala | 16 +-------------- core/src/main/scala/spark/Partitioner.scala | 19 ++++++++++++++++++ core/src/main/scala/spark/RDD.scala | 20 +++++++++++++------ core/src/main/scala/spark/SparkContext.scala | 2 +- .../scala/spark/api/java/JavaPairRDD.scala | 2 +- .../cluster/StandaloneSchedulerBackend.scala | 3 ++- docs/tuning.md | 8 ++++---- 8 files changed, 43 insertions(+), 28 deletions(-) create mode 120000 core/FileServerSuite.txt diff --git a/core/FileServerSuite.txt b/core/FileServerSuite.txt new file mode 120000 index 0000000000000..0a21b7bf2537a --- /dev/null +++ b/core/FileServerSuite.txt @@ -0,0 +1 @@ +/tmp/1359046053333-0/test/FileServerSuite.txt \ No newline at end of file diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 18b4a1eca486c..d840118b82614 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -23,6 +23,7 @@ import spark.partial.BoundedDouble import spark.partial.PartialResult import spark.rdd._ import spark.SparkContext._ +import spark.Partitioner._ /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -436,21 +437,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } - /** - * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. If any of - * the RDDs already has a partitioner, choose that one, otherwise use a default HashPartitioner. - * - * The number of partitions will be the same as the number of partitions in the largest upstream - * RDD, as this should be least likely to cause out-of-memory errors. - */ - def defaultPartitioner(rdds: RDD[_]*): Partitioner = { - val bySize = rdds.sortBy(_.splits.size).reverse - for (r <- bySize if r.partitioner != None) { - return r.partitioner.get - } - return new HashPartitioner(bySize.head.splits.size) - } - /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the * RDD has a known partitioner by only searching the partition that the key maps to. diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 9d5b966e1e5a0..69f9534d23f1d 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -9,6 +9,25 @@ abstract class Partitioner extends Serializable { def getPartition(key: Any): Int } +object Partitioner { + /** + * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. If any of + * the RDDs already has a partitioner, choose that one, otherwise use a default HashPartitioner. + * + * The number of partitions will be the same as the number of partitions in the largest upstream + * RDD, as this should be least likely to cause out-of-memory errors. + * + * We use two method parameters (rdd, others) to enforce callers passing at least 1 RDD. + */ + def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = { + val bySize = (Seq(rdd) ++ others).sortBy(_.splits.size).reverse + for (r <- bySize if r.partitioner != None) { + return r.partitioner.get + } + return new HashPartitioner(bySize.head.splits.size) + } +} + /** * A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`. * diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 6abb5c4792cc2..b3188956d976a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -16,6 +16,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import spark.Partitioner._ import spark.partial.BoundedDouble import spark.partial.CountEvaluator import spark.partial.GroupedCountEvaluator @@ -299,19 +300,26 @@ abstract class RDD[T: ClassManifest]( */ def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) + /** + * Return an RDD of grouped items. + */ + def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = + groupBy[K](f, defaultPartitioner(this)) + /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassManifest](f: T => K, numSplits: Int): RDD[(K, Seq[T])] = { - val cleanF = sc.clean(f) - this.map(t => (cleanF(t), t)).groupByKey(numSplits) - } - + def groupBy[K: ClassManifest](f: T => K, numSplits: Int): RDD[(K, Seq[T])] = + groupBy(f, new HashPartitioner(numSplits)) + /** * Return an RDD of grouped items. */ - def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = groupBy[K](f, sc.defaultParallelism) + def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + val cleanF = sc.clean(f) + this.map(t => (cleanF(t), t)).groupByKey(p) + } /** * Return an RDD created by piping elements to a forked external process. diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0efc00d5dd338..ff367eafb455f 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -693,7 +693,7 @@ class SparkContext( checkpointDir = Some(dir) } - /** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */ + /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ def defaultParallelism: Int = taskScheduler.defaultParallelism /** Default min number of splits for Hadoop RDDs when not given by user */ diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 8a123bdb47297..4fba8b858c0bb 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -237,7 +237,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * "combiner" in MapReduce. Output will be hash-partitioned with the default parallelism level. */ def reduceByKey(func: JFunction2[V, V, V]): JavaPairRDD[K, V] = { - val partitioner = rdd.defaultPartitioner(rdd) + val partitioner = Partitioner.defaultPartitioner(rdd) fromRDD(reduceByKey(partitioner, func)) } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 082022be1c9da..537d9c2e41933 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -147,7 +147,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor driverActor ! ReviveOffers } - override def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2) + override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism")) + .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2)) } private[spark] object StandaloneSchedulerBackend { diff --git a/docs/tuning.md b/docs/tuning.md index 9aaa53cd65205..e9b4d6717c766 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -213,10 +213,10 @@ but at a high level, managing how frequently full GC takes place can help in red Clusters will not be fully utilized unless you set the level of parallelism for each operation high enough. Spark automatically sets the number of "map" tasks to run on each file according to its size -(though you can control it through optional parameters to `SparkContext.textFile`, etc), but for -distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses a default value of 8. -You can pass the level of parallelism as a second argument (see the -[`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation), +(though you can control it through optional parameters to `SparkContext.textFile`, etc), and for +distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest +parent RDD's number of partitions. You can pass the level of parallelism as a second argument +(see the [`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. From 37397106ce64106b1d0dedd6ab428649e08a56cd Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 00:31:07 -0600 Subject: [PATCH 223/696] Remove fileServerSuite.txt. --- core/FileServerSuite.txt | 1 - 1 file changed, 1 deletion(-) delete mode 120000 core/FileServerSuite.txt diff --git a/core/FileServerSuite.txt b/core/FileServerSuite.txt deleted file mode 120000 index 0a21b7bf2537a..0000000000000 --- a/core/FileServerSuite.txt +++ /dev/null @@ -1 +0,0 @@ -/tmp/1359046053333-0/test/FileServerSuite.txt \ No newline at end of file From 6a2d95784366c8e7e54140ae6482e14acbd4d759 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 00:33:49 -0600 Subject: [PATCH 224/696] Tweak test names. --- core/src/test/scala/spark/ShuffleSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index ab7060a1ac2df..0cff5b7ff71ba 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -212,7 +212,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(rdd.values.collect().toList === List("a", "b")) } - test("default partition size uses split size") { + test("default partitioner uses split size") { sc = new SparkContext("local", "test") // specify 2000 splits val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) @@ -223,7 +223,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(c.splits.size === 2000) } - test("default partition uses largest partitioner") { + test("default partitioner uses largest partitioner") { sc = new SparkContext("local", "test") val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) From 858784459f27da6b969022339dcda4cb9970de1b Mon Sep 17 00:00:00 2001 From: "haitao.yao" Date: Sat, 16 Feb 2013 14:42:06 +0800 Subject: [PATCH 225/696] support customized java options for master, worker, executor, repl shell --- run | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/run b/run index 82b1da005a447..ed5fe3ba38175 100755 --- a/run +++ b/run @@ -25,6 +25,26 @@ if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default fi + +# Add java opts for master, worker, executor. The opts maybe null +case "$1" in + 'spark.deploy.master.Master') + SPARK_JAVA_OPTS+=" $SPARK_MASTER_OPTS" + ;; + 'spark.deploy.worker.Worker') + SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS" + ;; + 'spark.executor.StandaloneExecutorBackend') + SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + ;; + 'spark.executor.MesosExecutorBackend') + SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + ;; + 'spark.repl.Main') + SPARK_JAVA_OPTS+=" $SPARK_REPL_OPTS" + ;; +esac + if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then if [ `command -v scala` ]; then RUNNER="scala" From 4281e579c236d0125f44f5ca1d999adb5f894c24 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 00:45:03 -0600 Subject: [PATCH 226/696] Update more javadocs. --- .../main/scala/spark/PairRDDFunctions.scala | 13 +++++++------ .../scala/spark/api/java/JavaPairRDD.scala | 19 ++++++++++--------- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index d840118b82614..4c4ab60f430c4 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -247,8 +247,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } /** - * Simplified version of combineByKey that hash-partitions the resulting RDD using the default - * parallelism level. + * Simplified version of combineByKey that hash-partitions the resulting RDD using the + * existing partitioner/parallelism level. */ def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { @@ -258,7 +258,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the default parallelism level. + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. */ def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { reduceByKey(defaultPartitioner(self), func) @@ -266,7 +267,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with the default parallelism level. + * resulting RDD with the existing partitioner/parallelism level. */ def groupByKey(): RDD[(K, Seq[V])] = { groupByKey(defaultPartitioner(self)) @@ -294,7 +295,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the default level of parallelism. + * using the existing partitioner/parallelism level. */ def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, defaultPartitioner(self, other)) @@ -314,7 +315,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the default parallelism level. + * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, defaultPartitioner(self, other)) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 4fba8b858c0bb..cdfb8930fddfd 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -19,6 +19,7 @@ import spark.OrderedRDDFunctions import spark.storage.StorageLevel import spark.HashPartitioner import spark.Partitioner +import spark.Partitioner._ import spark.RDD import spark.SparkContext.rddToPairRDDFunctions @@ -220,30 +221,30 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif fromRDD(rdd.rightOuterJoin(other, partitioner)) /** - * Simplified version of combineByKey that hash-partitions the resulting RDD using the default - * parallelism level. + * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing + * partitioner/parallelism level. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = { implicit val cm: ClassManifest[C] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]] - fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners)) + fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd))) } /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the default parallelism level. + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. */ def reduceByKey(func: JFunction2[V, V, V]): JavaPairRDD[K, V] = { - val partitioner = Partitioner.defaultPartitioner(rdd) - fromRDD(reduceByKey(partitioner, func)) + fromRDD(reduceByKey(defaultPartitioner(rdd), func)) } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with the default parallelism level. + * resulting RDD with the existing partitioner/parallelism level. */ def groupByKey(): JavaPairRDD[K, JList[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) @@ -268,7 +269,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the default level of parallelism. + * using the existing partitioner/parallelism level. */ def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Option[W])] = fromRDD(rdd.leftOuterJoin(other)) @@ -286,7 +287,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the default parallelism level. + * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Option[V], W)] = fromRDD(rdd.rightOuterJoin(other)) From c34b8ad2c59697b3e1f5034074e5de0d3b32b8f9 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 00:54:03 -0600 Subject: [PATCH 227/696] Avoid a shuffle if combineByKey is passed the same partitioner. --- core/src/main/scala/spark/PairRDDFunctions.scala | 4 +++- core/src/test/scala/spark/ShuffleSuite.scala | 13 +++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index cc3cca2571359..4c4151933086f 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -62,7 +62,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) - if (mapSideCombine) { + if (Option(partitioner) == self.partitioner) { + self.mapPartitions(aggregator.combineValuesByKey(_), true) + } else if (mapSideCombine) { val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true) val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner) partitioned.mapPartitions(aggregator.combineCombinersByKey(_), true) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 3493b9511f6c2..d6efa3db43da3 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -98,6 +98,19 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val sums = pairs.reduceByKey(_+_, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } + + test("reduceByKey with partitioner") { + sc = new SparkContext("local", "test") + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = rddToPairRDDFunctions(sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1)))).partitionBy(p) + val sums = pairs.reduceByKey(p, _+_) + println(sums.toDebugString) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + } test("join") { sc = new SparkContext("local", "test") From 43288732942a29e7c7c42de66eec6246ea27a13b Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 01:16:40 -0600 Subject: [PATCH 228/696] Add assertion about dependencies. --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 +- core/src/test/scala/spark/ShuffleSuite.scala | 16 +++++++++++++--- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 4c4151933086f..112beb2320a06 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -62,7 +62,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) - if (Option(partitioner) == self.partitioner) { + if (self.partitioner == Some(partitioner)) { self.mapPartitions(aggregator.combineValuesByKey(_), true) } else if (mapSideCombine) { val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index d6efa3db43da3..50f2b294bfb05 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -1,6 +1,7 @@ package spark import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers @@ -105,11 +106,20 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { def numPartitions = 2 def getPartition(key: Any) = key.asInstanceOf[Int] } - val pairs = rddToPairRDDFunctions(sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1)))).partitionBy(p) - val sums = pairs.reduceByKey(p, _+_) - println(sums.toDebugString) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) + val sums = pairs.reduceByKey(_+_) assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = new HashSet[RDD[_]]() + def visit(r: RDD[_]) { + for (dep <- r.dependencies) { + deps += dep.rdd + visit(dep.rdd) + } + } + visit(sums) + assert(deps.size === 2) // ShuffledRDD, ParallelCollection } test("join") { From ae2234687d9040b42619c374eadfd40c896d386d Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 13:10:31 -0600 Subject: [PATCH 229/696] Make CoGroupedRDDs explicitly have the same key type. --- core/src/main/scala/spark/PairRDDFunctions.scala | 8 ++++---- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 4 ++-- core/src/test/scala/spark/CheckpointSuite.scala | 2 +- .../main/scala/spark/streaming/PairDStreamFunctions.scala | 2 +- .../scala/spark/streaming/dstream/CoGroupedDStream.scala | 2 +- .../spark/streaming/dstream/ReducedWindowedDStream.scala | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index cc3cca2571359..36b9880cd1856 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -361,7 +361,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K]( - Seq(self.asInstanceOf[RDD[(_, _)]], other.asInstanceOf[RDD[(_, _)]]), + Seq(self.asInstanceOf[RDD[(K, _)]], other.asInstanceOf[RDD[(K, _)]]), partitioner) val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) prfs.mapValues { @@ -380,9 +380,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K]( - Seq(self.asInstanceOf[RDD[(_, _)]], - other1.asInstanceOf[RDD[(_, _)]], - other2.asInstanceOf[RDD[(_, _)]]), + Seq(self.asInstanceOf[RDD[(K, _)]], + other1.asInstanceOf[RDD[(K, _)]], + other2.asInstanceOf[RDD[(K, _)]]), partitioner) val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) prfs.mapValues { diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 0a1e2cbee0964..868ee5a39f011 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -40,8 +40,8 @@ private[spark] class CoGroupAggregator { (b1, b2) => b1 ++ b2 }) with Serializable -class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) - extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) with Logging { +class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) + extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { private val aggr = new CoGroupAggregator diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 0d08fd239632b..51ff966ae4880 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -347,7 +347,7 @@ object CheckpointSuite { def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = { //println("First = " + first + ", second = " + second) new CoGroupedRDD[K]( - Seq(first.asInstanceOf[RDD[(_, _)]], second.asInstanceOf[RDD[(_, _)]]), + Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part ).asInstanceOf[RDD[(K, Seq[Seq[V]])]] } diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index fbcf06112676d..5db3844f1d21e 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -457,7 +457,7 @@ extends Serializable { ): DStream[(K, (Seq[V], Seq[W]))] = { val cgd = new CoGroupedDStream[K]( - Seq(self.asInstanceOf[DStream[(_, _)]], other.asInstanceOf[DStream[(_, _)]]), + Seq(self.asInstanceOf[DStream[(K, _)]], other.asInstanceOf[DStream[(K, _)]]), partitioner ) val pdfs = new PairDStreamFunctions[K, Seq[Seq[_]]](cgd)( diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala index ddb1bf6b28fdb..4ef4bb7de1023 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala @@ -6,7 +6,7 @@ import spark.streaming.{Time, DStream, Duration} private[streaming] class CoGroupedDStream[K : ClassManifest]( - parents: Seq[DStream[(_, _)]], + parents: Seq[DStream[(K, _)]], partitioner: Partitioner ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala index 733d5c4a25271..263655039cd11 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -101,7 +101,7 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( val allRDDs = new ArrayBuffer[RDD[(K, V)]]() += previousWindowRDD ++= oldRDDs ++= newRDDs // Cogroup the reduced RDDs and merge the reduced values - val cogroupedRDD = new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(_, _)]]], partitioner) + val cogroupedRDD = new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(K, _)]]], partitioner) //val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _ val numOldValues = oldRDDs.size From e7713adb99f6b377c2c2b79dba08d2ccf5fa8909 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 13:20:48 -0600 Subject: [PATCH 230/696] Move ParallelCollection into spark.rdd package. --- core/src/main/scala/spark/SparkContext.scala | 6 +-- .../ParallelCollectionRDD.scala} | 17 +++----- .../ParallelCollectionSplitSuite.scala | 40 +++++++++---------- 3 files changed, 29 insertions(+), 34 deletions(-) rename core/src/main/scala/spark/{ParallelCollection.scala => rdd/ParallelCollectionRDD.scala} (90%) rename core/src/test/scala/spark/{ => rdd}/ParallelCollectionSplitSuite.scala (83%) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0efc00d5dd338..047b57dc1f18e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -39,7 +39,7 @@ import spark.broadcast._ import spark.deploy.LocalSparkCluster import spark.partial.ApproximateEvaluator import spark.partial.PartialResult -import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD} +import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} @@ -216,7 +216,7 @@ class SparkContext( /** Distribute a local Scala collection to form an RDD. */ def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { - new ParallelCollection[T](this, seq, numSlices, Map[Int, Seq[String]]()) + new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]()) } /** Distribute a local Scala collection to form an RDD. */ @@ -229,7 +229,7 @@ class SparkContext( * Create a new partition for each collection item. */ def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = { val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap - new ParallelCollection[T](this, seq.map(_._1), seq.size, indexToPrefs) + new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs) } /** diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala similarity index 90% rename from core/src/main/scala/spark/ParallelCollection.scala rename to core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index 10adcd53ecafd..e703794787d4f 100644 --- a/core/src/main/scala/spark/ParallelCollection.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -1,8 +1,9 @@ -package spark +package spark.rdd import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer import scala.collection.Map +import spark.{RDD, TaskContext, SparkContext, Split} private[spark] class ParallelCollectionSplit[T: ClassManifest]( val rddId: Long, @@ -22,7 +23,7 @@ private[spark] class ParallelCollectionSplit[T: ClassManifest]( override val index: Int = slice } -private[spark] class ParallelCollection[T: ClassManifest]( +private[spark] class ParallelCollectionRDD[T: ClassManifest]( @transient sc: SparkContext, @transient data: Seq[T], numSlices: Int, @@ -33,26 +34,20 @@ private[spark] class ParallelCollection[T: ClassManifest]( // instead. // UPDATE: A parallel collection can be checkpointed to HDFS, which achieves this goal. - @transient var splits_ : Array[Split] = { - val slices = ParallelCollection.slice(data, numSlices).toArray + override def getSplits: Array[Split] = { + val slices = ParallelCollectionRDD.slice(data, numSlices).toArray slices.indices.map(i => new ParallelCollectionSplit(id, i, slices(i))).toArray } - override def getSplits = splits_ - override def compute(s: Split, context: TaskContext) = s.asInstanceOf[ParallelCollectionSplit[T]].iterator override def getPreferredLocations(s: Split): Seq[String] = { locationPrefs.getOrElse(s.index, Nil) } - - override def clearDependencies() { - splits_ = null - } } -private object ParallelCollection { +private object ParallelCollectionRDD { /** * Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range * collections specially, encoding the slices as other Ranges to minimize memory cost. This makes diff --git a/core/src/test/scala/spark/ParallelCollectionSplitSuite.scala b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala similarity index 83% rename from core/src/test/scala/spark/ParallelCollectionSplitSuite.scala rename to core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala index 450c69bd58adb..d27a2538e4489 100644 --- a/core/src/test/scala/spark/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala @@ -1,4 +1,4 @@ -package spark +package spark.rdd import scala.collection.immutable.NumericRange @@ -11,7 +11,7 @@ import org.scalacheck.Prop._ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("one element per slice") { val data = Array(1, 2, 3) - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices(0).mkString(",") === "1") assert(slices(1).mkString(",") === "2") @@ -20,14 +20,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("one slice") { val data = Array(1, 2, 3) - val slices = ParallelCollection.slice(data, 1) + val slices = ParallelCollectionRDD.slice(data, 1) assert(slices.size === 1) assert(slices(0).mkString(",") === "1,2,3") } test("equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9) - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices(0).mkString(",") === "1,2,3") assert(slices(1).mkString(",") === "4,5,6") @@ -36,7 +36,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("non-equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices(0).mkString(",") === "1,2,3") assert(slices(1).mkString(",") === "4,5,6") @@ -45,7 +45,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("splitting exclusive range") { val data = 0 until 100 - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices(0).mkString(",") === (0 to 32).mkString(",")) assert(slices(1).mkString(",") === (33 to 65).mkString(",")) @@ -54,7 +54,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("splitting inclusive range") { val data = 0 to 100 - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices(0).mkString(",") === (0 to 32).mkString(",")) assert(slices(1).mkString(",") === (33 to 66).mkString(",")) @@ -63,24 +63,24 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("empty data") { val data = new Array[Int](0) - val slices = ParallelCollection.slice(data, 5) + val slices = ParallelCollectionRDD.slice(data, 5) assert(slices.size === 5) for (slice <- slices) assert(slice.size === 0) } test("zero slices") { val data = Array(1, 2, 3) - intercept[IllegalArgumentException] { ParallelCollection.slice(data, 0) } + intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, 0) } } test("negative number of slices") { val data = Array(1, 2, 3) - intercept[IllegalArgumentException] { ParallelCollection.slice(data, -5) } + intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, -5) } } test("exclusive ranges sliced into ranges") { val data = 1 until 100 - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[Range])) @@ -88,7 +88,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("inclusive ranges sliced into ranges") { val data = 1 to 100 - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[Range])) @@ -97,7 +97,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("large ranges don't overflow") { val N = 100 * 1000 * 1000 val data = 0 until N - val slices = ParallelCollection.slice(data, 40) + val slices = ParallelCollectionRDD.slice(data, 40) assert(slices.size === 40) for (i <- 0 until 40) { assert(slices(i).isInstanceOf[Range]) @@ -117,7 +117,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { (tuple: (List[Int], Int)) => val d = tuple._1 val n = tuple._2 - val slices = ParallelCollection.slice(d, n) + val slices = ParallelCollectionRDD.slice(d, n) ("n slices" |: slices.size == n) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) @@ -134,7 +134,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } yield (a until b by step, n) val prop = forAll(gen) { case (d: Range, n: Int) => - val slices = ParallelCollection.slice(d, n) + val slices = ParallelCollectionRDD.slice(d, n) ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && @@ -152,7 +152,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } yield (a to b by step, n) val prop = forAll(gen) { case (d: Range, n: Int) => - val slices = ParallelCollection.slice(d, n) + val slices = ParallelCollectionRDD.slice(d, n) ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && @@ -163,7 +163,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("exclusive ranges of longs") { val data = 1L until 100L - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) @@ -171,7 +171,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("inclusive ranges of longs") { val data = 1L to 100L - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) @@ -179,7 +179,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("exclusive ranges of doubles") { val data = 1.0 until 100.0 by 1.0 - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) @@ -187,7 +187,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { test("inclusive ranges of doubles") { val data = 1.0 to 100.0 by 1.0 - val slices = ParallelCollection.slice(data, 3) + val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) From 924f47dd11dd9b44211372bc7d7960066e26f682 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 16 Feb 2013 13:38:42 -0600 Subject: [PATCH 231/696] Add RDD.subtract. Instead of reusing the cogroup primitive, this adds a SubtractedRDD that knows it only needs to keep rdd1's values (per split) in memory. --- core/src/main/scala/spark/RDD.scala | 21 ++++ .../main/scala/spark/rdd/SubtractedRDD.scala | 108 ++++++++++++++++++ core/src/test/scala/spark/ShuffleSuite.scala | 26 +++++ 3 files changed, 155 insertions(+) create mode 100644 core/src/main/scala/spark/rdd/SubtractedRDD.scala diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index f6e927a989c12..a4c51a0115c0a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -30,6 +30,7 @@ import spark.rdd.MapPartitionsRDD import spark.rdd.MapPartitionsWithSplitRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD +import spark.rdd.SubtractedRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD import spark.storage.StorageLevel @@ -383,6 +384,26 @@ abstract class RDD[T: ClassManifest]( filter(f.isDefinedAt).map(f) } + /** + * Return an RDD with the elements from `this` that are not in `other`. + * + * Uses `this` partitioner/split size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtract(other: RDD[T]): RDD[T] = + subtract(other, partitioner.getOrElse(new HashPartitioner(splits.size))) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: RDD[T], numSplits: Int): RDD[T] = + subtract(other, new HashPartitioner(numSplits)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: RDD[T], p: Partitioner): RDD[T] = new SubtractedRDD[T](this, other, p) + /** * Reduces the elements of this RDD using the specified commutative and associative binary operator. */ diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala new file mode 100644 index 0000000000000..244874e4e0d6b --- /dev/null +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -0,0 +1,108 @@ +package spark.rdd + +import java.util.{HashSet => JHashSet} +import scala.collection.JavaConversions._ +import spark.RDD +import spark.Partitioner +import spark.Dependency +import spark.TaskContext +import spark.Split +import spark.SparkEnv +import spark.ShuffleDependency +import spark.OneToOneDependency + +/** + * An optimized version of cogroup for set difference/subtraction. + * + * It is possible to implement this operation with just `cogroup`, but + * that is less efficient because all of the entries from `rdd2`, for + * both matching and non-matching values in `rdd1`, are kept in the + * JHashMap until the end. + * + * With this implementation, only the entries from `rdd1` are kept in-memory, + * and the entries from `rdd2` are essentially streamed, as we only need to + * touch each once to decide if the value needs to be removed. + * + * This is particularly helpful when `rdd1` is much smaller than `rdd2`, as + * you can use `rdd1`'s partitioner/split size and not worry about running + * out of memory because of the size of `rdd2`. + */ +private[spark] class SubtractedRDD[T: ClassManifest]( + @transient var rdd1: RDD[T], + @transient var rdd2: RDD[T], + part: Partitioner) extends RDD[T](rdd1.context, Nil) { + + override def getDependencies: Seq[Dependency[_]] = { + Seq(rdd1, rdd2).map { rdd => + if (rdd.partitioner == Some(part)) { + logInfo("Adding one-to-one dependency with " + rdd) + new OneToOneDependency(rdd) + } else { + logInfo("Adding shuffle dependency with " + rdd) + val mapSideCombinedRDD = rdd.mapPartitions(i => { + val set = new JHashSet[T]() + while (i.hasNext) { + set.add(i.next) + } + set.iterator + }, true) + // ShuffleDependency requires a tuple (k, v), which it will partition by k. + // We need this to partition to map to the same place as the k for + // OneToOneDependency, which means: + // - for already-tupled RDD[(A, B)], into getPartition(a) + // - for non-tupled RDD[C], into getPartition(c) + val part2 = new Partitioner() { + def numPartitions = part.numPartitions + def getPartition(key: Any) = key match { + case (k, v) => part.getPartition(k) + case k => part.getPartition(k) + } + } + new ShuffleDependency(mapSideCombinedRDD.map((_, null)), part2) + } + } + } + + override def getSplits: Array[Split] = { + val array = new Array[Split](part.numPartitions) + for (i <- 0 until array.size) { + // Each CoGroupSplit will dependend on rdd1 and rdd2 + array(i) = new CoGroupSplit(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) => + dependencies(j) match { + case s: ShuffleDependency[_, _] => + new ShuffleCoGroupSplitDep(s.shuffleId) + case _ => + new NarrowCoGroupSplitDep(rdd, i, rdd.splits(i)) + } + }.toList) + } + array + } + + override val partitioner = Some(part) + + override def compute(s: Split, context: TaskContext): Iterator[T] = { + val split = s.asInstanceOf[CoGroupSplit] + val set = new JHashSet[T] + def integrate(dep: CoGroupSplitDep, op: T => Unit) = dep match { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => + for (k <- rdd.iterator(itsSplit, context)) + op(k.asInstanceOf[T]) + case ShuffleCoGroupSplitDep(shuffleId) => + for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, split.index)) + op(k.asInstanceOf[T]) + } + // the first dep is rdd1; add all keys to the set + integrate(split.deps(0), set.add) + // the second dep is rdd2; remove all of its keys from the set + integrate(split.deps(1), set.remove) + set.iterator + } + + override def clearDependencies() { + super.clearDependencies() + rdd1 = null + rdd2 = null + } + +} \ No newline at end of file diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 3493b9511f6c2..367083eab3c06 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -211,6 +211,32 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(rdd.keys.collect().toList === List(1, 2)) assert(rdd.values.collect().toList === List("a", "b")) } + + test("subtract") { + sc = new SparkContext("local", "test") + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.splits.size === a.splits.size) + } + + test("subtract with narrow dependency") { + sc = new SparkContext("local", "test") + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + println(sc.runJob(a, (i: Iterator[(Int, String)]) => i.toList).toList) + // more splits/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + assert(c.partitioner.get === p) + } } object ShuffleSuite { From 08e444df0e8afec1bcae45ad1edcaff1fc21b39e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 17 Feb 2013 14:01:48 -0800 Subject: [PATCH 232/696] Change EC2 script to use 0.6 AMIs by default, for now --- ec2/spark_ec2.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index ce1072fd3963a..cb8f78db8d01f 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -87,8 +87,8 @@ def parse_args(): parser.add_option("-g", "--ganglia", action="store_true", default=True, help="Setup ganglia monitoring for the cluster. NOTE: The ganglia " + "monitoring page will be publicly accessible") - parser.add_option("--mesos-scripts", action="store_true", default=False, - help="Use older mesos-ec2 scripts to setup the cluster. NOTE: Ganglia " + + parser.add_option("--new-scripts", action="store_true", default=False, + help="Use new spark-ec2 scripts to setup the cluster. NOTE: Ganglia " + "will not be setup with this option") parser.add_option("-u", "--user", default="root", help="The ssh user you want to connect as (default: root)") @@ -380,17 +380,17 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k if opts.ganglia: modules.append('ganglia') - if not opts.mesos_scripts: + if opts.new_scripts: # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/shivaram/spark-ec2.git") + ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git") print "Deploying files to master..." deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, zoo_nodes, modules) print "Running setup on master..." - if opts.mesos_scripts: + if not opts.new_scripts: if opts.cluster_type == "mesos": setup_mesos_cluster(master, opts) elif opts.cluster_type == "standalone": From f98c7da23ef66812b8b4888230ee98c07f09af23 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 17 Feb 2013 15:06:41 -0800 Subject: [PATCH 233/696] Many changes to ensure better 2nd recovery if 2nd failure happens while recovering from 1st failure - Made the scheduler to checkpoint after clearing old metadata which ensures that a new checkpoint is written as soon as at least one batch gets computed while recovering from a failure. This ensures that if there is a 2nd failure while recovering from 1st failure, the system start 2nd recovery from a newer checkpoint. - Modified Checkpoint writer to write checkpoint in a different thread. - Added a check to make sure that compute for InputDStreams gets called only for strictly increasing times. - Changed implementation of slice to call getOrCompute on parent DStream in time-increasing order. - Added testcase to test slice. - Fixed testGroupByKeyAndWindow testcase in JavaAPISuite to verify results with expected output in an order-independent manner. --- .../scala/spark/streaming/Checkpoint.scala | 67 ++++++++++++------- .../main/scala/spark/streaming/DStream.scala | 27 +++++--- .../scala/spark/streaming/DStreamGraph.scala | 13 +++- .../scala/spark/streaming/JobManager.scala | 8 ++- .../scala/spark/streaming/Scheduler.scala | 27 +++++--- .../spark/streaming/StreamingContext.scala | 7 +- .../src/main/scala/spark/streaming/Time.scala | 11 ++- .../api/java/JavaStreamingContext.scala | 7 +- .../streaming/dstream/InputDStream.scala | 36 +++++++++- .../dstream/TwitterInputDStream.scala | 4 +- .../streaming/util/MasterFailureTest.scala | 2 +- .../java/spark/streaming/JavaAPISuite.java | 54 ++++++++++----- .../java/spark/streaming/JavaTestUtils.scala | 1 + streaming/src/test/resources/log4j.properties | 4 +- .../streaming/BasicOperationsSuite.scala | 20 ++++++ .../spark/streaming/CheckpointSuite.scala | 5 +- .../scala/spark/streaming/TestSuiteBase.scala | 7 +- .../streaming/WindowOperationsSuite.scala | 5 +- 18 files changed, 208 insertions(+), 97 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index b9eb7f8ec4a00..7405c8b22ebc9 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -6,6 +6,8 @@ import org.apache.hadoop.fs.{FileUtil, Path} import org.apache.hadoop.conf.Configuration import java.io._ +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} +import java.util.concurrent.Executors private[streaming] @@ -38,32 +40,50 @@ class CheckpointWriter(checkpointDir: String) extends Logging { val conf = new Configuration() var fs = file.getFileSystem(conf) val maxAttempts = 3 + val executor = Executors.newFixedThreadPool(1) - def write(checkpoint: Checkpoint) { - // TODO: maybe do this in a different thread from the main stream execution thread - var attempts = 0 - while (attempts < maxAttempts) { - attempts += 1 - try { - logDebug("Saving checkpoint for time " + checkpoint.checkpointTime + " to file '" + file + "'") - if (fs.exists(file)) { - val bkFile = new Path(file.getParent, file.getName + ".bk") - FileUtil.copy(fs, file, fs, bkFile, true, true, conf) - logDebug("Moved existing checkpoint file to " + bkFile) + class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable { + def run() { + var attempts = 0 + val startTime = System.currentTimeMillis() + while (attempts < maxAttempts) { + attempts += 1 + try { + logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'") + if (fs.exists(file)) { + val bkFile = new Path(file.getParent, file.getName + ".bk") + FileUtil.copy(fs, file, fs, bkFile, true, true, conf) + logDebug("Moved existing checkpoint file to " + bkFile) + } + val fos = fs.create(file) + fos.write(bytes) + fos.close() + fos.close() + val finishTime = System.currentTimeMillis(); + logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file + + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds") + return + } catch { + case ioe: IOException => + logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe) } - val fos = fs.create(file) - val oos = new ObjectOutputStream(fos) - oos.writeObject(checkpoint) - oos.close() - logInfo("Checkpoint for time " + checkpoint.checkpointTime + " saved to file '" + file + "'") - fos.close() - return - } catch { - case ioe: IOException => - logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe) } + logError("Could not write checkpoint for time " + checkpointTime + " to file '" + file + "'") } - logError("Could not write checkpoint for time " + checkpoint.checkpointTime + " to file '" + file + "'") + } + + def write(checkpoint: Checkpoint) { + val bos = new ByteArrayOutputStream() + val zos = new LZFOutputStream(bos) + val oos = new ObjectOutputStream(zos) + oos.writeObject(checkpoint) + oos.close() + bos.close() + executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + } + + def stop() { + executor.shutdown() } } @@ -85,7 +105,8 @@ object CheckpointReader extends Logging { // of ObjectInputStream is used to explicitly use the current thread's default class // loader to find and load classes. This is a well know Java issue and has popped up // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) - val ois = new ObjectInputStreamWithLoader(fis, Thread.currentThread().getContextClassLoader) + val zis = new LZFInputStream(fis) + val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader) val cp = ois.readObject.asInstanceOf[Checkpoint] ois.close() fs.close() diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index ce42b742d77e4..84e4b5bedb8df 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -238,13 +238,15 @@ abstract class DStream[T: ClassManifest] ( dependencies.foreach(_.remember(parentRememberDuration)) } - /** This method checks whether the 'time' is valid wrt slideDuration for generating RDD */ + /** Checks whether the 'time' is valid wrt slideDuration for generating RDD */ protected def isTimeValid(time: Time): Boolean = { if (!isInitialized) { throw new Exception (this + " has not been initialized") } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { + logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime)) false } else { + logInfo("Time " + time + " is valid") true } } @@ -627,16 +629,21 @@ abstract class DStream[T: ClassManifest] ( * Return all the RDDs between 'fromTime' to 'toTime' (both included) */ def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = { - val rdds = new ArrayBuffer[RDD[T]]() - var time = toTime.floor(slideDuration) - while (time >= zeroTime && time >= fromTime) { - getOrCompute(time) match { - case Some(rdd) => rdds += rdd - case None => //throw new Exception("Could not get RDD for time " + time) - } - time -= slideDuration + if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) { + logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")") + } + if (!(toTime - zeroTime).isMultipleOf(slideDuration)) { + logWarning("toTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")") } - rdds.toSeq + val alignedToTime = toTime.floor(slideDuration) + val alignedFromTime = fromTime.floor(slideDuration) + + logInfo("Slicing from " + fromTime + " to " + toTime + + " (aligned to " + alignedFromTime + " and " + alignedToTime + ")") + + alignedFromTime.to(alignedToTime, slideDuration).flatMap(time => { + if (time >= zeroTime) getOrCompute(time) else None + }) } /** diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index 22d9e24f05b99..adb7f3a24d25f 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -86,10 +86,12 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def getOutputStreams() = this.synchronized { outputStreams.toArray } - def generateRDDs(time: Time): Seq[Job] = { + def generateJobs(time: Time): Seq[Job] = { this.synchronized { - logInfo("Generating RDDs for time " + time) - outputStreams.flatMap(outputStream => outputStream.generateJob(time)) + logInfo("Generating jobs for time " + time) + val jobs = outputStreams.flatMap(outputStream => outputStream.generateJob(time)) + logInfo("Generated " + jobs.length + " jobs for time " + time) + jobs } } @@ -97,18 +99,23 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { this.synchronized { logInfo("Clearing old metadata for time " + time) outputStreams.foreach(_.clearOldMetadata(time)) + logInfo("Cleared old metadata for time " + time) } } def updateCheckpointData(time: Time) { this.synchronized { + logInfo("Updating checkpoint data for time " + time) outputStreams.foreach(_.updateCheckpointData(time)) + logInfo("Updated checkpoint data for time " + time) } } def restoreCheckpointData() { this.synchronized { + logInfo("Restoring checkpoint data") outputStreams.foreach(_.restoreCheckpointData()) + logInfo("Restored checkpoint data") } } diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala index 649494ff4a8d6..7696c4a592bf6 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/spark/streaming/JobManager.scala @@ -43,20 +43,24 @@ class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging { } private def clearJob(job: Job) { + var timeCleared = false + val time = job.time jobs.synchronized { - val time = job.time val jobsOfTime = jobs.get(time) if (jobsOfTime.isDefined) { jobsOfTime.get -= job if (jobsOfTime.get.isEmpty) { - ssc.scheduler.clearOldMetadata(time) jobs -= time + timeCleared = true } } else { throw new Exception("Job finished for time " + job.time + " but time does not exist in jobs") } } + if (timeCleared) { + ssc.scheduler.clearOldMetadata(time) + } } def getPendingTimes(): Array[Time] = { diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala index 57d494da8378e..1c4b22a8981c8 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala @@ -20,8 +20,9 @@ class Scheduler(ssc: StreamingContext) extends Logging { val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock") val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock] val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, - longTime => generateRDDs(new Time(longTime))) + longTime => generateJobs(new Time(longTime))) val graph = ssc.graph + var latestTime: Time = null def start() = synchronized { if (ssc.isCheckpointPresent) { @@ -35,6 +36,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { def stop() = synchronized { timer.stop() jobManager.stop() + if (checkpointWriter != null) checkpointWriter.stop() ssc.graph.stop() logInfo("Scheduler stopped") } @@ -73,35 +75,38 @@ class Scheduler(ssc: StreamingContext) extends Logging { val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering) logInfo("Batches to reschedule: " + timesToReschedule.mkString(", ")) timesToReschedule.foreach(time => - graph.generateRDDs(time).foreach(jobManager.runJob) + graph.generateJobs(time).foreach(jobManager.runJob) ) // Restart the timer timer.start(restartTime.milliseconds) - logInfo("Scheduler's timer restarted") + logInfo("Scheduler's timer restarted at " + restartTime) } - /** Generates the RDDs, clears old metadata and does checkpoint for the given time */ - def generateRDDs(time: Time) { + /** Generate jobs and perform checkpoint for the given `time`. */ + def generateJobs(time: Time) { SparkEnv.set(ssc.env) logInfo("\n-----------------------------------------------------\n") - graph.generateRDDs(time).foreach(jobManager.runJob) + graph.generateJobs(time).foreach(jobManager.runJob) + latestTime = time doCheckpoint(time) } - + /** + * Clear old metadata assuming jobs of `time` have finished processing. + * And also perform checkpoint. + */ def clearOldMetadata(time: Time) { ssc.graph.clearOldMetadata(time) + doCheckpoint(time) } - def doCheckpoint(time: Time) { + /** Perform checkpoint for the give `time`. */ + def doCheckpoint(time: Time) = synchronized { if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { logInfo("Checkpointing graph for time " + time) - val startTime = System.currentTimeMillis() ssc.graph.updateCheckpointData(time) checkpointWriter.write(new Checkpoint(ssc, time)) - val stopTime = System.currentTimeMillis() - logInfo("Checkpointing the graph took " + (stopTime - startTime) + " ms") } } } diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 9be9d884be66c..d1407b7869c03 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -119,18 +119,15 @@ class StreamingContext private ( /** * Set the context to periodically checkpoint the DStream operations for master - * fault-tolerance. By default, the graph will be checkpointed every batch interval. + * fault-tolerance. The graph will be checkpointed every batch interval. * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored - * @param interval checkpoint interval */ - def checkpoint(directory: String, interval: Duration = null) { + def checkpoint(directory: String) { if (directory != null) { sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory)) checkpointDir = directory - checkpointDuration = interval } else { checkpointDir = null - checkpointDuration = null } } diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala index 8201e84a203d7..f14decf08ba8f 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/spark/streaming/Time.scala @@ -38,15 +38,14 @@ case class Time(private val millis: Long) { def max(that: Time): Time = if (this > that) this else that def until(that: Time, interval: Duration): Seq[Time] = { - assert(that > this, "Cannot create sequence as " + that + " not more than " + this) - assert( - (that - this).isMultipleOf(interval), - "Cannot create sequence as gap between " + that + " and " + - this + " is not multiple of " + interval - ) (this.milliseconds) until (that.milliseconds) by (interval.milliseconds) map (new Time(_)) } + def to(that: Time, interval: Duration): Seq[Time] = { + (this.milliseconds) to (that.milliseconds) by (interval.milliseconds) map (new Time(_)) + } + + override def toString: String = (millis.toString + " ms") } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 5bbf2b084f32f..03933aae93680 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -314,12 +314,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Sets the context to periodically checkpoint the DStream operations for master - * fault-tolerance. By default, the graph will be checkpointed every batch interval. + * fault-tolerance. The graph will be checkpointed every batch interval. * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored - * @param interval checkpoint interval */ - def checkpoint(directory: String, interval: Duration = null) { - ssc.checkpoint(directory, interval) + def checkpoint(directory: String) { + ssc.checkpoint(directory) } /** diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala index 980ca5177eb1d..a4db44a608e08 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala @@ -1,10 +1,42 @@ package spark.streaming.dstream -import spark.streaming.{Duration, StreamingContext, DStream} +import spark.streaming.{Time, Duration, StreamingContext, DStream} +/** + * This is the abstract base class for all input streams. This class provides to methods + * start() and stop() which called by the scheduler to start and stop receiving data/ + * Input streams that can generated RDDs from new data just by running a service on + * the driver node (that is, without running a receiver onworker nodes) can be + * implemented by directly subclassing this InputDStream. For example, + * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory for + * new files and generates RDDs on the new files. For implementing input streams + * that requires running a receiver on the worker nodes, use NetworkInputDStream + * as the parent class. + */ abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext) extends DStream[T](ssc_) { + var lastValidTime: Time = null + + /** + * Checks whether the 'time' is valid wrt slideDuration for generating RDD. + * Additionally it also ensures valid times are in strictly increasing order. + * This ensures that InputDStream.compute() is called strictly on increasing + * times. + */ + override protected def isTimeValid(time: Time): Boolean = { + if (!super.isTimeValid(time)) { + false // Time not valid + } else { + // Time is valid, but check it it is more than lastValidTime + if (lastValidTime == null || lastValidTime <= time) { + logWarning("isTimeValid called with " + time + " where as last valid time is " + lastValidTime) + } + lastValidTime = time + true + } + } + override def dependencies = List() override def slideDuration: Duration = { @@ -13,7 +45,9 @@ abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContex ssc.graph.batchDuration } + /** Method called to start receiving data. Subclasses must implement this method. */ def start() + /** Method called to stop receiving data. Subclasses must implement this method. */ def stop() } diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index d733254ddbb91..e70822e5c3bd3 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -2,8 +2,8 @@ package spark.streaming.dstream import spark._ import spark.streaming._ -import dstream.{NetworkReceiver, NetworkInputDStream} import storage.StorageLevel + import twitter4j._ import twitter4j.auth.BasicAuthorization @@ -19,7 +19,7 @@ class TwitterInputDStream( password: String, filters: Seq[String], storageLevel: StorageLevel - ) extends NetworkInputDStream[Status](ssc_) { + ) extends NetworkInputDStream[Status](ssc_) { override def createReceiver(): NetworkReceiver[Status] = { new TwitterReceiver(username, password, filters, storageLevel) diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index 776e676063c1c..bdd9f4d7535ea 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -315,7 +315,7 @@ class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread override def run() { try { // If it is the first killing, then allow the first checkpoint to be created - var minKillWaitTime = if (MasterFailureTest.killCount == 0) 5000 else 1000 + var minKillWaitTime = if (MasterFailureTest.killCount == 0) 5000 else 2000 val killWaitTime = minKillWaitTime + math.abs(Random.nextLong % maxKillWaitTime) logInfo("Kill wait time = " + killWaitTime) Thread.sleep(killWaitTime) diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 7bea0b1fc4e73..16bacffb9285f 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -23,6 +23,7 @@ import spark.streaming.dstream.KafkaPartitionKey; import java.io.*; +import java.text.Collator; import java.util.*; // The test suite itself is Serializable so that anonymous Function implementations can be @@ -35,7 +36,7 @@ public class JavaAPISuite implements Serializable { public void setUp() { System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock"); ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); - ssc.checkpoint("checkpoint", new Duration(1000)); + ssc.checkpoint("checkpoint"); } @After @@ -587,26 +588,47 @@ public void testCountByValue() { @Test public void testGroupByKeyAndWindow() { - List>> inputData = stringStringKVStream; + List>> inputData = stringIntKVStream; - List>>> expected = Arrays.asList( - Arrays.asList(new Tuple2>("california", Arrays.asList("dodgers", "giants")), - new Tuple2>("new york", Arrays.asList("yankees", "mets"))), - Arrays.asList(new Tuple2>("california", - Arrays.asList("sharks", "ducks", "dodgers", "giants")), - new Tuple2>("new york", Arrays.asList("rangers", "islanders", "yankees", "mets"))), - Arrays.asList(new Tuple2>("california", Arrays.asList("sharks", "ducks")), - new Tuple2>("new york", Arrays.asList("rangers", "islanders")))); + List>>> expected = Arrays.asList( + Arrays.asList( + new Tuple2>("california", Arrays.asList(1, 3)), + new Tuple2>("new york", Arrays.asList(1, 4)) + ), + Arrays.asList( + new Tuple2>("california", Arrays.asList(1, 3, 5, 5)), + new Tuple2>("new york", Arrays.asList(1, 1, 3, 4)) + ), + Arrays.asList( + new Tuple2>("california", Arrays.asList(5, 5)), + new Tuple2>("new york", Arrays.asList(1, 3)) + ) + ); - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> groupWindowed = + JavaPairDStream> groupWindowed = pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(groupWindowed); - List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); + List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); - Assert.assertEquals(expected, result); + assert(result.size() == expected.size()); + for (int i = 0; i < result.size(); i++) { + assert(convert(result.get(i)).equals(convert(expected.get(i)))); + } + } + + private HashSet>> convert(List>> listOfTuples) { + List>> newListOfTuples = new ArrayList>>(); + for (Tuple2> tuple: listOfTuples) { + newListOfTuples.add(convert(tuple)); + } + return new HashSet>>(newListOfTuples); + } + + private Tuple2> convert(Tuple2> tuple) { + return new Tuple2>(tuple._1(), new HashSet(tuple._2())); } @Test @@ -894,7 +916,7 @@ public void testCheckpointMasterRecovery() throws InterruptedException { Arrays.asList(8,7)); File tempDir = Files.createTempDir(); - ssc.checkpoint(tempDir.getAbsolutePath(), new Duration(1000)); + ssc.checkpoint(tempDir.getAbsolutePath()); JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); JavaDStream letterCount = stream.map(new Function() { diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala index 56349837e5e6b..52ea28732ada8 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala @@ -57,6 +57,7 @@ trait JavaTestBase extends TestSuiteBase { } object JavaTestUtils extends JavaTestBase { + override def maxWaitTimeMillis = 20000 } diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index f0638e0e027f2..59c445e63f797 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -1,5 +1,5 @@ # Set everything to be logged to the file streaming/target/unit-tests.log -log4j.rootCategory=WARN, file +log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false @@ -9,6 +9,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN -log4j.logger.spark.streaming=INFO -log4j.logger.spark.streaming.dstream.FileInputDStream=DEBUG diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 1e86cf49bb75a..8fce91853c77e 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -229,6 +229,26 @@ class BasicOperationsSuite extends TestSuiteBase { testOperation(inputData, updateStateOperation, outputData, true) } + test("slice") { + val ssc = new StreamingContext("local[2]", "BasicOperationSuite", Seconds(1)) + val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) + val stream = new TestInputStream[Int](ssc, input, 2) + ssc.registerInputStream(stream) + stream.foreach(_ => {}) // Dummy output stream + ssc.start() + Thread.sleep(2000) + def getInputFromSlice(fromMillis: Long, toMillis: Long) = { + stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet + } + + assert(getInputFromSlice(0, 1000) == Set(1)) + assert(getInputFromSlice(0, 2000) == Set(1, 2)) + assert(getInputFromSlice(1000, 2000) == Set(1, 2)) + assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4)) + ssc.stop() + Thread.sleep(1000) + } + test("forgetting of RDDs - map and window operations") { assert(batchDuration === Seconds(1), "Batch duration has changed from 1 second") diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index c89c4a8d43f80..5250667bcbc0d 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -39,14 +39,11 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { override def batchDuration = Milliseconds(500) - override def checkpointInterval = batchDuration - override def actuallyWait = true test("basic rdd checkpoints + dstream graph checkpoint recovery") { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") - assert(checkpointInterval === batchDuration, "checkpointInterval for this test much be same as batchDuration") System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") @@ -188,7 +185,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val testDir = Files.createTempDir() var ssc = new StreamingContext(master, framework, Seconds(1)) - ssc.checkpoint(checkpointDir, checkpointInterval) + ssc.checkpoint(checkpointDir) val fileStream = ssc.textFileStream(testDir.toString) // Making value 3 take large time to process, to ensure that the master // shuts down in the middle of processing the 3rd batch diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index 2cc31d61376ad..ad6aa79d10233 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -75,9 +75,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Directory where the checkpoint data will be saved def checkpointDir = "checkpoint" - // Duration after which the graph is checkpointed - def checkpointInterval = batchDuration - // Number of partitions of the input parallel collections created for testing def numInputPartitions = 2 @@ -99,7 +96,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Create StreamingContext val ssc = new StreamingContext(master, framework, batchDuration) if (checkpointDir != null) { - ssc.checkpoint(checkpointDir, checkpointInterval) + ssc.checkpoint(checkpointDir) } // Setup the stream computation @@ -124,7 +121,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Create StreamingContext val ssc = new StreamingContext(master, framework, batchDuration) if (checkpointDir != null) { - ssc.checkpoint(checkpointDir, checkpointInterval) + ssc.checkpoint(checkpointDir) } // Setup the stream computation diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index f8380af331301..1b66f3bda20ad 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -273,6 +273,7 @@ class WindowOperationsSuite extends TestSuiteBase { slideDuration: Duration = Seconds(1) ) { test("reduceByKeyAndWindow - " + name) { + logInfo("reduceByKeyAndWindow - " + name) val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt val operation = (s: DStream[(String, Int)]) => { s.reduceByKeyAndWindow((x: Int, y: Int) => x + y, windowDuration, slideDuration) @@ -288,7 +289,8 @@ class WindowOperationsSuite extends TestSuiteBase { windowDuration: Duration = Seconds(2), slideDuration: Duration = Seconds(1) ) { - test("ReduceByKeyAndWindow with inverse function - " + name) { + test("reduceByKeyAndWindow with inverse function - " + name) { + logInfo("reduceByKeyAndWindow with inverse function - " + name) val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt val operation = (s: DStream[(String, Int)]) => { s.reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration) @@ -306,6 +308,7 @@ class WindowOperationsSuite extends TestSuiteBase { slideDuration: Duration = Seconds(1) ) { test("reduceByKeyAndWindow with inverse and filter functions - " + name) { + logInfo("reduceByKeyAndWindow with inverse and filter functions - " + name) val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt val filterFunc = (p: (String, Int)) => p._2 != 0 val operation = (s: DStream[(String, Int)]) => { From 455d015076ab1fcafa99484c8dcf7cc9d740686a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 17 Feb 2013 16:53:12 -0800 Subject: [PATCH 234/696] Clean up EC2 script options a bit --- ec2/spark_ec2.py | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index cb8f78db8d01f..7967bcac50822 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -82,18 +82,21 @@ def parse_args(): parser.add_option("--spot-price", metavar="PRICE", type="float", help="If specified, launch slaves as spot instances with the given " + "maximum price (in dollars)") - parser.add_option("-c", "--cluster-type", default="mesos", - help="'mesos' for a mesos cluster, 'standalone' for a standalone spark cluster (default: mesos)") - parser.add_option("-g", "--ganglia", action="store_true", default=True, - help="Setup ganglia monitoring for the cluster. NOTE: The ganglia " + - "monitoring page will be publicly accessible") + parser.add_option("--cluster-type", type="choice", metavar="TYPE", + choices=["mesos", "standalone"], default="mesos", + help="'mesos' for a Mesos cluster, 'standalone' for a standalone " + + "Spark cluster (default: mesos)") + parser.add_option("--ganglia", action="store_true", default=True, + help="Setup Ganglia monitoring on cluster (default: on). NOTE: " + + "the Ganglia page will be publicly accessible") + parser.add_option("--no-ganglia", action="store_false", dest="ganglia", + help="Disable Ganglia monitoring for the cluster") parser.add_option("--new-scripts", action="store_true", default=False, - help="Use new spark-ec2 scripts to setup the cluster. NOTE: Ganglia " + - "will not be setup with this option") + help="Use new spark-ec2 scripts, for Spark >= 0.7 AMIs") parser.add_option("-u", "--user", default="root", - help="The ssh user you want to connect as (default: root)") + help="The SSH user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, - help="When destroying a cluster, also destroy the security groups that were created") + help="When destroying a cluster, delete the security groups that were created") (opts, args) = parser.parse_args() if len(args) != 2: From 06e5e6627f3856b5c6e3e60cbb167044de9ef6d4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 17 Feb 2013 22:13:26 -0800 Subject: [PATCH 235/696] Renamed "splits" to "partitions" --- bagel/src/main/scala/spark/bagel/Bagel.scala | 20 ++-- .../bagel/examples/WikipediaPageRank.scala | 6 +- .../WikipediaPageRankStandalone.scala | 2 +- core/src/main/scala/spark/CacheManager.scala | 4 +- .../main/scala/spark/DoubleRDDFunctions.scala | 4 +- .../main/scala/spark/PairRDDFunctions.scala | 50 ++++---- .../spark/{Split.scala => Partition.scala} | 2 +- core/src/main/scala/spark/RDD.scala | 76 ++++++------ .../main/scala/spark/RDDCheckpointData.scala | 12 +- core/src/main/scala/spark/SparkContext.scala | 10 +- .../scala/spark/api/java/JavaDoubleRDD.scala | 6 +- .../scala/spark/api/java/JavaPairRDD.scala | 44 +++---- .../main/scala/spark/api/java/JavaRDD.scala | 6 +- .../scala/spark/api/java/JavaRDDLike.scala | 12 +- .../scala/spark/api/python/PythonRDD.scala | 10 +- .../partial/ApproximateActionListener.scala | 2 +- core/src/main/scala/spark/rdd/BlockRDD.scala | 16 +-- .../main/scala/spark/rdd/CartesianRDD.scala | 36 +++--- .../main/scala/spark/rdd/CheckpointRDD.scala | 20 ++-- .../main/scala/spark/rdd/CoGroupedRDD.scala | 22 ++-- .../main/scala/spark/rdd/CoalescedRDD.scala | 24 ++-- .../main/scala/spark/rdd/FilteredRDD.scala | 6 +- .../main/scala/spark/rdd/FlatMappedRDD.scala | 6 +- .../src/main/scala/spark/rdd/GlommedRDD.scala | 6 +- core/src/main/scala/spark/rdd/HadoopRDD.scala | 20 ++-- .../scala/spark/rdd/MapPartitionsRDD.scala | 8 +- ....scala => MapPartitionsWithIndexRDD.scala} | 12 +- core/src/main/scala/spark/rdd/MappedRDD.scala | 6 +- .../main/scala/spark/rdd/NewHadoopRDD.scala | 20 ++-- .../spark/rdd/ParallelCollectionRDD.scala | 18 +-- .../scala/spark/rdd/PartitionPruningRDD.scala | 16 +-- core/src/main/scala/spark/rdd/PipedRDD.scala | 6 +- .../src/main/scala/spark/rdd/SampledRDD.scala | 16 +-- .../main/scala/spark/rdd/ShuffledRDD.scala | 10 +- core/src/main/scala/spark/rdd/UnionRDD.scala | 30 ++--- core/src/main/scala/spark/rdd/ZippedRDD.scala | 32 ++--- .../scala/spark/scheduler/DAGScheduler.scala | 14 +-- .../scala/spark/scheduler/ResultTask.scala | 6 +- .../spark/scheduler/ShuffleMapTask.scala | 6 +- .../main/scala/spark/scheduler/Stage.scala | 2 +- .../scala/spark/storage/BlockManager.scala | 2 +- .../scala/spark/storage/StorageUtils.scala | 2 +- .../test/scala/spark/CheckpointSuite.scala | 110 +++++++++--------- core/src/test/scala/spark/RDDSuite.scala | 13 ++- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- core/src/test/scala/spark/SortingSuite.scala | 10 +- .../spark/scheduler/DAGSchedulerSuite.scala | 22 ++-- .../spark/scheduler/TaskContextSuite.scala | 10 +- 48 files changed, 405 insertions(+), 390 deletions(-) rename core/src/main/scala/spark/{Split.scala => Partition.scala} (84%) rename core/src/main/scala/spark/rdd/{MapPartitionsWithSplitRDD.scala => MapPartitionsWithIndexRDD.scala} (57%) diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala index fa0ba4a57319e..094e57dacb7cc 100644 --- a/bagel/src/main/scala/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/spark/bagel/Bagel.scala @@ -14,11 +14,11 @@ object Bagel extends Logging { combiner: Combiner[M, C], aggregator: Option[Aggregator[V, A]], partitioner: Partitioner, - numSplits: Int + numPartitions: Int )( compute: (V, Option[C], Option[A], Int) => (V, Array[M]) ): RDD[(K, V)] = { - val splits = if (numSplits != 0) numSplits else sc.defaultParallelism + val splits = if (numPartitions != 0) numPartitions else sc.defaultParallelism var superstep = 0 var verts = vertices @@ -56,12 +56,12 @@ object Bagel extends Logging { messages: RDD[(K, M)], combiner: Combiner[M, C], partitioner: Partitioner, - numSplits: Int + numPartitions: Int )( compute: (V, Option[C], Int) => (V, Array[M]) ): RDD[(K, V)] = { run[K, V, M, C, Nothing]( - sc, vertices, messages, combiner, None, partitioner, numSplits)( + sc, vertices, messages, combiner, None, partitioner, numPartitions)( addAggregatorArg[K, V, M, C](compute)) } @@ -70,13 +70,13 @@ object Bagel extends Logging { vertices: RDD[(K, V)], messages: RDD[(K, M)], combiner: Combiner[M, C], - numSplits: Int + numPartitions: Int )( compute: (V, Option[C], Int) => (V, Array[M]) ): RDD[(K, V)] = { - val part = new HashPartitioner(numSplits) + val part = new HashPartitioner(numPartitions) run[K, V, M, C, Nothing]( - sc, vertices, messages, combiner, None, part, numSplits)( + sc, vertices, messages, combiner, None, part, numPartitions)( addAggregatorArg[K, V, M, C](compute)) } @@ -84,13 +84,13 @@ object Bagel extends Logging { sc: SparkContext, vertices: RDD[(K, V)], messages: RDD[(K, M)], - numSplits: Int + numPartitions: Int )( compute: (V, Option[Array[M]], Int) => (V, Array[M]) ): RDD[(K, V)] = { - val part = new HashPartitioner(numSplits) + val part = new HashPartitioner(numPartitions) run[K, V, M, Array[M], Nothing]( - sc, vertices, messages, new DefaultCombiner(), None, part, numSplits)( + sc, vertices, messages, new DefaultCombiner(), None, part, numPartitions)( addAggregatorArg[K, V, M, Array[M]](compute)) } diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala index 03843019c010c..bc32663e0fde6 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala @@ -16,7 +16,7 @@ import scala.xml.{XML,NodeSeq} object WikipediaPageRank { def main(args: Array[String]) { if (args.length < 5) { - System.err.println("Usage: WikipediaPageRank ") + System.err.println("Usage: WikipediaPageRank ") System.exit(-1) } @@ -25,7 +25,7 @@ object WikipediaPageRank { val inputFile = args(0) val threshold = args(1).toDouble - val numSplits = args(2).toInt + val numPartitions = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean val sc = new SparkContext(host, "WikipediaPageRank") @@ -69,7 +69,7 @@ object WikipediaPageRank { val result = Bagel.run( sc, vertices, messages, combiner = new PRCombiner(), - numSplits = numSplits)( + numPartitions = numPartitions)( utils.computeWithCombiner(numVertices, epsilon)) // Print the result diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala index 06cc8c748b982..9d9d80d809d0d 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala @@ -88,7 +88,7 @@ object WikipediaPageRankStandalone { n: Long, partitioner: Partitioner, usePartitioner: Boolean, - numSplits: Int + numPartitions: Int ): RDD[(String, Double)] = { var ranks = links.mapValues { edges => defaultRank } for (i <- 1 to numIterations) { diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala index 711435c3335fa..c7b379a3fbe2d 100644 --- a/core/src/main/scala/spark/CacheManager.scala +++ b/core/src/main/scala/spark/CacheManager.scala @@ -11,13 +11,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { private val loading = new HashSet[String] /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */ - def getOrCompute[T](rdd: RDD[T], split: Split, context: TaskContext, storageLevel: StorageLevel) + def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, storageLevel: StorageLevel) : Iterator[T] = { val key = "rdd_%d_%d".format(rdd.id, split.index) logInfo("Cache key is " + key) blockManager.get(key) match { case Some(cachedValues) => - // Split is in cache, so just return its values + // Partition is in cache, so just return its values logInfo("Found partition in cache!") return cachedValues.asInstanceOf[Iterator[T]] diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala index b2a0e2b631e7a..178d31a73b913 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala @@ -42,14 +42,14 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new MeanEvaluator(self.splits.size, confidence) + val evaluator = new MeanEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new SumEvaluator(self.splits.size, confidence) + val evaluator = new SumEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 019be11ea8848..4319cbd892a37 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -83,8 +83,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - numSplits: Int): RDD[(K, C)] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numSplits)) + numPartitions: Int): RDD[(K, C)] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) } /** @@ -145,10 +145,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numSplits splits. + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. */ - def reduceByKey(func: (V, V) => V, numSplits: Int): RDD[(K, V)] = { - reduceByKey(new HashPartitioner(numSplits), func) + def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { + reduceByKey(new HashPartitioner(numPartitions), func) } /** @@ -166,10 +166,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with into `numSplits` partitions. + * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numSplits: Int): RDD[(K, Seq[V])] = { - groupByKey(new HashPartitioner(numSplits)) + def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { + groupByKey(new HashPartitioner(numPartitions)) } /** @@ -287,8 +287,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (V, W))] = { - join(other, new HashPartitioner(numSplits)) + def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + join(other, new HashPartitioner(numPartitions)) } /** @@ -305,10 +305,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * into `numSplits` partitions. + * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (V, Option[W]))] = { - leftOuterJoin(other, new HashPartitioner(numSplits)) + def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + leftOuterJoin(other, new HashPartitioner(numPartitions)) } /** @@ -327,8 +327,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (Option[V], W))] = { - rightOuterJoin(other, new HashPartitioner(numSplits)) + def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + rightOuterJoin(other, new HashPartitioner(numPartitions)) } /** @@ -414,17 +414,17 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (Seq[V], Seq[W]))] = { - cogroup(other, new HashPartitioner(numSplits)) + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + cogroup(other, new HashPartitioner(numPartitions)) } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numSplits: Int) + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - cogroup(other1, other2, new HashPartitioner(numSplits)) + cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ @@ -636,9 +636,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(ascending: Boolean = true, numSplits: Int = self.splits.size): RDD[(K,V)] = { + def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[(K,V)] = { val shuffled = - new ShuffledRDD[K, V](self, new RangePartitioner(numSplits, self, ascending)) + new ShuffledRDD[K, V](self, new RangePartitioner(numPartitions, self, ascending)) shuffled.mapPartitions(iter => { val buf = iter.toArray if (ascending) { @@ -652,9 +652,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( private[spark] class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev) { - override def getSplits = firstParent[(K, V)].splits + override def getPartitions = firstParent[(K, V)].partitions override val partitioner = firstParent[(K, V)].partitioner - override def compute(split: Split, context: TaskContext) = + override def compute(split: Partition, context: TaskContext) = firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) } } @@ -662,9 +662,9 @@ private[spark] class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U]) extends RDD[(K, U)](prev) { - override def getSplits = firstParent[(K, V)].splits + override def getPartitions = firstParent[(K, V)].partitions override val partitioner = firstParent[(K, V)].partitioner - override def compute(split: Split, context: TaskContext) = { + override def compute(split: Partition, context: TaskContext) = { firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) } } } diff --git a/core/src/main/scala/spark/Split.scala b/core/src/main/scala/spark/Partition.scala similarity index 84% rename from core/src/main/scala/spark/Split.scala rename to core/src/main/scala/spark/Partition.scala index 90d4b47c553c5..e384308ef6e93 100644 --- a/core/src/main/scala/spark/Split.scala +++ b/core/src/main/scala/spark/Partition.scala @@ -3,7 +3,7 @@ package spark /** * A partition of an RDD. */ -trait Split extends Serializable { +trait Partition extends Serializable { /** * Get the split's index within its parent RDD */ diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index f6e927a989c12..da82dfd10f290 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -27,7 +27,7 @@ import spark.rdd.FlatMappedRDD import spark.rdd.GlommedRDD import spark.rdd.MappedRDD import spark.rdd.MapPartitionsRDD -import spark.rdd.MapPartitionsWithSplitRDD +import spark.rdd.MapPartitionsWithIndexRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD import spark.rdd.UnionRDD @@ -49,7 +49,7 @@ import SparkContext._ * * Internally, each RDD is characterized by five main properties: * - * - A list of splits (partitions) + * - A list of partitions * - A function for computing each split * - A list of dependencies on other RDDs * - Optionally, a Partitioner for key-value RDDs (e.g. to say that the RDD is hash-partitioned) @@ -76,13 +76,13 @@ abstract class RDD[T: ClassManifest]( // ======================================================================= /** Implemented by subclasses to compute a given partition. */ - def compute(split: Split, context: TaskContext): Iterator[T] + def compute(split: Partition, context: TaskContext): Iterator[T] /** * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - protected def getSplits: Array[Split] + protected def getPartitions: Array[Partition] /** * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only @@ -91,7 +91,7 @@ abstract class RDD[T: ClassManifest]( protected def getDependencies: Seq[Dependency[_]] = deps /** Optionally overridden by subclasses to specify placement preferences. */ - protected def getPreferredLocations(split: Split): Seq[String] = Nil + protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ val partitioner: Option[Partitioner] = None @@ -137,10 +137,10 @@ abstract class RDD[T: ClassManifest]( /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ def getStorageLevel = storageLevel - // Our dependencies and splits will be gotten by calling subclass's methods below, and will + // Our dependencies and partitions will be gotten by calling subclass's methods below, and will // be overwritten when we're checkpointed private var dependencies_ : Seq[Dependency[_]] = null - @transient private var splits_ : Array[Split] = null + @transient private var partitions_ : Array[Partition] = null /** An Option holding our checkpoint RDD, if we are checkpointed */ private def checkpointRDD: Option[RDD[T]] = checkpointData.flatMap(_.checkpointRDD) @@ -159,15 +159,15 @@ abstract class RDD[T: ClassManifest]( } /** - * Get the array of splits of this RDD, taking into account whether the + * Get the array of partitions of this RDD, taking into account whether the * RDD is checkpointed or not. */ - final def splits: Array[Split] = { - checkpointRDD.map(_.splits).getOrElse { - if (splits_ == null) { - splits_ = getSplits + final def partitions: Array[Partition] = { + checkpointRDD.map(_.partitions).getOrElse { + if (partitions_ == null) { + partitions_ = getPartitions } - splits_ + partitions_ } } @@ -175,7 +175,7 @@ abstract class RDD[T: ClassManifest]( * Get the preferred location of a split, taking into account whether the * RDD is checkpointed or not. */ - final def preferredLocations(split: Split): Seq[String] = { + final def preferredLocations(split: Partition): Seq[String] = { checkpointRDD.map(_.getPreferredLocations(split)).getOrElse { getPreferredLocations(split) } @@ -186,7 +186,7 @@ abstract class RDD[T: ClassManifest]( * This should ''not'' be called by users directly, but is available for implementors of custom * subclasses of RDD. */ - final def iterator(split: Split, context: TaskContext): Iterator[T] = { + final def iterator(split: Partition, context: TaskContext): Iterator[T] = { if (storageLevel != StorageLevel.NONE) { SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel) } else { @@ -197,7 +197,7 @@ abstract class RDD[T: ClassManifest]( /** * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. */ - private[spark] def computeOrReadCheckpoint(split: Split, context: TaskContext): Iterator[T] = { + private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = { if (isCheckpointed) { firstParent[T].iterator(split, context) } else { @@ -227,15 +227,15 @@ abstract class RDD[T: ClassManifest]( /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numSplits: Int): RDD[T] = - map(x => (x, null)).reduceByKey((x, y) => x, numSplits).map(_._1) + def distinct(numPartitions: Int): RDD[T] = + map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) - def distinct(): RDD[T] = distinct(splits.size) + def distinct(): RDD[T] = distinct(partitions.size) /** - * Return a new RDD that is reduced into `numSplits` partitions. + * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numSplits: Int): RDD[T] = new CoalescedRDD(this, numSplits) + def coalesce(numPartitions: Int): RDD[T] = new CoalescedRDD(this, numPartitions) /** * Return a sampled subset of this RDD. @@ -303,9 +303,9 @@ abstract class RDD[T: ClassManifest]( * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassManifest](f: T => K, numSplits: Int): RDD[(K, Seq[T])] = { + def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = { val cleanF = sc.clean(f) - this.map(t => (cleanF(t), t)).groupByKey(numSplits) + this.map(t => (cleanF(t), t)).groupByKey(numPartitions) } /** @@ -336,14 +336,24 @@ abstract class RDD[T: ClassManifest]( preservesPartitioning: Boolean = false): RDD[U] = new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning) - /** + /** + * Return a new RDD by applying a function to each partition of this RDD, while tracking the index + * of the original partition. + */ + def mapPartitionsWithIndex[U: ClassManifest]( + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = + new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) + + /** * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ + @deprecated("use mapPartitionsWithIndex") def mapPartitionsWithSplit[U: ClassManifest]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = - new MapPartitionsWithSplitRDD(this, sc.clean(f), preservesPartitioning) + new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) /** * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, @@ -471,7 +481,7 @@ abstract class RDD[T: ClassManifest]( } result } - val evaluator = new CountEvaluator(splits.size, confidence) + val evaluator = new CountEvaluator(partitions.size, confidence) sc.runApproximateJob(this, countElements, evaluator, timeout) } @@ -522,7 +532,7 @@ abstract class RDD[T: ClassManifest]( } map } - val evaluator = new GroupedCountEvaluator[T](splits.size, confidence) + val evaluator = new GroupedCountEvaluator[T](partitions.size, confidence) sc.runApproximateJob(this, countPartition, evaluator, timeout) } @@ -537,7 +547,7 @@ abstract class RDD[T: ClassManifest]( } val buf = new ArrayBuffer[T] var p = 0 - while (buf.size < num && p < splits.size) { + while (buf.size < num && p < partitions.size) { val left = num - buf.size val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, Array(p), true) buf ++= res(0) @@ -657,11 +667,11 @@ abstract class RDD[T: ClassManifest]( /** * Changes the dependencies of this RDD from its original parents to a new RDD (`newRDD`) - * created from the checkpoint file, and forget its old dependencies and splits. + * created from the checkpoint file, and forget its old dependencies and partitions. */ private[spark] def markCheckpointed(checkpointRDD: RDD[_]) { clearDependencies() - splits_ = null + partitions_ = null deps = null // Forget the constructor argument for dependencies too } @@ -676,15 +686,15 @@ abstract class RDD[T: ClassManifest]( } /** A description of this RDD and its recursive dependencies for debugging. */ - def toDebugString(): String = { + def toDebugString: String = { def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = { - Seq(prefix + rdd + " (" + rdd.splits.size + " splits)") ++ + Seq(prefix + rdd + " (" + rdd.partitions.size + " partitions)") ++ rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) } debugString(this).mkString("\n") } - override def toString(): String = "%s%s[%d] at %s".format( + override def toString: String = "%s%s[%d] at %s".format( Option(name).map(_ + " ").getOrElse(""), getClass.getSimpleName, id, diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index a4a4ebaf53af8..d00092e9845e2 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -16,7 +16,7 @@ private[spark] object CheckpointState extends Enumeration { /** * This class contains all the information related to RDD checkpointing. Each instance of this class * is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as, - * manages the post-checkpoint state by providing the updated splits, iterator and preferred locations + * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations * of the checkpointed RDD. */ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) @@ -67,11 +67,11 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path) _) val newRDD = new CheckpointRDD[T](rdd.context, path) - // Change the dependencies and splits of the RDD + // Change the dependencies and partitions of the RDD RDDCheckpointData.synchronized { cpFile = Some(path) cpRDD = Some(newRDD) - rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and splits + rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions cpState = Checkpointed RDDCheckpointData.clearTaskCaches() logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id) @@ -79,15 +79,15 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) } // Get preferred location of a split after checkpointing - def getPreferredLocations(split: Split): Seq[String] = { + def getPreferredLocations(split: Partition): Seq[String] = { RDDCheckpointData.synchronized { cpRDD.get.preferredLocations(split) } } - def getSplits: Array[Split] = { + def getPartitions: Array[Partition] = { RDDCheckpointData.synchronized { - cpRDD.get.splits + cpRDD.get.partitions } } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 047b57dc1f18e..f299b7ea46b76 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -614,14 +614,14 @@ class SparkContext( * Run a job on all partitions in an RDD and return the results in an array. */ def runJob[T, U: ClassManifest](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = { - runJob(rdd, func, 0 until rdd.splits.size, false) + runJob(rdd, func, 0 until rdd.partitions.size, false) } /** * Run a job on all partitions in an RDD and return the results in an array. */ def runJob[T, U: ClassManifest](rdd: RDD[T], func: Iterator[T] => U): Array[U] = { - runJob(rdd, func, 0 until rdd.splits.size, false) + runJob(rdd, func, 0 until rdd.partitions.size, false) } /** @@ -632,7 +632,7 @@ class SparkContext( processPartition: (TaskContext, Iterator[T]) => U, resultHandler: (Int, U) => Unit) { - runJob[T, U](rdd, processPartition, 0 until rdd.splits.size, false, resultHandler) + runJob[T, U](rdd, processPartition, 0 until rdd.partitions.size, false, resultHandler) } /** @@ -644,7 +644,7 @@ class SparkContext( resultHandler: (Int, U) => Unit) { val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter) - runJob[T, U](rdd, processFunc, 0 until rdd.splits.size, false, resultHandler) + runJob[T, U](rdd, processFunc, 0 until rdd.partitions.size, false, resultHandler) } /** @@ -696,7 +696,7 @@ class SparkContext( /** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */ def defaultParallelism: Int = taskScheduler.defaultParallelism - /** Default min number of splits for Hadoop RDDs when not given by user */ + /** Default min number of partitions for Hadoop RDDs when not given by user */ def defaultMinSplits: Int = math.min(defaultParallelism, 2) private var nextShuffleId = new AtomicInteger(0) diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 2810631b4158a..da3cb2cd31395 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -44,7 +44,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numSplits)) + def distinct(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numPartitions)) /** * Return a new RDD containing only the elements that satisfy a predicate. @@ -53,9 +53,9 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav fromRDD(srdd.filter(x => f(x).booleanValue())) /** - * Return a new RDD that is reduced into `numSplits` partitions. + * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numSplits)) + def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions)) /** * Return a sampled subset of this RDD. diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 55dc755358ffa..df3af3817dc81 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -54,7 +54,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numSplits: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numSplits)) + def distinct(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numPartitions)) /** * Return a new RDD containing only the elements that satisfy a predicate. @@ -63,9 +63,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue())) /** - * Return a new RDD that is reduced into `numSplits` partitions. + * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numSplits: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.coalesce(numSplits)) + def coalesce(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.coalesce(numPartitions)) /** * Return a sampled subset of this RDD. @@ -122,8 +122,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C], - numSplits: Int): JavaPairRDD[K, C] = - combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numSplits)) + numPartitions: Int): JavaPairRDD[K, C] = + combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) /** * Merge the values for each key using an associative reduce function. This will also perform @@ -162,10 +162,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numSplits splits. + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. */ - def reduceByKey(func: JFunction2[V, V, V], numSplits: Int): JavaPairRDD[K, V] = - fromRDD(rdd.reduceByKey(func, numSplits)) + def reduceByKey(func: JFunction2[V, V, V], numPartitions: Int): JavaPairRDD[K, V] = + fromRDD(rdd.reduceByKey(func, numPartitions)) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the @@ -176,10 +176,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with into `numSplits` partitions. + * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numSplits: Int): JavaPairRDD[K, JList[V]] = - fromRDD(groupByResultToJava(rdd.groupByKey(numSplits))) + def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = + fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) /** * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine` @@ -261,8 +261,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (V, W)] = - fromRDD(rdd.join(other, numSplits)) + def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = + fromRDD(rdd.join(other, numPartitions)) /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -277,10 +277,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * into `numSplits` partitions. + * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (V, Option[W])] = - fromRDD(rdd.leftOuterJoin(other, numSplits)) + def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Option[W])] = + fromRDD(rdd.leftOuterJoin(other, numPartitions)) /** * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the @@ -297,8 +297,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (Option[V], W)] = - fromRDD(rdd.rightOuterJoin(other, numSplits)) + def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Option[V], W)] = + fromRDD(rdd.rightOuterJoin(other, numPartitions)) /** * Return the key-value pairs in this RDD to the master as a Map. @@ -362,16 +362,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (JList[V], JList[W])] - = fromRDD(cogroupResultToJava(rdd.cogroup(other, numSplits))) + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] + = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numSplits: Int) + def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = - fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numSplits))) + fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) /** Alias for cogroup. */ def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 23e7ae2726f64..3ccd6f055ebef 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -30,7 +30,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numSplits: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numSplits)) + def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions)) /** * Return a new RDD containing only the elements that satisfy a predicate. @@ -39,9 +39,9 @@ JavaRDDLike[T, JavaRDD[T]] { wrapRDD(rdd.filter((x => f(x).booleanValue()))) /** - * Return a new RDD that is reduced into `numSplits` partitions. + * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numSplits: Int): JavaRDD[T] = rdd.coalesce(numSplits) + def coalesce(numPartitions: Int): JavaRDD[T] = rdd.coalesce(numPartitions) /** * Return a sampled subset of this RDD. diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index d34d56d169f5f..90b45cf875ea7 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -4,7 +4,7 @@ import java.util.{List => JList} import scala.Tuple2 import scala.collection.JavaConversions._ -import spark.{SparkContext, Split, RDD, TaskContext} +import spark.{SparkContext, Partition, RDD, TaskContext} import spark.api.java.JavaPairRDD._ import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import spark.partial.{PartialResult, BoundedDouble} @@ -20,7 +20,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround def rdd: RDD[T] /** Set of partitions in this RDD. */ - def splits: JList[Split] = new java.util.ArrayList(rdd.splits.toSeq) + def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) /** The [[spark.SparkContext]] that this RDD was created on. */ def context: SparkContext = rdd.context @@ -36,7 +36,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround * This should ''not'' be called by users directly, but is available for implementors of custom * subclasses of RDD. */ - def iterator(split: Split, taskContext: TaskContext): java.util.Iterator[T] = + def iterator(split: Partition, taskContext: TaskContext): java.util.Iterator[T] = asJavaIterator(rdd.iterator(split, taskContext)) // Transformations (return a new RDD) @@ -146,12 +146,12 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numSplits: Int): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] implicit val vcm: ClassManifest[JList[T]] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]] - JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numSplits)(f.returnType)))(kcm, vcm) + JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm) } /** @@ -333,6 +333,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround /** A description of this RDD and its recursive dependencies for debugging. */ def toDebugString(): String = { - rdd.toDebugString() + rdd.toDebugString } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index ab8351e55e9ef..8c734773847b5 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -32,11 +32,11 @@ private[spark] class PythonRDD[T: ClassManifest]( this(parent, PipedRDD.tokenize(command), envVars, preservePartitoning, pythonExec, broadcastVars, accumulator) - override def getSplits = parent.splits + override def getPartitions = parent.partitions override val partitioner = if (preservePartitoning) parent.partitioner else None - override def compute(split: Split, context: TaskContext): Iterator[Array[Byte]] = { + override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME") val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/python/pyspark/worker.py")) @@ -65,7 +65,7 @@ private[spark] class PythonRDD[T: ClassManifest]( SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) val dOut = new DataOutputStream(proc.getOutputStream) - // Split index + // Partition index dOut.writeInt(split.index) // sparkFilesDir PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dOut) @@ -155,8 +155,8 @@ private class PythonException(msg: String) extends Exception(msg) */ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Array[Byte], Array[Byte])](prev) { - override def getSplits = prev.splits - override def compute(split: Split, context: TaskContext) = + override def getPartitions = prev.partitions + override def compute(split: Partition, context: TaskContext) = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (a, b) case x => throw new Exception("PairwiseRDD: unexpected value: " + x) diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala index 24b4909380c66..de2dce161a090 100644 --- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala @@ -20,7 +20,7 @@ private[spark] class ApproximateActionListener[T, U, R]( extends JobListener { val startTime = System.currentTimeMillis() - val totalTasks = rdd.splits.size + val totalTasks = rdd.partitions.size var finishedTasks = 0 var failure: Option[Exception] = None // Set if the job has failed (permanently) var resultObject: Option[PartialResult[R]] = None // Set if we've already returned a PartialResult diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 17989c5ce5d18..7348c4f15bad6 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,9 +1,9 @@ package spark.rdd import scala.collection.mutable.HashMap -import spark.{RDD, SparkContext, SparkEnv, Split, TaskContext} +import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} -private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split { +private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { val index = idx } @@ -18,14 +18,14 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St HashMap(blockIds.zip(locations):_*) } - override def getSplits: Array[Split] = (0 until blockIds.size).map(i => { - new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split] + override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { + new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] }).toArray - override def compute(split: Split, context: TaskContext): Iterator[T] = { + override def compute(split: Partition, context: TaskContext): Iterator[T] = { val blockManager = SparkEnv.get.blockManager - val blockId = split.asInstanceOf[BlockRDDSplit].blockId + val blockId = split.asInstanceOf[BlockRDDPartition].blockId blockManager.get(blockId) match { case Some(block) => block.asInstanceOf[Iterator[T]] case None => @@ -33,8 +33,8 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St } } - override def getPreferredLocations(split: Split): Seq[String] = - locations_(split.asInstanceOf[BlockRDDSplit].blockId) + override def getPreferredLocations(split: Partition): Seq[String] = + locations_(split.asInstanceOf[BlockRDDPartition].blockId) } diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 41cbbd0093dea..38600b8be4e54 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -5,22 +5,22 @@ import spark._ private[spark] -class CartesianSplit( +class CartesianPartition( idx: Int, @transient rdd1: RDD[_], @transient rdd2: RDD[_], s1Index: Int, s2Index: Int - ) extends Split { - var s1 = rdd1.splits(s1Index) - var s2 = rdd2.splits(s2Index) + ) extends Partition { + var s1 = rdd1.partitions(s1Index) + var s2 = rdd2.partitions(s2Index) override val index: Int = idx @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { // Update the reference to parent split at the time of task serialization - s1 = rdd1.splits(s1Index) - s2 = rdd2.splits(s2Index) + s1 = rdd1.partitions(s1Index) + s2 = rdd2.partitions(s2Index) oos.defaultWriteObject() } } @@ -33,35 +33,35 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( extends RDD[Pair[T, U]](sc, Nil) with Serializable { - val numSplitsInRdd2 = rdd2.splits.size + val numPartitionsInRdd2 = rdd2.partitions.size - override def getSplits: Array[Split] = { + override def getPartitions: Array[Partition] = { // create the cross product split - val array = new Array[Split](rdd1.splits.size * rdd2.splits.size) - for (s1 <- rdd1.splits; s2 <- rdd2.splits) { - val idx = s1.index * numSplitsInRdd2 + s2.index - array(idx) = new CartesianSplit(idx, rdd1, rdd2, s1.index, s2.index) + val array = new Array[Partition](rdd1.partitions.size * rdd2.partitions.size) + for (s1 <- rdd1.partitions; s2 <- rdd2.partitions) { + val idx = s1.index * numPartitionsInRdd2 + s2.index + array(idx) = new CartesianPartition(idx, rdd1, rdd2, s1.index, s2.index) } array } - override def getPreferredLocations(split: Split): Seq[String] = { - val currSplit = split.asInstanceOf[CartesianSplit] + override def getPreferredLocations(split: Partition): Seq[String] = { + val currSplit = split.asInstanceOf[CartesianPartition] rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2) } - override def compute(split: Split, context: TaskContext) = { - val currSplit = split.asInstanceOf[CartesianSplit] + override def compute(split: Partition, context: TaskContext) = { + val currSplit = split.asInstanceOf[CartesianPartition] for (x <- rdd1.iterator(currSplit.s1, context); y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) } override def getDependencies: Seq[Dependency[_]] = List( new NarrowDependency(rdd1) { - def getParents(id: Int): Seq[Int] = List(id / numSplitsInRdd2) + def getParents(id: Int): Seq[Int] = List(id / numPartitionsInRdd2) }, new NarrowDependency(rdd2) { - def getParents(id: Int): Seq[Int] = List(id % numSplitsInRdd2) + def getParents(id: Int): Seq[Int] = List(id % numPartitionsInRdd2) } ) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 3558d4673f216..36bfb0355e0e1 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -9,7 +9,7 @@ import org.apache.hadoop.fs.Path import java.io.{File, IOException, EOFException} import java.text.NumberFormat -private[spark] class CheckpointRDDSplit(val index: Int) extends Split {} +private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} /** * This RDD represents a RDD checkpoint file (similar to HadoopRDD). @@ -20,27 +20,27 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) - override def getSplits: Array[Split] = { + override def getPartitions: Array[Partition] = { val dirContents = fs.listStatus(new Path(checkpointPath)) val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted - val numSplits = splitFiles.size + val numPartitions = splitFiles.size if (!splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || - !splitFiles(numSplits-1).endsWith(CheckpointRDD.splitIdToFile(numSplits-1))) { + !splitFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1))) { throw new SparkException("Invalid checkpoint directory: " + checkpointPath) } - Array.tabulate(numSplits)(i => new CheckpointRDDSplit(i)) + Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i)) } checkpointData = Some(new RDDCheckpointData[T](this)) checkpointData.get.cpFile = Some(checkpointPath) - override def getPreferredLocations(split: Split): Seq[String] = { + override def getPreferredLocations(split: Partition): Seq[String] = { val status = fs.getFileStatus(new Path(checkpointPath)) val locations = fs.getFileBlockLocations(status, 0, status.getLen) locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost") } - override def compute(split: Split, context: TaskContext): Iterator[T] = { + override def compute(split: Partition, context: TaskContext): Iterator[T] = { val file = new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index)) CheckpointRDD.readFromFile(file, context) } @@ -107,7 +107,7 @@ private[spark] object CheckpointRDD extends Logging { deserializeStream.asIterator.asInstanceOf[Iterator[T]] } - // Test whether CheckpointRDD generate expected number of splits despite + // Test whether CheckpointRDD generate expected number of partitions despite // each split file having multiple blocks. This needs to be run on a // cluster (mesos or standalone) using HDFS. def main(args: Array[String]) { @@ -120,8 +120,8 @@ private[spark] object CheckpointRDD extends Logging { val fs = path.getFileSystem(new Configuration()) sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) - assert(cpRDD.splits.length == rdd.splits.length, "Number of splits is not the same") - assert(cpRDD.collect.toList == rdd.collect.toList, "Data of splits not the same") + assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") + assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same") fs.delete(path) } } diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 868ee5a39f011..5200fb6b656ad 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -5,7 +5,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Split, TaskContext} +import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Partition, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -14,13 +14,13 @@ private[spark] sealed trait CoGroupSplitDep extends Serializable private[spark] case class NarrowCoGroupSplitDep( rdd: RDD[_], splitIndex: Int, - var split: Split + var split: Partition ) extends CoGroupSplitDep { @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { // Update the reference to parent split at the time of task serialization - split = rdd.splits(splitIndex) + split = rdd.partitions(splitIndex) oos.defaultWriteObject() } } @@ -28,7 +28,7 @@ private[spark] case class NarrowCoGroupSplitDep( private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep private[spark] -class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Split with Serializable { +class CoGroupPartition(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Partition with Serializable { override val index: Int = idx override def hashCode(): Int = idx } @@ -58,17 +58,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) } } - override def getSplits: Array[Split] = { - val array = new Array[Split](part.numPartitions) + override def getPartitions: Array[Partition] = { + val array = new Array[Partition](part.numPartitions) for (i <- 0 until array.size) { - // Each CoGroupSplit will have a dependency per contributing RDD - array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (rdd, j) => + // Each CoGroupPartition will have a dependency per contributing RDD + array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => // Assume each RDD contributed a single dependency, and get it dependencies(j) match { case s: ShuffleDependency[_, _] => new ShuffleCoGroupSplitDep(s.shuffleId) case _ => - new NarrowCoGroupSplitDep(rdd, i, rdd.splits(i)) + new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } }.toList) } @@ -77,8 +77,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) override val partitioner = Some(part) - override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { - val split = s.asInstanceOf[CoGroupSplit] + override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { + val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) val map = new JHashMap[K, Seq[ArrayBuffer[Any]]] diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index fcd26da43abe5..0d16cf6e85459 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -1,19 +1,19 @@ package spark.rdd -import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Split, TaskContext} +import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} -private[spark] case class CoalescedRDDSplit( +private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int] - ) extends Split { - var parents: Seq[Split] = parentsIndices.map(rdd.splits(_)) + ) extends Partition { + var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { // Update the reference to parent split at the time of task serialization - parents = parentsIndices.map(rdd.splits(_)) + parents = parentsIndices.map(rdd.partitions(_)) oos.defaultWriteObject() } } @@ -31,21 +31,21 @@ class CoalescedRDD[T: ClassManifest]( maxPartitions: Int) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies - override def getSplits: Array[Split] = { - val prevSplits = prev.splits + override def getPartitions: Array[Partition] = { + val prevSplits = prev.partitions if (prevSplits.length < maxPartitions) { - prevSplits.map(_.index).map{idx => new CoalescedRDDSplit(idx, prev, Array(idx)) } + prevSplits.map(_.index).map{idx => new CoalescedRDDPartition(idx, prev, Array(idx)) } } else { (0 until maxPartitions).map { i => val rangeStart = (i * prevSplits.length) / maxPartitions val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions - new CoalescedRDDSplit(i, prev, (rangeStart until rangeEnd).toArray) + new CoalescedRDDPartition(i, prev, (rangeStart until rangeEnd).toArray) }.toArray } } - override def compute(split: Split, context: TaskContext): Iterator[T] = { - split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { parentSplit => + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + split.asInstanceOf[CoalescedRDDPartition].parents.iterator.flatMap { parentSplit => firstParent[T].iterator(parentSplit, context) } } @@ -53,7 +53,7 @@ class CoalescedRDD[T: ClassManifest]( override def getDependencies: Seq[Dependency[_]] = { Seq(new NarrowDependency(prev) { def getParents(id: Int): Seq[Int] = - splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices + partitions(id).asInstanceOf[CoalescedRDDPartition].parentsIndices }) } diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index 93e398ea2b34a..c84ec39d21ff7 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -1,16 +1,16 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, Split, TaskContext} +import spark.{OneToOneDependency, RDD, Partition, TaskContext} private[spark] class FilteredRDD[T: ClassManifest]( prev: RDD[T], f: T => Boolean) extends RDD[T](prev) { - override def getSplits: Array[Split] = firstParent[T].splits + override def getPartitions: Array[Partition] = firstParent[T].partitions override val partitioner = prev.partitioner // Since filter cannot change a partition's keys - override def compute(split: Split, context: TaskContext) = + override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator(split, context).filter(f) } diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 8c2a610593306..8ebc77892514c 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -1,6 +1,6 @@ package spark.rdd -import spark.{RDD, Split, TaskContext} +import spark.{RDD, Partition, TaskContext} private[spark] @@ -9,8 +9,8 @@ class FlatMappedRDD[U: ClassManifest, T: ClassManifest]( f: T => TraversableOnce[U]) extends RDD[U](prev) { - override def getSplits: Array[Split] = firstParent[T].splits + override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Split, context: TaskContext) = + override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator(split, context).flatMap(f) } diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index 70b9b4e34ed49..e16c7ba881977 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -1,12 +1,12 @@ package spark.rdd -import spark.{RDD, Split, TaskContext} +import spark.{RDD, Partition, TaskContext} private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev) { - override def getSplits: Array[Split] = firstParent[T].splits + override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Split, context: TaskContext) = + override def compute(split: Partition, context: TaskContext) = Array(firstParent[T].iterator(split, context).toArray).iterator } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 854993737bc47..8139a2a40c66f 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -15,14 +15,14 @@ import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils -import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext} +import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext} /** * A Spark split class that wraps around a Hadoop InputSplit. */ -private[spark] class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit) - extends Split { +private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSplit) + extends Partition { val inputSplit = new SerializableWritable[InputSplit](s) @@ -47,12 +47,12 @@ class HadoopRDD[K, V]( // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) - override def getSplits: Array[Split] = { + override def getPartitions: Array[Partition] = { val inputFormat = createInputFormat(conf) val inputSplits = inputFormat.getSplits(conf, minSplits) - val array = new Array[Split](inputSplits.size) + val array = new Array[Partition](inputSplits.size) for (i <- 0 until inputSplits.size) { - array(i) = new HadoopSplit(id, i, inputSplits(i)) + array(i) = new HadoopPartition(id, i, inputSplits(i)) } array } @@ -62,8 +62,8 @@ class HadoopRDD[K, V]( .asInstanceOf[InputFormat[K, V]] } - override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] { - val split = theSplit.asInstanceOf[HadoopSplit] + override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] { + val split = theSplit.asInstanceOf[HadoopPartition] var reader: RecordReader[K, V] = null val conf = confBroadcast.value.value @@ -106,9 +106,9 @@ class HadoopRDD[K, V]( } } - override def getPreferredLocations(split: Split): Seq[String] = { + override def getPreferredLocations(split: Partition): Seq[String] = { // TODO: Filtering out "localhost" in case of file:// URLs - val hadoopSplit = split.asInstanceOf[HadoopSplit] + val hadoopSplit = split.asInstanceOf[HadoopPartition] hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost") } diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index 7b0b4525c7e62..d283c5b2bb8df 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -1,6 +1,6 @@ package spark.rdd -import spark.{RDD, Split, TaskContext} +import spark.{RDD, Partition, TaskContext} private[spark] @@ -13,8 +13,8 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None - override def getSplits: Array[Split] = firstParent[T].splits + override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Split, context: TaskContext) = + override def compute(split: Partition, context: TaskContext) = f(firstParent[T].iterator(split, context)) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala similarity index 57% rename from core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala rename to core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala index c6dc1080a9089..afb7504ba120d 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -1,24 +1,24 @@ package spark.rdd -import spark.{RDD, Split, TaskContext} +import spark.{RDD, Partition, TaskContext} /** - * A variant of the MapPartitionsRDD that passes the split index into the + * A variant of the MapPartitionsRDD that passes the partition index into the * closure. This can be used to generate or collect partition specific * information such as the number of tuples in a partition. */ private[spark] -class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest]( prev: RDD[T], f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean ) extends RDD[U](prev) { - override def getSplits: Array[Split] = firstParent[T].splits + override def getPartitions: Array[Partition] = firstParent[T].partitions override val partitioner = if (preservesPartitioning) prev.partitioner else None - override def compute(split: Split, context: TaskContext) = + override def compute(split: Partition, context: TaskContext) = f(split.index, firstParent[T].iterator(split, context)) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index 6074f411e3b48..af07311b6d038 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -1,13 +1,13 @@ package spark.rdd -import spark.{RDD, Split, TaskContext} +import spark.{RDD, Partition, TaskContext} private[spark] class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) extends RDD[U](prev) { - override def getSplits: Array[Split] = firstParent[T].splits + override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Split, context: TaskContext) = + override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator(split, context).map(f) } diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 345ae79d74d0d..ebd4c3f0e2d86 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -7,12 +7,12 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext} +import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext} private[spark] -class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) - extends Split { +class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) + extends Partition { val serializableHadoopSplit = new SerializableWritable(rawSplit) @@ -39,19 +39,19 @@ class NewHadoopRDD[K, V]( @transient private val jobId = new JobID(jobtrackerId, id) - override def getSplits: Array[Split] = { + override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray - val result = new Array[Split](rawSplits.size) + val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) { - result(i) = new NewHadoopSplit(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) } result } - override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] { - val split = theSplit.asInstanceOf[NewHadoopSplit] + override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] { + val split = theSplit.asInstanceOf[NewHadoopPartition] val conf = confBroadcast.value.value val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) @@ -83,8 +83,8 @@ class NewHadoopRDD[K, V]( } } - override def getPreferredLocations(split: Split): Seq[String] = { - val theSplit = split.asInstanceOf[NewHadoopSplit] + override def getPreferredLocations(split: Partition): Seq[String] = { + val theSplit = split.asInstanceOf[NewHadoopPartition] theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") } } diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index e703794787d4f..07585a88ceb36 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -3,20 +3,20 @@ package spark.rdd import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer import scala.collection.Map -import spark.{RDD, TaskContext, SparkContext, Split} +import spark.{RDD, TaskContext, SparkContext, Partition} -private[spark] class ParallelCollectionSplit[T: ClassManifest]( +private[spark] class ParallelCollectionPartition[T: ClassManifest]( val rddId: Long, val slice: Int, values: Seq[T]) - extends Split with Serializable { + extends Partition with Serializable { def iterator: Iterator[T] = values.iterator override def hashCode(): Int = (41 * (41 + rddId) + slice).toInt override def equals(other: Any): Boolean = other match { - case that: ParallelCollectionSplit[_] => (this.rddId == that.rddId && this.slice == that.slice) + case that: ParallelCollectionPartition[_] => (this.rddId == that.rddId && this.slice == that.slice) case _ => false } @@ -34,15 +34,15 @@ private[spark] class ParallelCollectionRDD[T: ClassManifest]( // instead. // UPDATE: A parallel collection can be checkpointed to HDFS, which achieves this goal. - override def getSplits: Array[Split] = { + override def getPartitions: Array[Partition] = { val slices = ParallelCollectionRDD.slice(data, numSlices).toArray - slices.indices.map(i => new ParallelCollectionSplit(id, i, slices(i))).toArray + slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray } - override def compute(s: Split, context: TaskContext) = - s.asInstanceOf[ParallelCollectionSplit[T]].iterator + override def compute(s: Partition, context: TaskContext) = + s.asInstanceOf[ParallelCollectionPartition[T]].iterator - override def getPreferredLocations(s: Split): Seq[String] = { + override def getPreferredLocations(s: Partition): Seq[String] = { locationPrefs.getOrElse(s.index, Nil) } } diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index d1553181c166d..f2f4fd56d1349 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -1,9 +1,9 @@ package spark.rdd -import spark.{NarrowDependency, RDD, SparkEnv, Split, TaskContext} +import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} -class PartitionPruningRDDSplit(idx: Int, val parentSplit: Split) extends Split { +class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { override val index = idx } @@ -16,15 +16,15 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Split] = rdd.splits.filter(s => partitionFilterFunc(s.index)) - .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDSplit(idx, split) : Split } + val partitions: Array[Partition] = rdd.partitions.filter(s => partitionFilterFunc(s.index)) + .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } /** - * A RDD used to prune RDD partitions/splits so we can avoid launching tasks on + * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. @@ -34,9 +34,9 @@ class PartitionPruningRDD[T: ClassManifest]( @transient partitionFilterFunc: Int => Boolean) extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { - override def compute(split: Split, context: TaskContext) = firstParent[T].iterator( - split.asInstanceOf[PartitionPruningRDDSplit].parentSplit, context) + override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator( + split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context) - override protected def getSplits: Array[Split] = + override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions } diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 56032a8659a05..962a1b21ad1d3 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -8,7 +8,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.io.Source -import spark.{RDD, SparkEnv, Split, TaskContext} +import spark.{RDD, SparkEnv, Partition, TaskContext} /** @@ -27,9 +27,9 @@ class PipedRDD[T: ClassManifest]( // using a standard StringTokenizer (i.e. by spaces) def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command)) - override def getSplits: Array[Split] = firstParent[T].splits + override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Split, context: TaskContext): Iterator[String] = { + override def compute(split: Partition, context: TaskContext): Iterator[String] = { val pb = new ProcessBuilder(command) // Add the environmental variables to the process. val currentEnvVars = pb.environment() diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index f2a144e2e025f..243673f151872 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -5,10 +5,10 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import spark.{RDD, Split, TaskContext} +import spark.{RDD, Partition, TaskContext} private[spark] -class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable { +class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable { override val index: Int = prev.index } @@ -19,16 +19,16 @@ class SampledRDD[T: ClassManifest]( seed: Int) extends RDD[T](prev) { - override def getSplits: Array[Split] = { + override def getPartitions: Array[Partition] = { val rg = new Random(seed) - firstParent[T].splits.map(x => new SampledRDDSplit(x, rg.nextInt)) + firstParent[T].partitions.map(x => new SampledRDDPartition(x, rg.nextInt)) } - override def getPreferredLocations(split: Split): Seq[String] = - firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev) + override def getPreferredLocations(split: Partition): Seq[String] = + firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDPartition].prev) - override def compute(splitIn: Split, context: TaskContext): Iterator[T] = { - val split = splitIn.asInstanceOf[SampledRDDSplit] + override def compute(splitIn: Partition, context: TaskContext): Iterator[T] = { + val split = splitIn.asInstanceOf[SampledRDDPartition] if (withReplacement) { // For large datasets, the expected number of occurrences of each element in a sample with // replacement is Poisson(frac). We use that to get a count for each element. diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index bf69b5150bcc4..c2f118305f33f 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -1,9 +1,9 @@ package spark.rdd -import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext} +import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} import spark.SparkContext._ -private[spark] class ShuffledRDDSplit(val idx: Int) extends Split { +private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx override def hashCode(): Int = idx } @@ -22,11 +22,11 @@ class ShuffledRDD[K, V]( override val partitioner = Some(part) - override def getSplits: Array[Split] = { - Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i)) + override def getPartitions: Array[Partition] = { + Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i)) } - override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = { + override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index) } diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index ebc006822804f..2c52a67e22635 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -1,13 +1,13 @@ package spark.rdd import scala.collection.mutable.ArrayBuffer -import spark.{Dependency, RangeDependency, RDD, SparkContext, Split, TaskContext} +import spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} -private[spark] class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int) - extends Split { +private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int) + extends Partition { - var split: Split = rdd.splits(splitIndex) + var split: Partition = rdd.partitions(splitIndex) def iterator(context: TaskContext) = rdd.iterator(split, context) @@ -18,7 +18,7 @@ private[spark] class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], splitIn @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { // Update the reference to parent split at the time of task serialization - split = rdd.splits(splitIndex) + split = rdd.partitions(splitIndex) oos.defaultWriteObject() } } @@ -28,11 +28,11 @@ class UnionRDD[T: ClassManifest]( @transient var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies - override def getSplits: Array[Split] = { - val array = new Array[Split](rdds.map(_.splits.size).sum) + override def getPartitions: Array[Partition] = { + val array = new Array[Partition](rdds.map(_.partitions.size).sum) var pos = 0 - for (rdd <- rdds; split <- rdd.splits) { - array(pos) = new UnionSplit(pos, rdd, split.index) + for (rdd <- rdds; split <- rdd.partitions) { + array(pos) = new UnionPartition(pos, rdd, split.index) pos += 1 } array @@ -42,15 +42,15 @@ class UnionRDD[T: ClassManifest]( val deps = new ArrayBuffer[Dependency[_]] var pos = 0 for (rdd <- rdds) { - deps += new RangeDependency(rdd, 0, pos, rdd.splits.size) - pos += rdd.splits.size + deps += new RangeDependency(rdd, 0, pos, rdd.partitions.size) + pos += rdd.partitions.size } deps } - override def compute(s: Split, context: TaskContext): Iterator[T] = - s.asInstanceOf[UnionSplit[T]].iterator(context) + override def compute(s: Partition, context: TaskContext): Iterator[T] = + s.asInstanceOf[UnionPartition[T]].iterator(context) - override def getPreferredLocations(s: Split): Seq[String] = - s.asInstanceOf[UnionSplit[T]].preferredLocations() + override def getPreferredLocations(s: Partition): Seq[String] = + s.asInstanceOf[UnionPartition[T]].preferredLocations() } diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 1ce70268bb491..e80ec17aa5e0a 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,17 +1,17 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkContext, Split, TaskContext} +import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} -private[spark] class ZippedSplit[T: ClassManifest, U: ClassManifest]( +private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest]( idx: Int, @transient rdd1: RDD[T], @transient rdd2: RDD[U] - ) extends Split { + ) extends Partition { - var split1 = rdd1.splits(idx) - var split2 = rdd1.splits(idx) + var split1 = rdd1.partitions(idx) + var split2 = rdd1.partitions(idx) override val index: Int = idx def splits = (split1, split2) @@ -19,8 +19,8 @@ private[spark] class ZippedSplit[T: ClassManifest, U: ClassManifest]( @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { // Update the reference to parent split at the time of task serialization - split1 = rdd1.splits(idx) - split2 = rdd2.splits(idx) + split1 = rdd1.partitions(idx) + split2 = rdd2.partitions(idx) oos.defaultWriteObject() } } @@ -31,24 +31,24 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( var rdd2: RDD[U]) extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) { - override def getSplits: Array[Split] = { - if (rdd1.splits.size != rdd2.splits.size) { + override def getPartitions: Array[Partition] = { + if (rdd1.partitions.size != rdd2.partitions.size) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } - val array = new Array[Split](rdd1.splits.size) - for (i <- 0 until rdd1.splits.size) { - array(i) = new ZippedSplit(i, rdd1, rdd2) + val array = new Array[Partition](rdd1.partitions.size) + for (i <- 0 until rdd1.partitions.size) { + array(i) = new ZippedPartition(i, rdd1, rdd2) } array } - override def compute(s: Split, context: TaskContext): Iterator[(T, U)] = { - val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits + override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = { + val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context)) } - override def getPreferredLocations(s: Split): Seq[String] = { - val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits + override def getPreferredLocations(s: Partition): Seq[String] = { + val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2)) } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 319eef69780ed..bf0837c0660cb 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -106,7 +106,7 @@ class DAGScheduler( private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { if (!cacheLocs.contains(rdd.id)) { - val blockIds = rdd.splits.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray + val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map { locations => locations.map(_.ip).toList }.toArray @@ -141,9 +141,9 @@ class DAGScheduler( private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { if (shuffleDep != None) { // Kind of ugly: need to register RDDs with the cache and map output tracker here - // since we can't do it in the RDD constructor because # of splits is unknown + // since we can't do it in the RDD constructor because # of partitions is unknown logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") - mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.splits.size) + mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) } val id = nextStageId.getAndIncrement() val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) @@ -162,7 +162,7 @@ class DAGScheduler( if (!visited(r)) { visited += r // Kind of ugly: need to register RDDs with the cache here since - // we can't do it in its constructor because # of splits is unknown + // we can't do it in its constructor because # of partitions is unknown for (dep <- r.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => @@ -257,7 +257,7 @@ class DAGScheduler( { val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val partitions = (0 until rdd.splits.size).toArray + val partitions = (0 until rdd.partitions.size).toArray eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener)) return listener.awaitResult() // Will throw an exception if the job fails } @@ -386,7 +386,7 @@ class DAGScheduler( try { SparkEnv.set(env) val rdd = job.finalStage.rdd - val split = rdd.splits(job.partitions(0)) + val split = rdd.partitions(job.partitions(0)) val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) @@ -672,7 +672,7 @@ class DAGScheduler( return cached } // If the RDD has some placement preferences (as is the case for input RDDs), get those - val rddPrefs = rdd.preferredLocations(rdd.splits(partition)).toList + val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList if (rddPrefs != Nil) { return rddPrefs } diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 8cd4c661eb70d..1721f78f483cf 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -67,7 +67,7 @@ private[spark] class ResultTask[T, U]( var split = if (rdd == null) { null } else { - rdd.splits(partition) + rdd.partitions(partition) } override def run(attemptId: Long): U = { @@ -85,7 +85,7 @@ private[spark] class ResultTask[T, U]( override def writeExternal(out: ObjectOutput) { RDDCheckpointData.synchronized { - split = rdd.splits(partition) + split = rdd.partitions(partition) out.writeInt(stageId) val bytes = ResultTask.serializeInfo( stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _]) @@ -107,6 +107,6 @@ private[spark] class ResultTask[T, U]( func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] partition = in.readInt() val outputId = in.readInt() - split = in.readObject().asInstanceOf[Split] + split = in.readObject().asInstanceOf[Partition] } } diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index bed9f1864fffd..59ee3c0a095ac 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -86,12 +86,12 @@ private[spark] class ShuffleMapTask( var split = if (rdd == null) { null } else { - rdd.splits(partition) + rdd.partitions(partition) } override def writeExternal(out: ObjectOutput) { RDDCheckpointData.synchronized { - split = rdd.splits(partition) + split = rdd.partitions(partition) out.writeInt(stageId) val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep) out.writeInt(bytes.length) @@ -112,7 +112,7 @@ private[spark] class ShuffleMapTask( dep = dep_ partition = in.readInt() generation = in.readLong() - split = in.readObject().asInstanceOf[Split] + split = in.readObject().asInstanceOf[Partition] } override def run(attemptId: Long): MapStatus = { diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 374114d87034c..552061e46b8c1 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -28,7 +28,7 @@ private[spark] class Stage( extends Logging { val isShuffleMap = shuffleDep != None - val numPartitions = rdd.splits.size + val numPartitions = rdd.partitions.size val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) var numAvailableOutputs = 0 diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 2e7db608414fe..2462721fb8443 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -513,7 +513,7 @@ class BlockManager( } } - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // Partition local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] for ((address, blockInfos) <- blocksByAddress) { diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 5f72b67b2cc27..dec47a9d4113b 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -63,7 +63,7 @@ object StorageUtils { val rddName = Option(rdd.name).getOrElse(rddKey) val rddStorageLevel = rdd.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.splits.size, memSize, diskSize) + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize) }.toArray } diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 51ff966ae4880..3e5ffa81d6f8f 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -34,7 +34,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { testCheckpointing(_.sample(false, 0.5, 0)) testCheckpointing(_.glom()) testCheckpointing(_.mapPartitions(_.map(_.toString))) - testCheckpointing(r => new MapPartitionsWithSplitRDD(r, + testCheckpointing(r => new MapPartitionsWithIndexRDD(r, (i: Int, iter: Iterator[Int]) => iter.map(_.toString), false )) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString)) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x)) @@ -43,14 +43,14 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("ParallelCollection") { val parCollection = sc.makeRDD(1 to 4, 2) - val numSplits = parCollection.splits.size + val numPartitions = parCollection.partitions.size parCollection.checkpoint() assert(parCollection.dependencies === Nil) val result = parCollection.collect() assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result) assert(parCollection.dependencies != Nil) - assert(parCollection.splits.length === numSplits) - assert(parCollection.splits.toList === parCollection.checkpointData.get.getSplits.toList) + assert(parCollection.partitions.length === numPartitions) + assert(parCollection.partitions.toList === parCollection.checkpointData.get.getPartitions.toList) assert(parCollection.collect() === result) } @@ -59,13 +59,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val blockManager = SparkEnv.get.blockManager blockManager.putSingle(blockId, "test", StorageLevel.MEMORY_ONLY) val blockRDD = new BlockRDD[String](sc, Array(blockId)) - val numSplits = blockRDD.splits.size + val numPartitions = blockRDD.partitions.size blockRDD.checkpoint() val result = blockRDD.collect() assert(sc.checkpointFile[String](blockRDD.getCheckpointFile.get).collect() === result) assert(blockRDD.dependencies != Nil) - assert(blockRDD.splits.length === numSplits) - assert(blockRDD.splits.toList === blockRDD.checkpointData.get.getSplits.toList) + assert(blockRDD.partitions.length === numPartitions) + assert(blockRDD.partitions.toList === blockRDD.checkpointData.get.getPartitions.toList) assert(blockRDD.collect() === result) } @@ -79,9 +79,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("UnionRDD") { def otherRDD = sc.makeRDD(1 to 10, 1) - // Test whether the size of UnionRDDSplits reduce in size after parent RDD is checkpointed. + // Test whether the size of UnionRDDPartitions reduce in size after parent RDD is checkpointed. // Current implementation of UnionRDD has transient reference to parent RDDs, - // so only the splits will reduce in serialized size, not the RDD. + // so only the partitions will reduce in serialized size, not the RDD. testCheckpointing(_.union(otherRDD), false, true) testParentCheckpointing(_.union(otherRDD), false, true) } @@ -91,21 +91,21 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { testCheckpointing(new CartesianRDD(sc, _, otherRDD)) // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed - // Current implementation of CoalescedRDDSplit has transient reference to parent RDD, - // so only the RDD will reduce in serialized size, not the splits. + // Current implementation of CoalescedRDDPartition has transient reference to parent RDD, + // so only the RDD will reduce in serialized size, not the partitions. testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false) - // Test that the CartesianRDD updates parent splits (CartesianRDD.s1/s2) after - // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits. + // Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after + // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions. // Note that this test is very specific to the current implementation of CartesianRDD. val ones = sc.makeRDD(1 to 100, 10).map(x => x) ones.checkpoint() // checkpoint that MappedRDD val cartesian = new CartesianRDD(sc, ones, ones) val splitBeforeCheckpoint = - serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit]) + serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition]) cartesian.count() // do the checkpointing val splitAfterCheckpoint = - serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit]) + serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition]) assert( (splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) && (splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2), @@ -117,24 +117,24 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { testCheckpointing(_.coalesce(2)) // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed - // Current implementation of CoalescedRDDSplit has transient reference to parent RDD, - // so only the RDD will reduce in serialized size, not the splits. + // Current implementation of CoalescedRDDPartition has transient reference to parent RDD, + // so only the RDD will reduce in serialized size, not the partitions. testParentCheckpointing(_.coalesce(2), true, false) - // Test that the CoalescedRDDSplit updates parent splits (CoalescedRDDSplit.parents) after - // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits. - // Note that this test is very specific to the current implementation of CoalescedRDDSplits + // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents) after + // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions. + // Note that this test is very specific to the current implementation of CoalescedRDDPartitions val ones = sc.makeRDD(1 to 100, 10).map(x => x) ones.checkpoint() // checkpoint that MappedRDD val coalesced = new CoalescedRDD(ones, 2) val splitBeforeCheckpoint = - serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit]) + serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition]) coalesced.count() // do the checkpointing val splitAfterCheckpoint = - serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit]) + serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition]) assert( splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head, - "CoalescedRDDSplit.parents not updated after parent RDD checkpointed" + "CoalescedRDDPartition.parents not updated after parent RDD checkpointed" ) } @@ -156,8 +156,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false) // Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed - // Current implementation of ZippedRDDSplit has transient references to parent RDDs, - // so only the RDD will reduce in serialized size, not the splits. + // Current implementation of ZippedRDDPartitions has transient references to parent RDDs, + // so only the RDD will reduce in serialized size, not the partitions. testParentCheckpointing( rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false) } @@ -165,21 +165,21 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { /** * Test checkpointing of the final RDD generated by the given operation. By default, * this method tests whether the size of serialized RDD has reduced after checkpointing or not. - * It can also test whether the size of serialized RDD splits has reduced after checkpointing or - * not, but this is not done by default as usually the splits do not refer to any RDD and + * It can also test whether the size of serialized RDD partitions has reduced after checkpointing or + * not, but this is not done by default as usually the partitions do not refer to any RDD and * therefore never store the lineage. */ def testCheckpointing[U: ClassManifest]( op: (RDD[Int]) => RDD[U], testRDDSize: Boolean = true, - testRDDSplitSize: Boolean = false + testRDDPartitionSize: Boolean = false ) { // Generate the final RDD using given RDD operation val baseRDD = generateLongLineageRDD() val operatedRDD = op(baseRDD) val parentRDD = operatedRDD.dependencies.headOption.orNull val rddType = operatedRDD.getClass.getSimpleName - val numSplits = operatedRDD.splits.length + val numPartitions = operatedRDD.partitions.length // Find serialized sizes before and after the checkpoint val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) @@ -193,11 +193,11 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { // Test whether dependencies have been changed from its earlier parent RDD assert(operatedRDD.dependencies.head.rdd != parentRDD) - // Test whether the splits have been changed to the new Hadoop splits - assert(operatedRDD.splits.toList === operatedRDD.checkpointData.get.getSplits.toList) + // Test whether the partitions have been changed to the new Hadoop partitions + assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList) - // Test whether the number of splits is same as before - assert(operatedRDD.splits.length === numSplits) + // Test whether the number of partitions is same as before + assert(operatedRDD.partitions.length === numPartitions) // Test whether the data in the checkpointed RDD is same as original assert(operatedRDD.collect() === result) @@ -215,18 +215,18 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { ) } - // Test whether serialized size of the splits has reduced. If the splits - // do not have any non-transient reference to another RDD or another RDD's splits, it + // Test whether serialized size of the partitions has reduced. If the partitions + // do not have any non-transient reference to another RDD or another RDD's partitions, it // does not refer to a lineage and therefore may not reduce in size after checkpointing. - // However, if the original splits before checkpointing do refer to a parent RDD, the splits + // However, if the original partitions before checkpointing do refer to a parent RDD, the partitions // must be forgotten after checkpointing (to remove all reference to parent RDDs) and - // replaced with the HadoopSplits of the checkpointed RDD. - if (testRDDSplitSize) { - logInfo("Size of " + rddType + " splits " + // replaced with the HadooPartitions of the checkpointed RDD. + if (testRDDPartitionSize) { + logInfo("Size of " + rddType + " partitions " + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]") assert( splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint, - "Size of " + rddType + " splits did not reduce after checkpointing " + + "Size of " + rddType + " partitions did not reduce after checkpointing " + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]" ) } @@ -235,13 +235,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { /** * Test whether checkpointing of the parent of the generated RDD also * truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent - * RDDs splits. So even if the parent RDD is checkpointed and its splits changed, - * this RDD will remember the splits and therefore potentially the whole lineage. + * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed, + * this RDD will remember the partitions and therefore potentially the whole lineage. */ def testParentCheckpointing[U: ClassManifest]( op: (RDD[Int]) => RDD[U], testRDDSize: Boolean, - testRDDSplitSize: Boolean + testRDDPartitionSize: Boolean ) { // Generate the final RDD using given RDD operation val baseRDD = generateLongLineageRDD() @@ -250,9 +250,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val rddType = operatedRDD.getClass.getSimpleName val parentRDDType = parentRDD.getClass.getSimpleName - // Get the splits and dependencies of the parent in case they're lazily computed + // Get the partitions and dependencies of the parent in case they're lazily computed parentRDD.dependencies - parentRDD.splits + parentRDD.partitions // Find serialized sizes before and after the checkpoint val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) @@ -275,16 +275,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { ) } - // Test whether serialized size of the splits has reduced because of its parent being - // checkpointed. If the splits do not have any non-transient reference to another RDD - // or another RDD's splits, it does not refer to a lineage and therefore may not reduce - // in size after checkpointing. However, if the splits do refer to the *splits* of a parent - // RDD, then these splits must update reference to the parent RDD splits as the parent RDD's - // splits must have changed after checkpointing. - if (testRDDSplitSize) { + // Test whether serialized size of the partitions has reduced because of its parent being + // checkpointed. If the partitions do not have any non-transient reference to another RDD + // or another RDD's partitions, it does not refer to a lineage and therefore may not reduce + // in size after checkpointing. However, if the partitions do refer to the *partitions* of a parent + // RDD, then these partitions must update reference to the parent RDD partitions as the parent RDD's + // partitions must have changed after checkpointing. + if (testRDDPartitionSize) { assert( splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint, - "Size of " + rddType + " splits did not reduce after checkpointing parent " + parentRDDType + + "Size of " + rddType + " partitions did not reduce after checkpointing parent " + parentRDDType + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]" ) } @@ -321,12 +321,12 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } /** - * Get serialized sizes of the RDD and its splits, in order to test whether the size shrinks + * Get serialized sizes of the RDD and its partitions, in order to test whether the size shrinks * upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint. */ def getSerializedSizes(rdd: RDD[_]): (Int, Int) = { (Utils.serialize(rdd).length - Utils.serialize(rdd.checkpointData).length, - Utils.serialize(rdd.splits).length) + Utils.serialize(rdd.partitions).length) } /** diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ffa866de7532f..9739ba869b318 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -33,6 +33,11 @@ class RDDSuite extends FunSuite with LocalSparkContext { } assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7))) + val partitionSumsWithIndex = nums.mapPartitionsWithIndex { + case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) + } + assert(partitionSumsWithIndex.collect().toList === List((0, 3), (1, 7))) + intercept[UnsupportedOperationException] { nums.filter(_ > 5).reduce(_ + _) } @@ -97,12 +102,12 @@ class RDDSuite extends FunSuite with LocalSparkContext { test("caching with failures") { sc = new SparkContext("local", "test") - val onlySplit = new Split { override def index: Int = 0 } + val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true val rdd = new RDD[Int](sc, Nil) { - override def getSplits: Array[Split] = Array(onlySplit) + override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() - override def compute(split: Split, context: TaskContext): Iterator[Int] = { + override def compute(split: Partition, context: TaskContext): Iterator[Int] = { if (shouldFail) { throw new Exception("injected failure") } else { @@ -168,7 +173,7 @@ class RDDSuite extends FunSuite with LocalSparkContext { val data = sc.parallelize(1 to 10, 10) // Note that split number starts from 0, so > 8 means only 10th partition left. val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8) - assert(prunedRdd.splits.size === 1) + assert(prunedRdd.partitions.size === 1) val prunedData = prunedRdd.collect() assert(prunedData.size === 1) assert(prunedData(0) === 10) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 50f2b294bfb05..92c3f6741668b 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -222,7 +222,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { sc = new SparkContext("local", "test") val emptyDir = Files.createTempDir() val file = sc.textFile(emptyDir.getAbsolutePath) - assert(file.splits.size == 0) + assert(file.partitions.size == 0) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index edb8c839fcb70..495f957e53f25 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -19,7 +19,7 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) val sorted = pairs.sortByKey() - assert(sorted.splits.size === 2) + assert(sorted.partitions.size === 2) assert(sorted.collect() === pairArr.sortBy(_._1)) } @@ -29,17 +29,17 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) val sorted = pairs.sortByKey(true, 1) - assert(sorted.splits.size === 1) + assert(sorted.partitions.size === 1) assert(sorted.collect() === pairArr.sortBy(_._1)) } - test("large array with many splits") { + test("large array with many partitions") { sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) val sorted = pairs.sortByKey(true, 20) - assert(sorted.splits.size === 20) + assert(sorted.partitions.size === 20) assert(sorted.collect() === pairArr.sortBy(_._1)) } @@ -59,7 +59,7 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) } - test("sort descending with many splits") { + test("sort descending with many partitions") { sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 83663ac702a5b..8de490eb86f34 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -24,7 +24,7 @@ import spark.MapOutputTracker import spark.RDD import spark.SparkContext import spark.SparkException -import spark.Split +import spark.Partition import spark.TaskContext import spark.TaskEndReason @@ -144,18 +144,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * so we can test that DAGScheduler does not try to execute RDDs locally. */ def makeRdd( - numSplits: Int, + numPartitions: Int, dependencies: List[Dependency[_]], locations: Seq[Seq[String]] = Nil ): MyRDD = { - val maxSplit = numSplits - 1 + val maxPartition = numPartitions - 1 return new MyRDD(sc, dependencies) { - override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] = + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new RuntimeException("should not be reached") - override def getSplits() = (0 to maxSplit).map(i => new Split { + override def getPartitions = (0 to maxPartition).map(i => new Partition { override def index = i }).toArray - override def getPreferredLocations(split: Split): Seq[String] = + override def getPreferredLocations(split: Partition): Seq[String] = if (locations.isDefinedAt(split.index)) locations(split.index) else @@ -295,11 +295,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar * collect the result of the job via callbacks from DAGScheduler. */ def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): (JobWaiter[Int], Array[Int]) = { - val resultArray = new Array[Int](rdd.splits.size) + val resultArray = new Array[Int](rdd.partitions.size) val (toSubmit, waiter) = scheduler.prepareJob[(Int, Int), Int]( rdd, jobComputeFunc, - (0 to (rdd.splits.size - 1)), + (0 to (rdd.partitions.size - 1)), "test-site", allowLocal, (i: Int, value: Int) => resultArray(i) = value @@ -355,10 +355,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("local job") { val rdd = new MyRDD(sc, Nil) { - override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] = + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = Array(42 -> 0).iterator - override def getSplits() = Array( new Split { override def index = 0 } ) - override def getPreferredLocations(split: Split) = Nil + override def getPartitions = Array( new Partition { override def index = 0 } ) + override def getPreferredLocations(split: Partition) = Nil override def toString = "DAGSchedulerSuite Local RDD" } submitRdd(rdd, true) diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index a5db7103f5ce8..647bcaf860a37 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -5,7 +5,7 @@ import org.scalatest.BeforeAndAfter import spark.TaskContext import spark.RDD import spark.SparkContext -import spark.Split +import spark.Partition import spark.LocalSparkContext class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { @@ -14,8 +14,8 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte var completed = false sc = new SparkContext("local", "test") val rdd = new RDD[String](sc, List()) { - override def getSplits = Array[Split](StubSplit(0)) - override def compute(split: Split, context: TaskContext) = { + override def getPartitions = Array[Partition](StubPartition(0)) + override def compute(split: Partition, context: TaskContext) = { context.addOnCompleteCallback(() => completed = true) sys.error("failed") } @@ -28,5 +28,5 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte assert(completed === true) } - case class StubSplit(val index: Int) extends Split -} \ No newline at end of file + case class StubPartition(val index: Int) extends Partition +} From 7151e1e4c8f4f764c54047ef82b988f887a0b9c7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 17 Feb 2013 23:23:08 -0800 Subject: [PATCH 236/696] Rename "jobs" to "applications" in the standalone cluster --- core/src/main/scala/spark/SparkContext.scala | 12 +- .../spark/api/java/JavaSparkContext.scala | 22 +-- .../spark/api/python/PythonPartitioner.scala | 2 +- ...ion.scala => ApplicationDescription.scala} | 4 +- .../scala/spark/deploy/DeployMessage.scala | 19 +- .../scala/spark/deploy/JsonProtocol.scala | 18 +- .../scala/spark/deploy/client/Client.scala | 22 +-- .../spark/deploy/client/ClientListener.scala | 2 +- .../spark/deploy/client/TestClient.scala | 6 +- .../{JobInfo.scala => ApplicationInfo.scala} | 10 +- .../deploy/master/ApplicationState.scala | 11 ++ .../spark/deploy/master/ExecutorInfo.scala | 4 +- .../scala/spark/deploy/master/JobState.scala | 9 - .../scala/spark/deploy/master/Master.scala | 174 +++++++++--------- .../spark/deploy/master/MasterWebUI.scala | 22 +-- .../spark/deploy/master/WorkerInfo.scala | 4 +- .../spark/deploy/worker/ExecutorRunner.scala | 26 +-- .../scala/spark/deploy/worker/Worker.scala | 20 +- .../spark/deploy/worker/WorkerArguments.scala | 2 +- .../spark/deploy/worker/WorkerWebUI.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 15 +- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../mesos/MesosSchedulerBackend.scala | 4 +- .../deploy/master/app_details.scala.html | 40 ++++ .../spark/deploy/master/app_row.scala.html | 20 ++ ..._table.scala.html => app_table.scala.html} | 8 +- .../deploy/master/executor_row.scala.html | 6 +- .../spark/deploy/master/index.scala.html | 16 +- .../deploy/master/job_details.scala.html | 40 ---- .../spark/deploy/master/job_row.scala.html | 20 -- .../deploy/worker/executor_row.scala.html | 10 +- .../scala/spark/streaming/Checkpoint.scala | 2 +- .../spark/streaming/StreamingContext.scala | 10 +- .../api/java/JavaStreamingContext.scala | 6 +- 34 files changed, 299 insertions(+), 295 deletions(-) rename core/src/main/scala/spark/deploy/{JobDescription.scala => ApplicationDescription.scala} (66%) rename core/src/main/scala/spark/deploy/master/{JobInfo.scala => ApplicationInfo.scala} (84%) create mode 100644 core/src/main/scala/spark/deploy/master/ApplicationState.scala delete mode 100644 core/src/main/scala/spark/deploy/master/JobState.scala create mode 100644 core/src/main/twirl/spark/deploy/master/app_details.scala.html create mode 100644 core/src/main/twirl/spark/deploy/master/app_row.scala.html rename core/src/main/twirl/spark/deploy/master/{job_table.scala.html => app_table.scala.html} (74%) delete mode 100644 core/src/main/twirl/spark/deploy/master/job_details.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/job_row.scala.html diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index f299b7ea46b76..d39767c3b3ce3 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -53,7 +53,7 @@ import storage.{StorageStatus, StorageUtils, RDDInfo} * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param jobName A name for your job, to display on the cluster web UI. + * @param appName A name for your application, to display on the cluster web UI. * @param sparkHome Location where Spark is installed on cluster nodes. * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. @@ -61,7 +61,7 @@ import storage.{StorageStatus, StorageUtils, RDDInfo} */ class SparkContext( val master: String, - val jobName: String, + val appName: String, val sparkHome: String = null, val jars: Seq[String] = Nil, environment: Map[String, String] = Map()) @@ -143,7 +143,7 @@ class SparkContext( case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) - val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName) + val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) scheduler.initialize(backend) scheduler @@ -162,7 +162,7 @@ class SparkContext( val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName) + val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() @@ -178,9 +178,9 @@ class SparkContext( val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean val masterWithoutProtocol = master.replaceFirst("^mesos://", "") // Strip initial mesos:// val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, this, masterWithoutProtocol, jobName) + new CoarseMesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName) } else { - new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, jobName) + new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName) } scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index 50b8970cd8aad..f75fc27c7b2f6 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -23,41 +23,41 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param jobName A name for your job, to display on the cluster web UI + * @param appName A name for your application, to display on the cluster web UI */ - def this(master: String, jobName: String) = this(new SparkContext(master, jobName)) + def this(master: String, appName: String) = this(new SparkContext(master, appName)) /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param jobName A name for your job, to display on the cluster web UI + * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the slave nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. */ - def this(master: String, jobName: String, sparkHome: String, jarFile: String) = - this(new SparkContext(master, jobName, sparkHome, Seq(jarFile))) + def this(master: String, appName: String, sparkHome: String, jarFile: String) = + this(new SparkContext(master, appName, sparkHome, Seq(jarFile))) /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param jobName A name for your job, to display on the cluster web UI + * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the slave nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. */ - def this(master: String, jobName: String, sparkHome: String, jars: Array[String]) = - this(new SparkContext(master, jobName, sparkHome, jars.toSeq)) + def this(master: String, appName: String, sparkHome: String, jars: Array[String]) = + this(new SparkContext(master, appName, sparkHome, jars.toSeq)) /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param jobName A name for your job, to display on the cluster web UI + * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the slave nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. * @param environment Environment variables to set on worker nodes */ - def this(master: String, jobName: String, sparkHome: String, jars: Array[String], + def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, jobName, sparkHome, jars.toSeq, environment)) + this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment)) private[spark] val env = sc.env diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index 519e31032304e..d618c098c2bed 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -9,7 +9,7 @@ import java.util.Arrays * * Stores the unique id() of the Python-side partitioning function so that it is incorporated into * equality comparisons. Correctness requires that the id is a unique identifier for the - * lifetime of the job (i.e. that it is not re-used as the id of a different partitioning + * lifetime of the program (i.e. that it is not re-used as the id of a different partitioning * function). This can be ensured by using the Python id() function and maintaining a reference * to the Python partitioning function so that its id() is not reused. */ diff --git a/core/src/main/scala/spark/deploy/JobDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala similarity index 66% rename from core/src/main/scala/spark/deploy/JobDescription.scala rename to core/src/main/scala/spark/deploy/ApplicationDescription.scala index 7160fc05fc822..6659e53b25f37 100644 --- a/core/src/main/scala/spark/deploy/JobDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -1,6 +1,6 @@ package spark.deploy -private[spark] class JobDescription( +private[spark] class ApplicationDescription( val name: String, val cores: Int, val memoryPerSlave: Int, @@ -10,5 +10,5 @@ private[spark] class JobDescription( val user = System.getProperty("user.name", "") - override def toString: String = "JobDescription(" + name + ")" + override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 1d88d4bc84926..3cbf4fdd98be0 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -1,7 +1,7 @@ package spark.deploy import spark.deploy.ExecutorState.ExecutorState -import spark.deploy.master.{WorkerInfo, JobInfo} +import spark.deploy.master.{WorkerInfo, ApplicationInfo} import spark.deploy.worker.ExecutorRunner import scala.collection.immutable.List @@ -23,7 +23,7 @@ case class RegisterWorker( private[spark] case class ExecutorStateChanged( - jobId: String, + appId: String, execId: Int, state: ExecutorState, message: Option[String], @@ -36,12 +36,12 @@ private[spark] case class Heartbeat(workerId: String) extends DeployMessage private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage -private[spark] case class KillExecutor(jobId: String, execId: Int) extends DeployMessage +private[spark] case class KillExecutor(appId: String, execId: Int) extends DeployMessage private[spark] case class LaunchExecutor( - jobId: String, + appId: String, execId: Int, - jobDesc: JobDescription, + appDesc: ApplicationDescription, cores: Int, memory: Int, sparkHome: String) @@ -49,12 +49,13 @@ private[spark] case class LaunchExecutor( // Client to Master -private[spark] case class RegisterJob(jobDescription: JobDescription) extends DeployMessage +private[spark] case class RegisterApplication(appDescription: ApplicationDescription) + extends DeployMessage // Master to Client private[spark] -case class RegisteredJob(jobId: String) extends DeployMessage +case class RegisteredApplication(appId: String) extends DeployMessage private[spark] case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) @@ -64,7 +65,7 @@ case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String exitStatus: Option[Int]) private[spark] -case class JobKilled(message: String) +case class appKilled(message: String) // Internal message in Client @@ -78,7 +79,7 @@ private[spark] case object RequestMasterState private[spark] case class MasterState(host: String, port: Int, workers: Array[WorkerInfo], - activeJobs: Array[JobInfo], completedJobs: Array[JobInfo]) { + activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { def uri = "spark://" + host + ":" + port } diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 732fa080645b0..38a6ebfc242c1 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,6 +1,6 @@ package spark.deploy -import master.{JobInfo, WorkerInfo} +import master.{ApplicationInfo, WorkerInfo} import worker.ExecutorRunner import cc.spray.json._ @@ -20,8 +20,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { ) } - implicit object JobInfoJsonFormat extends RootJsonWriter[JobInfo] { - def write(obj: JobInfo) = JsObject( + implicit object AppInfoJsonFormat extends RootJsonWriter[ApplicationInfo] { + def write(obj: ApplicationInfo) = JsObject( "starttime" -> JsNumber(obj.startTime), "id" -> JsString(obj.id), "name" -> JsString(obj.desc.name), @@ -31,8 +31,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { "submitdate" -> JsString(obj.submitDate.toString)) } - implicit object JobDescriptionJsonFormat extends RootJsonWriter[JobDescription] { - def write(obj: JobDescription) = JsObject( + implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] { + def write(obj: ApplicationDescription) = JsObject( "name" -> JsString(obj.name), "cores" -> JsNumber(obj.cores), "memoryperslave" -> JsNumber(obj.memoryPerSlave), @@ -44,8 +44,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { def write(obj: ExecutorRunner) = JsObject( "id" -> JsNumber(obj.execId), "memory" -> JsNumber(obj.memory), - "jobid" -> JsString(obj.jobId), - "jobdesc" -> obj.jobDesc.toJson.asJsObject + "appid" -> JsString(obj.appId), + "appdesc" -> obj.appDesc.toJson.asJsObject ) } @@ -57,8 +57,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { "coresused" -> JsNumber(obj.workers.map(_.coresUsed).sum), "memory" -> JsNumber(obj.workers.map(_.memory).sum), "memoryused" -> JsNumber(obj.workers.map(_.memoryUsed).sum), - "activejobs" -> JsArray(obj.activeJobs.toList.map(_.toJson)), - "completedjobs" -> JsArray(obj.completedJobs.toList.map(_.toJson)) + "activeapps" -> JsArray(obj.activeApps.toList.map(_.toJson)), + "completedapps" -> JsArray(obj.completedApps.toList.map(_.toJson)) ) } diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index e01181d1b2807..1a95524cf9c49 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -8,25 +8,25 @@ import akka.pattern.AskTimeoutException import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown -import spark.deploy.RegisterJob +import spark.deploy.RegisterApplication import spark.deploy.master.Master import akka.remote.RemoteClientDisconnected import akka.actor.Terminated import akka.dispatch.Await /** - * The main class used to talk to a Spark deploy cluster. Takes a master URL, a job description, - * and a listener for job events, and calls back the listener when various events occur. + * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description, + * and a listener for cluster events, and calls back the listener when various events occur. */ private[spark] class Client( actorSystem: ActorSystem, masterUrl: String, - jobDescription: JobDescription, + appDescription: ApplicationDescription, listener: ClientListener) extends Logging { var actor: ActorRef = null - var jobId: String = null + var appId: String = null class ClientActor extends Actor with Logging { var master: ActorRef = null @@ -38,7 +38,7 @@ private[spark] class Client( try { master = context.actorFor(Master.toAkkaUrl(masterUrl)) masterAddress = master.path.address - master ! RegisterJob(jobDescription) + master ! RegisterApplication(appDescription) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { @@ -50,17 +50,17 @@ private[spark] class Client( } override def receive = { - case RegisteredJob(jobId_) => - jobId = jobId_ - listener.connected(jobId) + case RegisteredApplication(appId_) => + appId = appId_ + listener.connected(appId) case ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) => - val fullId = jobId + "/" + id + val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, host, cores)) listener.executorAdded(fullId, workerId, host, cores, memory) case ExecutorUpdated(id, state, message, exitStatus) => - val fullId = jobId + "/" + id + val fullId = appId + "/" + id val messageText = message.map(s => " (" + s + ")").getOrElse("") logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText)) if (ExecutorState.isFinished(state)) { diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index 7035f4b394242..b7008321df564 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -8,7 +8,7 @@ package spark.deploy.client * Users of this API should *not* block inside the callback methods. */ private[spark] trait ClientListener { - def connected(jobId: String): Unit + def connected(appId: String): Unit def disconnected(): Unit diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index 8764c400e26d8..dc004b59ca5ac 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -2,13 +2,13 @@ package spark.deploy.client import spark.util.AkkaUtils import spark.{Logging, Utils} -import spark.deploy.{Command, JobDescription} +import spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { class TestListener extends ClientListener with Logging { def connected(id: String) { - logInfo("Connected to master, got job ID " + id) + logInfo("Connected to master, got app ID " + id) } def disconnected() { @@ -24,7 +24,7 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) - val desc = new JobDescription( + val desc = new ApplicationDescription( "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home") val listener = new TestListener val client = new Client(actorSystem, url, desc, listener) diff --git a/core/src/main/scala/spark/deploy/master/JobInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala similarity index 84% rename from core/src/main/scala/spark/deploy/master/JobInfo.scala rename to core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index a274b21c346f2..3591a9407237a 100644 --- a/core/src/main/scala/spark/deploy/master/JobInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -1,18 +1,18 @@ package spark.deploy.master -import spark.deploy.JobDescription +import spark.deploy.ApplicationDescription import java.util.Date import akka.actor.ActorRef import scala.collection.mutable -private[spark] class JobInfo( +private[spark] class ApplicationInfo( val startTime: Long, val id: String, - val desc: JobDescription, + val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef) { - var state = JobState.WAITING + var state = ApplicationState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] var coresGranted = 0 var endTime = -1L @@ -48,7 +48,7 @@ private[spark] class JobInfo( _retryCount } - def markFinished(endState: JobState.Value) { + def markFinished(endState: ApplicationState.Value) { state = endState endTime = System.currentTimeMillis() } diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala new file mode 100644 index 0000000000000..15016b388d2d8 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala @@ -0,0 +1,11 @@ +package spark.deploy.master + +private[spark] object ApplicationState + extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { + + type ApplicationState = Value + + val WAITING, RUNNING, FINISHED, FAILED = Value + + val MAX_NUM_RETRY = 10 +} diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala index 1db2c326333f1..48e6055fb572a 100644 --- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala @@ -4,12 +4,12 @@ import spark.deploy.ExecutorState private[spark] class ExecutorInfo( val id: Int, - val job: JobInfo, + val application: ApplicationInfo, val worker: WorkerInfo, val cores: Int, val memory: Int) { var state = ExecutorState.LAUNCHING - def fullId: String = job.id + "/" + id + def fullId: String = application.id + "/" + id } diff --git a/core/src/main/scala/spark/deploy/master/JobState.scala b/core/src/main/scala/spark/deploy/master/JobState.scala deleted file mode 100644 index 2b70cf01918d7..0000000000000 --- a/core/src/main/scala/spark/deploy/master/JobState.scala +++ /dev/null @@ -1,9 +0,0 @@ -package spark.deploy.master - -private[spark] object JobState extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { - type JobState = Value - - val WAITING, RUNNING, FINISHED, FAILED = Value - - val MAX_NUM_RETRY = 10 -} diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index a5de23261c61a..1cd68a2aa61dd 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -16,22 +16,22 @@ import spark.util.AkkaUtils private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { - val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For job IDs + val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 - var nextJobNumber = 0 + var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] val idToWorker = new HashMap[String, WorkerInfo] val actorToWorker = new HashMap[ActorRef, WorkerInfo] val addressToWorker = new HashMap[Address, WorkerInfo] - val jobs = new HashSet[JobInfo] - val idToJob = new HashMap[String, JobInfo] - val actorToJob = new HashMap[ActorRef, JobInfo] - val addressToJob = new HashMap[Address, JobInfo] + val apps = new HashSet[ApplicationInfo] + val idToApp = new HashMap[String, ApplicationInfo] + val actorToApp = new HashMap[ActorRef, ApplicationInfo] + val addressToApp = new HashMap[Address, ApplicationInfo] - val waitingJobs = new ArrayBuffer[JobInfo] - val completedJobs = new ArrayBuffer[JobInfo] + val waitingApps = new ArrayBuffer[ApplicationInfo] + val completedApps = new ArrayBuffer[ApplicationInfo] val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -39,9 +39,9 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } // As a temporary workaround before better ways of configuring memory, we allow users to set - // a flag that will perform round-robin scheduling across the nodes (spreading out each job - // among all the nodes) instead of trying to consolidate each job onto a small # of nodes. - val spreadOutJobs = System.getProperty("spark.deploy.spreadOut", "false").toBoolean + // a flag that will perform round-robin scheduling across the nodes (spreading out each app + // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. + val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "false").toBoolean override def preStart() { logInfo("Starting Spark master at spark://" + ip + ":" + port) @@ -76,41 +76,41 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } } - case RegisterJob(description) => { - logInfo("Registering job " + description.name) - val job = addJob(description, sender) - logInfo("Registered job " + description.name + " with ID " + job.id) - waitingJobs += job + case RegisterApplication(description) => { + logInfo("Registering app " + description.name) + val app = addApplication(description, sender) + logInfo("Registered app " + description.name + " with ID " + app.id) + waitingApps += app context.watch(sender) // This doesn't work with remote actors but helps for testing - sender ! RegisteredJob(job.id) + sender ! RegisteredApplication(app.id) schedule() } - case ExecutorStateChanged(jobId, execId, state, message, exitStatus) => { - val execOption = idToJob.get(jobId).flatMap(job => job.executors.get(execId)) + case ExecutorStateChanged(appId, execId, state, message, exitStatus) => { + val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId)) execOption match { case Some(exec) => { exec.state = state - exec.job.driver ! ExecutorUpdated(execId, state, message, exitStatus) + exec.application.driver ! ExecutorUpdated(execId, state, message, exitStatus) if (ExecutorState.isFinished(state)) { - val jobInfo = idToJob(jobId) - // Remove this executor from the worker and job + val appInfo = idToApp(appId) + // Remove this executor from the worker and app logInfo("Removing executor " + exec.fullId + " because it is " + state) - jobInfo.removeExecutor(exec) + appInfo.removeExecutor(exec) exec.worker.removeExecutor(exec) // Only retry certain number of times so we don't go into an infinite loop. - if (jobInfo.incrementRetryCount < JobState.MAX_NUM_RETRY) { + if (appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { schedule() } else { - logError("Job %s with ID %s failed %d times, removing it".format( - jobInfo.desc.name, jobInfo.id, jobInfo.retryCount)) - removeJob(jobInfo) + logError("Application %s with ID %s failed %d times, removing it".format( + appInfo.desc.name, appInfo.id, appInfo.retryCount)) + removeApplication(appInfo) } } } case None => - logWarning("Got status update for unknown executor " + jobId + "/" + execId) + logWarning("Got status update for unknown executor " + appId + "/" + execId) } } @@ -124,53 +124,53 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } case Terminated(actor) => { - // The disconnected actor could've been either a worker or a job; remove whichever of + // The disconnected actor could've been either a worker or an app; remove whichever of // those we have an entry for in the corresponding actor hashmap actorToWorker.get(actor).foreach(removeWorker) - actorToJob.get(actor).foreach(removeJob) + actorToApp.get(actor).foreach(removeApplication) } case RemoteClientDisconnected(transport, address) => { - // The disconnected client could've been either a worker or a job; remove whichever it was + // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) - addressToJob.get(address).foreach(removeJob) + addressToApp.get(address).foreach(removeApplication) } case RemoteClientShutdown(transport, address) => { - // The disconnected client could've been either a worker or a job; remove whichever it was + // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) - addressToJob.get(address).foreach(removeJob) + addressToApp.get(address).foreach(removeApplication) } case RequestMasterState => { - sender ! MasterState(ip, port, workers.toArray, jobs.toArray, completedJobs.toArray) + sender ! MasterState(ip, port, workers.toArray, apps.toArray, completedApps.toArray) } } /** - * Can a job use the given worker? True if the worker has enough memory and we haven't already - * launched an executor for the job on it (right now the standalone backend doesn't like having + * Can an app use the given worker? True if the worker has enough memory and we haven't already + * launched an executor for the app on it (right now the standalone backend doesn't like having * two executors on the same worker). */ - def canUse(job: JobInfo, worker: WorkerInfo): Boolean = { - worker.memoryFree >= job.desc.memoryPerSlave && !worker.hasExecutor(job) + def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { + worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) } /** - * Schedule the currently available resources among waiting jobs. This method will be called - * every time a new job joins or resource availability changes. + * Schedule the currently available resources among waiting apps. This method will be called + * every time a new app joins or resource availability changes. */ def schedule() { - // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first job - // in the queue, then the second job, etc. - if (spreadOutJobs) { - // Try to spread out each job among all the nodes, until it has all its cores - for (job <- waitingJobs if job.coresLeft > 0) { + // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app + // in the queue, then the second app, etc. + if (spreadOutApps) { + // Try to spread out each app among all the nodes, until it has all its cores + for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(job, _)).sortBy(_.coresFree).reverse + .filter(canUse(app, _)).sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node - var toAssign = math.min(job.coresLeft, usableWorkers.map(_.coresFree).sum) + var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) var pos = 0 while (toAssign > 0) { if (usableWorkers(pos).coresFree - assigned(pos) > 0) { @@ -182,22 +182,22 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor // Now that we've decided how many cores to give on each node, let's actually give them for (pos <- 0 until numUsable) { if (assigned(pos) > 0) { - val exec = job.addExecutor(usableWorkers(pos), assigned(pos)) - launchExecutor(usableWorkers(pos), exec, job.desc.sparkHome) - job.state = JobState.RUNNING + val exec = app.addExecutor(usableWorkers(pos), assigned(pos)) + launchExecutor(usableWorkers(pos), exec, app.desc.sparkHome) + app.state = ApplicationState.RUNNING } } } } else { - // Pack each job into as few nodes as possible until we've assigned all its cores + // Pack each app into as few nodes as possible until we've assigned all its cores for (worker <- workers if worker.coresFree > 0) { - for (job <- waitingJobs if job.coresLeft > 0) { - if (canUse(job, worker)) { - val coresToUse = math.min(worker.coresFree, job.coresLeft) + for (app <- waitingApps if app.coresLeft > 0) { + if (canUse(app, worker)) { + val coresToUse = math.min(worker.coresFree, app.coresLeft) if (coresToUse > 0) { - val exec = job.addExecutor(worker, coresToUse) - launchExecutor(worker, exec, job.desc.sparkHome) - job.state = JobState.RUNNING + val exec = app.addExecutor(worker, coresToUse) + launchExecutor(worker, exec, app.desc.sparkHome) + app.state = ApplicationState.RUNNING } } } @@ -208,8 +208,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) - worker.actor ! LaunchExecutor(exec.job.id, exec.id, exec.job.desc, exec.cores, exec.memory, sparkHome) - exec.job.driver ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory) + worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) + exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory) } def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int, @@ -231,46 +231,46 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor actorToWorker -= worker.actor addressToWorker -= worker.actor.path.address for (exec <- worker.executors.values) { - logInfo("Telling job of lost executor: " + exec.id) - exec.job.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None) - exec.job.removeExecutor(exec) + logInfo("Telling app of lost executor: " + exec.id) + exec.application.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None) + exec.application.removeExecutor(exec) } } - def addJob(desc: JobDescription, driver: ActorRef): JobInfo = { + def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val job = new JobInfo(now, newJobId(date), desc, date, driver) - jobs += job - idToJob(job.id) = job - actorToJob(driver) = job - addressToJob(driver.path.address) = job - return job + val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver) + apps += app + idToApp(app.id) = app + actorToApp(driver) = app + addressToApp(driver.path.address) = app + return app } - def removeJob(job: JobInfo) { - if (jobs.contains(job)) { - logInfo("Removing job " + job.id) - jobs -= job - idToJob -= job.id - actorToJob -= job.driver - addressToWorker -= job.driver.path.address - completedJobs += job // Remember it in our history - waitingJobs -= job - for (exec <- job.executors.values) { + def removeApplication(app: ApplicationInfo) { + if (apps.contains(app)) { + logInfo("Removing app " + app.id) + apps -= app + idToApp -= app.id + actorToApp -= app.driver + addressToWorker -= app.driver.path.address + completedApps += app // Remember it in our history + waitingApps -= app + for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) - exec.worker.actor ! KillExecutor(exec.job.id, exec.id) + exec.worker.actor ! KillExecutor(exec.application.id, exec.id) } - job.markFinished(JobState.FINISHED) // TODO: Mark it as FAILED if it failed + app.markFinished(ApplicationState.FINISHED) // TODO: Mark it as FAILED if it failed schedule() } } - /** Generate a new job ID given a job's submission date */ - def newJobId(submitDate: Date): String = { - val jobId = "job-%s-%04d".format(DATE_FORMAT.format(submitDate), nextJobNumber) - nextJobNumber += 1 - jobId + /** Generate a new app ID given a app's submission date */ + def newApplicationId(submitDate: Date): String = { + val appId = "app-%s-%04d".format(DATE_FORMAT.format(submitDate), nextAppNumber) + nextAppNumber += 1 + appId } /** Check for, and remove, any timed-out workers */ diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 529f72e9da1c2..54faa375fbd46 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -40,27 +40,27 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct } } } ~ - path("job") { - parameters("jobId", 'format ?) { - case (jobId, Some(js)) if (js.equalsIgnoreCase("json")) => + path("app") { + parameters("appId", 'format ?) { + case (appId, Some(js)) if (js.equalsIgnoreCase("json")) => val future = master ? RequestMasterState - val jobInfo = for (masterState <- future.mapTo[MasterState]) yield { - masterState.activeJobs.find(_.id == jobId).getOrElse({ - masterState.completedJobs.find(_.id == jobId).getOrElse(null) + val appInfo = for (masterState <- future.mapTo[MasterState]) yield { + masterState.activeApps.find(_.id == appId).getOrElse({ + masterState.completedApps.find(_.id == appId).getOrElse(null) }) } respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(jobInfo.mapTo[JobInfo]) + ctx.complete(appInfo.mapTo[ApplicationInfo]) } - case (jobId, _) => + case (appId, _) => completeWith { val future = master ? RequestMasterState future.map { state => val masterState = state.asInstanceOf[MasterState] - val job = masterState.activeJobs.find(_.id == jobId).getOrElse({ - masterState.completedJobs.find(_.id == jobId).getOrElse(null) + val app = masterState.activeApps.find(_.id == appId).getOrElse({ + masterState.completedApps.find(_.id == appId).getOrElse(null) }) - spark.deploy.master.html.job_details.render(job) + spark.deploy.master.html.app_details.render(app) } } } diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 2e467007a0165..23df1bb463288 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -37,8 +37,8 @@ private[spark] class WorkerInfo( } } - def hasExecutor(job: JobInfo): Boolean = { - executors.values.exists(_.job == job) + def hasExecutor(app: ApplicationInfo): Boolean = { + executors.values.exists(_.application == app) } def webUiAddress : String = { diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 69f34e604a6fc..de11771c8e62d 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -1,7 +1,7 @@ package spark.deploy.worker import java.io._ -import spark.deploy.{ExecutorState, ExecutorStateChanged, JobDescription} +import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription} import akka.actor.ActorRef import spark.{Utils, Logging} import java.net.{URI, URL} @@ -14,9 +14,9 @@ import spark.deploy.ExecutorStateChanged * Manages the execution of one executor process. */ private[spark] class ExecutorRunner( - val jobId: String, + val appId: String, val execId: Int, - val jobDesc: JobDescription, + val appDesc: ApplicationDescription, val cores: Int, val memory: Int, val worker: ActorRef, @@ -26,7 +26,7 @@ private[spark] class ExecutorRunner( val workDir: File) extends Logging { - val fullId = jobId + "/" + execId + val fullId = appId + "/" + execId var workerThread: Thread = null var process: Process = null var shutdownHook: Thread = null @@ -60,7 +60,7 @@ private[spark] class ExecutorRunner( process.destroy() process.waitFor() } - worker ! ExecutorStateChanged(jobId, execId, ExecutorState.KILLED, None, None) + worker ! ExecutorStateChanged(appId, execId, ExecutorState.KILLED, None, None) Runtime.getRuntime.removeShutdownHook(shutdownHook) } } @@ -74,10 +74,10 @@ private[spark] class ExecutorRunner( } def buildCommandSeq(): Seq[String] = { - val command = jobDesc.command - val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run"; + val command = appDesc.command + val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run" val runScript = new File(sparkHome, script).getCanonicalPath - Seq(runScript, command.mainClass) ++ (command.arguments ++ Seq(jobId)).map(substituteVariables) + Seq(runScript, command.mainClass) ++ (command.arguments ++ Seq(appId)).map(substituteVariables) } /** Spawn a thread that will redirect a given stream to a file */ @@ -96,12 +96,12 @@ private[spark] class ExecutorRunner( } /** - * Download and run the executor described in our JobDescription + * Download and run the executor described in our ApplicationDescription */ def fetchAndRunExecutor() { try { // Create the executor's working directory - val executorDir = new File(workDir, jobId + "/" + execId) + val executorDir = new File(workDir, appId + "/" + execId) if (!executorDir.mkdirs()) { throw new IOException("Failed to create directory " + executorDir) } @@ -110,7 +110,7 @@ private[spark] class ExecutorRunner( val command = buildCommandSeq() val builder = new ProcessBuilder(command: _*).directory(executorDir) val env = builder.environment() - for ((key, value) <- jobDesc.command.environment) { + for ((key, value) <- appDesc.command.environment) { env.put(key, value) } env.put("SPARK_MEM", memory.toString + "m") @@ -128,7 +128,7 @@ private[spark] class ExecutorRunner( // times on the same machine. val exitCode = process.waitFor() val message = "Command exited with code " + exitCode - worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message), + worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message), Some(exitCode)) } catch { case interrupted: InterruptedException => @@ -140,7 +140,7 @@ private[spark] class ExecutorRunner( process.destroy() } val message = e.getClass + ": " + e.getMessage - worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message), None) + worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message), None) } } } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 924935a5fdb2c..2bbc931316291 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -109,19 +109,19 @@ private[spark] class Worker( logError("Worker registration failed: " + message) System.exit(1) - case LaunchExecutor(jobId, execId, jobDesc, cores_, memory_, execSparkHome_) => - logInfo("Asked to launch executor %s/%d for %s".format(jobId, execId, jobDesc.name)) + case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) => + logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner( - jobId, execId, jobDesc, cores_, memory_, self, workerId, ip, new File(execSparkHome_), workDir) - executors(jobId + "/" + execId) = manager + appId, execId, appDesc, cores_, memory_, self, workerId, ip, new File(execSparkHome_), workDir) + executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ memoryUsed += memory_ - master ! ExecutorStateChanged(jobId, execId, ExecutorState.RUNNING, None, None) + master ! ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None) - case ExecutorStateChanged(jobId, execId, state, message, exitStatus) => - master ! ExecutorStateChanged(jobId, execId, state, message, exitStatus) - val fullId = jobId + "/" + execId + case ExecutorStateChanged(appId, execId, state, message, exitStatus) => + master ! ExecutorStateChanged(appId, execId, state, message, exitStatus) + val fullId = appId + "/" + execId if (ExecutorState.isFinished(state)) { val executor = executors(fullId) logInfo("Executor " + fullId + " finished with state " + state + @@ -133,8 +133,8 @@ private[spark] class Worker( memoryUsed -= executor.memory } - case KillExecutor(jobId, execId) => - val fullId = jobId + "/" + execId + case KillExecutor(appId, execId) => + val fullId = appId + "/" + execId executors.get(fullId) match { case Some(executor) => logInfo("Asked to kill executor " + fullId) diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala index 37524a7c82f8b..08f02bad80d7f 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala @@ -92,7 +92,7 @@ private[spark] class WorkerArguments(args: Array[String]) { "Options:\n" + " -c CORES, --cores CORES Number of cores to use\n" + " -m MEM, --memory MEM Amount of memory to use (e.g. 1000M, 2G)\n" + - " -d DIR, --work-dir DIR Directory to run jobs in (default: SPARK_HOME/work)\n" + + " -d DIR, --work-dir DIR Directory to run apps in (default: SPARK_HOME/work)\n" + " -i IP, --ip IP IP address or DNS name to listen on\n" + " -p PORT, --port PORT Port to listen on (default: random)\n" + " --webui-port PORT Port for web UI (default: 8081)") diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index ef81f072a3082..135cc2e86cc92 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -41,9 +41,9 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct } } ~ path("log") { - parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) => + parameters("appId", "executorId", "logType") { (appId, executorId, logType) => respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) { - getFromFileName("work/" + jobId + "/" + executorId + "/" + logType) + getFromFileName("work/" + appId + "/" + executorId + "/" + logType) } } } ~ diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index e77355c6cd41a..bb289c9cf391b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -2,14 +2,14 @@ package spark.scheduler.cluster import spark.{Utils, Logging, SparkContext} import spark.deploy.client.{Client, ClientListener} -import spark.deploy.{Command, JobDescription} +import spark.deploy.{Command, ApplicationDescription} import scala.collection.mutable.HashMap private[spark] class SparkDeploySchedulerBackend( scheduler: ClusterScheduler, sc: SparkContext, master: String, - jobName: String) + appName: String) extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem) with ClientListener with Logging { @@ -29,10 +29,11 @@ private[spark] class SparkDeploySchedulerBackend( StandaloneSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) - val sparkHome = sc.getSparkHome().getOrElse(throw new IllegalArgumentException("must supply spark home for spark standalone")) - val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command, sparkHome) + val sparkHome = sc.getSparkHome().getOrElse( + throw new IllegalArgumentException("must supply spark home for spark standalone")) + val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome) - client = new Client(sc.env.actorSystem, master, jobDesc, this) + client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() } @@ -45,8 +46,8 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def connected(jobId: String) { - logInfo("Connected to Spark cluster with job ID " + jobId) + override def connected(appId: String) { + logInfo("Connected to Spark cluster with app ID " + appId) } override def disconnected() { diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 7caf06e917125..f4a2994b6d61d 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ private[spark] class CoarseMesosSchedulerBackend( scheduler: ClusterScheduler, sc: SparkContext, master: String, - frameworkName: String) + appName: String) extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem) with MScheduler with Logging { @@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = CoarseMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(frameworkName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { { val ret = driver.run() diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 300766d0f5c72..ca7fab4cc5ff9 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -24,7 +24,7 @@ private[spark] class MesosSchedulerBackend( scheduler: ClusterScheduler, sc: SparkContext, master: String, - frameworkName: String) + appName: String) extends SchedulerBackend with MScheduler with Logging { @@ -49,7 +49,7 @@ private[spark] class MesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = MesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(frameworkName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { val ret = driver.run() diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html new file mode 100644 index 0000000000000..301a7e212495d --- /dev/null +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -0,0 +1,40 @@ +@(app: spark.deploy.master.ApplicationInfo) + +@spark.common.html.layout(title = "Application Details") { + + +
    +
    +
      +
    • ID: @app.id
    • +
    • Description: @app.desc.name
    • +
    • User: @app.desc.user
    • +
    • Cores: + @app.desc.cores + (@app.coresGranted Granted + @if(app.desc.cores == Integer.MAX_VALUE) { + + } else { + , @app.coresLeft + } + ) +
    • +
    • Memory per Slave: @app.desc.memoryPerSlave
    • +
    • Submit Date: @app.submitDate
    • +
    • State: @app.state
    • +
    +
    +
    + +
    + + +
    +
    +

    Executor Summary

    +
    + @executors_table(app.executors.values.toList) +
    +
    + +} diff --git a/core/src/main/twirl/spark/deploy/master/app_row.scala.html b/core/src/main/twirl/spark/deploy/master/app_row.scala.html new file mode 100644 index 0000000000000..feb306f35ccb5 --- /dev/null +++ b/core/src/main/twirl/spark/deploy/master/app_row.scala.html @@ -0,0 +1,20 @@ +@(app: spark.deploy.master.ApplicationInfo) + +@import spark.Utils +@import spark.deploy.WebUI.formatDate +@import spark.deploy.WebUI.formatDuration + + + + @app.id + + @app.desc.name + + @app.coresGranted + + @Utils.memoryMegabytesToString(app.desc.memoryPerSlave) + @formatDate(app.submitDate) + @app.desc.user + @app.state.toString() + @formatDuration(app.duration) + diff --git a/core/src/main/twirl/spark/deploy/master/job_table.scala.html b/core/src/main/twirl/spark/deploy/master/app_table.scala.html similarity index 74% rename from core/src/main/twirl/spark/deploy/master/job_table.scala.html rename to core/src/main/twirl/spark/deploy/master/app_table.scala.html index d267d6e85e0b7..f789cee0f16ea 100644 --- a/core/src/main/twirl/spark/deploy/master/job_table.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_table.scala.html @@ -1,9 +1,9 @@ -@(jobs: Array[spark.deploy.master.JobInfo]) +@(apps: Array[spark.deploy.master.ApplicationInfo]) - + @@ -14,8 +14,8 @@ - @for(j <- jobs) { - @job_row(j) + @for(j <- apps) { + @app_row(j) }
    JobIDID Description Cores Memory per Node
    diff --git a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html index 784d692fc2f91..d2d80fad48992 100644 --- a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html @@ -9,7 +9,7 @@ @executor.memory @executor.state - stdout - stderr + stdout + stderr - \ No newline at end of file + diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html index cb1651c7e1b05..ac51a39a5199d 100644 --- a/core/src/main/twirl/spark/deploy/master/index.scala.html +++ b/core/src/main/twirl/spark/deploy/master/index.scala.html @@ -14,7 +14,7 @@ @{state.workers.map(_.coresUsed).sum} Used
  • Memory: @{Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, @{Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
  • -
  • Jobs: @state.activeJobs.size Running, @state.completedJobs.size Completed
  • +
  • Applications: @state.activeApps.size Running, @state.completedApps.size Completed
  • @@ -22,7 +22,7 @@
    -

    Cluster Summary

    +

    Workers


    @worker_table(state.workers.sortBy(_.id))
    @@ -30,23 +30,23 @@

    Cluster Summary


    - +
    -

    Running Jobs

    +

    Running Applications


    - @job_table(state.activeJobs.sortBy(_.startTime).reverse) + @app_table(state.activeApps.sortBy(_.startTime).reverse)

    - +
    -

    Completed Jobs

    +

    Completed Applications


    - @job_table(state.completedJobs.sortBy(_.endTime).reverse) + @app_table(state.completedApps.sortBy(_.endTime).reverse)
    diff --git a/core/src/main/twirl/spark/deploy/master/job_details.scala.html b/core/src/main/twirl/spark/deploy/master/job_details.scala.html deleted file mode 100644 index d02a51b214180..0000000000000 --- a/core/src/main/twirl/spark/deploy/master/job_details.scala.html +++ /dev/null @@ -1,40 +0,0 @@ -@(job: spark.deploy.master.JobInfo) - -@spark.common.html.layout(title = "Job Details") { - - -
    -
    -
      -
    • ID: @job.id
    • -
    • Description: @job.desc.name
    • -
    • User: @job.desc.user
    • -
    • Cores: - @job.desc.cores - (@job.coresGranted Granted - @if(job.desc.cores == Integer.MAX_VALUE) { - - } else { - , @job.coresLeft - } - ) -
    • -
    • Memory per Slave: @job.desc.memoryPerSlave
    • -
    • Submit Date: @job.submitDate
    • -
    • State: @job.state
    • -
    -
    -
    - -
    - - -
    -
    -

    Executor Summary

    -
    - @executors_table(job.executors.values.toList) -
    -
    - -} diff --git a/core/src/main/twirl/spark/deploy/master/job_row.scala.html b/core/src/main/twirl/spark/deploy/master/job_row.scala.html deleted file mode 100644 index 7c466a6a2ce4d..0000000000000 --- a/core/src/main/twirl/spark/deploy/master/job_row.scala.html +++ /dev/null @@ -1,20 +0,0 @@ -@(job: spark.deploy.master.JobInfo) - -@import spark.Utils -@import spark.deploy.WebUI.formatDate -@import spark.deploy.WebUI.formatDuration - - - - @job.id - - @job.desc.name - - @job.coresGranted - - @Utils.memoryMegabytesToString(job.desc.memoryPerSlave) - @formatDate(job.submitDate) - @job.desc.user - @job.state.toString() - @formatDuration(job.duration) - diff --git a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html index ea9542461e5f7..dad0a89080f25 100644 --- a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html +++ b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html @@ -8,13 +8,13 @@ @Utils.memoryMegabytesToString(executor.memory) - stdout - stderr + stdout + stderr diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 2f3adb39c2506..80244520a3726 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -12,7 +12,7 @@ private[streaming] class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { val master = ssc.sc.master - val framework = ssc.sc.jobName + val framework = ssc.sc.appName val sparkHome = ssc.sc.sparkHome val jars = ssc.sc.jars val graph = ssc.graph diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 37ba524b4897e..0cce2b13cf8c4 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -39,11 +39,11 @@ class StreamingContext private ( /** * Creates a StreamingContext by providing the details necessary for creating a new SparkContext. * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param frameworkName A name for your job, to display on the cluster web UI + * @param appName A name for your job, to display on the cluster web UI * @param batchDuration The time interval at which streaming data will be divided into batches */ - def this(master: String, frameworkName: String, batchDuration: Duration) = - this(StreamingContext.createNewSparkContext(master, frameworkName), null, batchDuration) + def this(master: String, appName: String, batchDuration: Duration) = + this(StreamingContext.createNewSparkContext(master, appName), null, batchDuration) /** * Re-creates a StreamingContext from a checkpoint file. @@ -384,14 +384,14 @@ object StreamingContext { new PairDStreamFunctions[K, V](stream) } - protected[streaming] def createNewSparkContext(master: String, frameworkName: String): SparkContext = { + protected[streaming] def createNewSparkContext(master: String, appName: String): SparkContext = { // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second interval. if (MetadataCleaner.getDelaySeconds < 0) { MetadataCleaner.setDelaySeconds(3600) } - new SparkContext(master, frameworkName) + new SparkContext(master, appName) } protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = { diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index e7f446a49b581..e5b5e9ac233ed 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -27,11 +27,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Creates a StreamingContext. * @param master Name of the Spark Master - * @param frameworkName Name to be used when registering with the scheduler + * @param appName Name to be used when registering with the scheduler * @param batchDuration The time interval at which streaming data will be divided into batches */ - def this(master: String, frameworkName: String, batchDuration: Duration) = - this(new StreamingContext(master, frameworkName, batchDuration)) + def this(master: String, appName: String, batchDuration: Duration) = + this(new StreamingContext(master, appName, batchDuration)) /** * Creates a StreamingContext. From 8ad561dc7d6475d7b217ec3f57bac3b584fed31a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 18 Feb 2013 02:12:41 -0800 Subject: [PATCH 237/696] Added checkpointing and fault-tolerance semantics to the programming guide. Fixed default checkpoint interval to being a multiple of slide duration. Fixed visibility of some classes and objects to clean up docs. --- docs/python-programming-guide.md | 2 +- docs/streaming-programming-guide.md | 246 ++++++++++++++---- .../main/scala/spark/streaming/DStream.scala | 2 +- .../main/scala/spark/streaming/Duration.scala | 2 +- .../main/scala/spark/streaming/Interval.scala | 1 + .../streaming/PairDStreamFunctions.scala | 8 +- .../dstream/TwitterInputDStream.scala | 2 + .../spark/streaming/CheckpointSuite.scala | 2 +- 8 files changed, 206 insertions(+), 59 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 4e84d23edf56b..2012241a6a77b 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -87,7 +87,7 @@ By default, the `pyspark` shell creates SparkContext that runs jobs locally. To connect to a non-local cluster, set the `MASTER` environment variable. For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html): -{% highlight shell %} +{% highlight bash %} $ MASTER=spark://IP:PORT ./pyspark {% endhighlight %} diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index b6da7af654ee6..d408e80359c46 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -34,8 +34,8 @@ The StreamingContext is used to creating InputDStreams from input sources: {% highlight scala %} // Assuming ssc is the StreamingContext -ssc.networkStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port -ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory +ssc.networkStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port +ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory {% endhighlight %} A complete list of input sources is available in the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext). Data received from these sources can be processed using DStream operations, which are explained next. @@ -50,18 +50,18 @@ Once an input DStream has been created, you can transform it using _DStream oper DStreams support many of the transformations available on normal Spark RDD's: - + - + - + - + @@ -70,73 +70,92 @@ DStreams support many of the transformations available on normal Spark RDD's: - + + + + + + + + + + + + + - - + - + - - - - - + + + + + +
    TransformationMeaning
    TransformationMeaning
    map(func) Returns a new DStream formed by passing each element of the source through a function func. Returns a new DStream formed by passing each element of the source DStream through a function func.
    filter(func) Returns a new stream formed by selecting those elements of the source on which func returns true. Returns a new DStream formed by selecting those elements of the source DStream on which func returns true.
    flatMap(func) Similar to map, but each input item can be mapped to 0 or more output items (so func should return a Seq rather than a single item). Similar to map, but each input item can be mapped to 0 or more output items (so func should return a Seq rather than a single item).
    mapPartitions(func)
    union(otherStream) Return a new stream that contains the union of the elements in the source stream and the argument. Return a new DStream that contains the union of the elements in the source DStream and the argument DStream.
    count() Returns a new DStream of single-element RDDs by counting the number of elements in each RDD of the source DStream.
    reduce(func) Returns a new DStream of single-element RDDs by aggregating the elements in each RDD of the source DStream using a function func (which takes two arguments and returns one). The function should be associative so that it can be computed in parallel.
    countByValue() When called on a DStream of elements of type K, returns a new DStream of (K, Long) pairs where the value of each key is its frequency in each RDD of the source DStream.
    groupByKey([numTasks]) When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs.
    -Note: By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional numTasks argument to set a different number of tasks. +
    When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together all the values of each key in the RDDs of the source DStream.
    + Note: By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional numTasks argument to set a different number of tasks.
    reduceByKey(func, [numTasks]) When called on a stream of (K, V) pairs, returns a stream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument. When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument.
    join(otherStream, [numTasks]) When called on streams of type (K, V) and (K, W), returns a stream of (K, (V, W)) pairs with all pairs of elements for each key. When called on two DStreams of (K, V) and (K, W) pairs, returns a new DStream of (K, (V, W)) pairs with all pairs of elements for each key.
    cogroup(otherStream, [numTasks]) When called on DStream of type (K, V) and (K, W), returns a DStream of (K, Seq[V], Seq[W]) tuples.
    reduce(func) Returns a new DStream of single-element RDDs by aggregating the elements of the stream using a function func (which takes two arguments and returns one). The function should be associative so that it can be computed correctly in parallel. When called on DStream of (K, V) and (K, W) pairs, returns a new DStream of (K, Seq[V], Seq[W]) tuples.
    transform(func) Returns a new DStream by applying func (a RDD-to-RDD function) to every RDD of the stream. This can be used to do arbitrary RDD operations on the DStream.
    updateStateByKey(func) Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of each key. This can be used to track session state by using the session-id as the key and updating the session state as new data is received.
    -Spark Streaming features windowed computations, which allow you to report statistics over a sliding window of data. All window functions take a windowDuration, which represents the width of the window and a slideTime, which represents the frequency during which the window is calculated. +Spark Streaming features windowed computations, which allow you to apply transformations over a sliding window of data. All window functions take a windowDuration, which represents the width of the window and a slideTime, which represents the frequency during which the window is calculated. - + - - + - + - + - - + - - + - - + + + + + +
    TransformationMeaning
    TransformationMeaning
    window(windowDuration, slideTime) Return a new stream which is computed based on windowed batches of the source stream. windowDuration is the width of the window and slideTime is the frequency during which the window is calculated. Both times must be multiples of the batch interval. + window(windowDuration, slideDuration) Return a new DStream which is computed based on windowed batches of the source DStream. windowDuration is the width of the window and slideTime is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
    countByWindow(windowDuration, slideTime) countByWindow(windowDuration, slideDuration) Return a sliding count of elements in the stream. windowDuration and slideDuration are exactly as defined in window().
    reduceByWindow(func, windowDuration, slideDuration) reduceByWindow(func, windowDuration, slideDuration) Return a new single-element stream, created by aggregating elements in the stream over a sliding interval using func. The function should be associative so that it can be computed correctly in parallel. windowDuration and slideDuration are exactly as defined in window().
    groupByKeyAndWindow(windowDuration, slideDuration, [numTasks]) + groupByKeyAndWindow(windowDuration, slideDuration, [numTasks]) When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs over a sliding window.
    -Note: By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional numTasks argument to set a different number of tasks. windowDuration and slideDuration are exactly as defined in window(). -
    When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together values of each key over batches in a sliding window.
    +Note: By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional numTasks argument to set a different number of tasks.
    reduceByKeyAndWindow(func, [numTasks]) When called on a stream of (K, V) pairs, returns a stream of (K, V) pairs where the values for each key are aggregated using the given reduce function over batches within a sliding window. Like in groupByKeyAndWindow, the number of reduce tasks is configurable through an optional second argument. + reduceByKeyAndWindow(func, windowDuration, slideDuration, [numTasks]) When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function func over batches in a sliding window. Like in groupByKeyAndWindow, the number of reduce tasks is configurable through an optional second argument. windowDuration and slideDuration are exactly as defined in window().
    countByKeyAndWindow([numTasks]) When called on a stream of (K, V) pairs, returns a stream of (K, Int) pairs where the values for each key are the count within a sliding window. Like in countByKeyAndWindow, the number of reduce tasks is configurable through an optional second argument. + reduceByKeyAndWindow(func, invFunc, windowDuration, slideDuration, [numTasks]) A more efficient version of the above reduceByKeyAndWindow() where the reduce value of each window is calculated + incrementally using the reduce values of the previous window. This is done by reducing the new data that enter the sliding window, and "inverse reducing" the old data that leave the window. An example would be that of "adding" and "subtracting" counts of keys as the window slides. However, it is applicable to only "invertible reduce functions", that is, those reduce functions which have a corresponding "inverse reduce" function (taken as parameter invFunc. Like in groupByKeyAndWindow, the number of reduce tasks is configurable through an optional second argument. windowDuration and slideDuration are exactly as defined in window(). -
    countByValueAndWindow(windowDuration, slideDuration, [numTasks]) When called on a DStream of (K, V) pairs, returns a new DStream of (K, Long) pairs where the value of each key is its frequency within a sliding window. Like in groupByKeyAndWindow, the number of reduce tasks is configurable through an optional second argument. + windowDuration and slideDuration are exactly as defined in window(). +
    @@ -147,7 +166,7 @@ A complete list of DStream operations is available in the API documentation of [ When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined: - + @@ -176,11 +195,6 @@ When an output operator is called, it triggers the computation of a stream. Curr
    OperatorMeaning
    OperatorMeaning
    foreach(func) The fundamental output operator. Applies a function, func, to each RDD generated from the stream. This function should have side effects, such as printing output, saving the RDD to external files, or writing it over the network to an external system.
    -## DStream Persistence -Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple DStream operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. - -Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence). - # Starting the Streaming computation All the above DStream operations are completely lazy, that is, the operations will start executing only after the context is started by using {% highlight scala %} @@ -192,8 +206,8 @@ Conversely, the computation can be stopped by using ssc.stop() {% endhighlight %} -# Example - NetworkWordCount.scala -A good example to start off is the spark.streaming.examples.NetworkWordCount. This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in /streaming/src/main/scala/spark/streaming/examples/WordCountNetwork.scala. +# Example +A simple example to start off is the [NetworkWordCount](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/spark/streaming/examples/WordCountNetwork.scala` . {% highlight scala %} import spark.streaming.{Seconds, StreamingContext} @@ -260,6 +274,31 @@ Time: 1357008430000 ms +You can find more examples in `/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. + +# DStream Persistence +Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. + +For input streams that receive data from the network (that is, subclasses of NetworkInputDStream like FlumeInputDStream and KafkaInputDStream), the default persistence level is set to replicate the data to two nodes for fault-tolerance. + +Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence). + +# RDD Checkpointing within DStreams +DStreams created by stateful operations like `updateStateByKey` require the RDDs in the DStream to be periodically saved to HDFS files for checkpointing. This is because, unless checkpointed, the lineage of operations of the state RDDs can increase indefinitely (since each RDD in the DStream depends on the previous RDD). This leads to two problems - (i) the size of Spark tasks increase proportionally with the RDD lineage leading higher task launch times, (ii) no limit on the amount of recomputation required on failure. Checkpointing RDDs at some interval by writing them to HDFS allows the lineage to be truncated. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. + +To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using + +{% highlight scala %} +ssc.checkpoint(hdfsPath) // assuming ssc is the StreamingContext +{% endhighlight %} + +The interval of checkpointing of a DStream can be set by using + +{% highlight scala %} +dstream.checkpoint(checkpointInterval) // checkpointInterval must be a multiple of slide duration of dstream +{% endhighlight %} + +For DStreams that must be checkpointed (that is, DStreams created by `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds. # Performance Tuning @@ -273,17 +312,21 @@ Getting the best performance of a Spark Streaming application on a cluster requi There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones. ### Level of Parallelism -Cluster resources maybe underutilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`spark.PairDStreamFunctions`](api/streaming/index.html#spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. +Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`spark.PairDStreamFunctions`](api/streaming/index.html#spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. ### Data Serialization The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it. -* Serialization of RDD data in Spark: Please refer to the detailed discussion on data serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default RDDs are persisted as serialized byte arrays to minimize pauses related to GC. -* Serialization of input data: To ingest external data into Spark, data received as bytes (say, from the network) needs to deserialized from bytes and re-serialized into Spark's serialization format. Hence, the deserialization overhead of input data may be a bottleneck. + +* **Serialization of RDD data in Spark**: Please refer to the detailed discussion on data serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default RDDs are persisted as serialized byte arrays to minimize pauses related to GC. + +* **Serialization of input data**: To ingest external data into Spark, data received as bytes (say, from the network) needs to deserialized from bytes and re-serialized into Spark's serialization format. Hence, the deserialization overhead of input data may be a bottleneck. ### Task Launching Overheads If the number of tasks launched per second is high (say, 50 or more per second), then the overhead of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: -* Task Serialization: Using Kryo serialization for serializing tasks can reduced the task sizes, and therefore reduce the time taken to send them to the slaves. -* Execution mode: Running Spark in Standalone mode or coarse-grained Mesos mode leads to better task launch times than the fine-grained Mesos mode. Please refer to the [Running on Mesos guide](running-on-mesos.html) for more details. + +* **Task Serialization**: Using Kryo serialization for serializing tasks can reduced the task sizes, and therefore reduce the time taken to send them to the slaves. + +* **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to better task launch times than the fine-grained Mesos mode. Please refer to the [Running on Mesos guide](running-on-mesos.html) for more details. These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable. ## Setting the Right Batch Size @@ -292,22 +335,121 @@ For a Spark Streaming application running on a cluster to be stable, the process A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). ## 24/7 Operation -By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.delay` to the number of minutes you want any metadata to persist. For example, setting `spark.cleaner.delay` to 10 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created. +By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.delay` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.delay` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created. This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so. ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, we highlight a few customizations that are strongly recommended to minimize GC related pauses in Spark Streaming applications and achieving more consistent batch processing times. -* Default persistence level of DStreams: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses. +* **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses. + +* **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times. -* Concurrent garbage collector: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times. +# Fault-tolerance Properties +There are two aspects to fault-tolerance - failure of a worker node and that of a driver node. In this section, we are going to discuss the fault-tolerance behavior and the semantics of the processed data. -# Master Fault-tolerance (Alpha) -TODO +## Failure of a Worker Node +In case of the worker node failure, none of the processed data will be lost because -* Checkpointing of DStream graph +1. All the input data is fault-tolerant (either the data is on HDFS, or it replicated Spark Streaming if received from the network) +1. All intermediate data is expressed as RDDs with their lineage to the input data, which allows Spark to recompute any part of the intermediate data is lost to worker node failure. + +If the worker node where a network data receiver is running fails, then the receiver will be restarted on a different node and it will continue to receive data. However, data that was accepted by the receiver but not yet replicated to other Spark nodes may be lost, which is a fraction of a second of data. + +Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation always leads to the same result. As a result, all DStream transformations are guaranteed to have _exactly-once_ semantics. That is, the final transformed result will be same even if there were was a worker node failure. However, output operations (like `foreach`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to HDFS using the `saveAs*Files` operations (as the file will simply get over-written by the same data), additional transactions-like mechanisms may be necessary to achieve exactly-once semantics for output operations. + +## Failure of a Driver Node +A system that is required to operate 24/7 needs to be able tolerate the failure of the drive node as well. Spark Streaming does this by saving the state of the DStream computation periodically to a HDFS file, that can be used to restart the streaming computation in the event of a failure of the driver node. To elaborate, the following state is periodically saved to a file. + +1. The DStream operator graph (input streams, output streams, etc.) +1. The configuration of each DStream (checkpoint interval, etc.) +1. The RDD checkpoint files of each DStream + +All this is periodically saved in the file `/graph` where `` is the HDFS path set using `ssc.checkpoint(...)` as described earlier. To recover, a new Streaming Context can be created with this directory by using + +{% highlight scala %} +val ssc = new StreamingContext(checkpointDirectory) +{% endhighlight %} + +Calling `ssc.start()` on this new context will restart the receivers and the stream computations. + +In case of stateful operations (that is, `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the intermediate data at the time of failure also needs to be recomputed.This requires two things - (i) the RDD checkpoints and (ii) the data received since the checkpoints. In the current _alpha_ release, the input data received from the network is not saved durably across driver failures (the data is only replicated in memory of the worker processes and gets lost when the driver fails). Only with file input streams (where the data is already durably stored) is the recovery from driver failure complete and all intermediate data is recomputed. In a future release, this will be true for all input streams. Note that for non-stateful operations, with _all_ input streams, the system will recover and continue receiving and processing new data. + +To understand the behavior of the system under driver failure, lets consider what will happen with a file input stream Specifically, in the case of the file input stream, it will correctly identify new files that were created while the driver was down and process them in the same way as it would have if the driver had not failed. To explain further in the case of file input stream, we shall use an example. Lets say, files are being generated every second, and a Spark Streaming program reads every new file and output the number of lines in the file. This is what the sequence of outputs would be with and without a driver failure. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Time Number of lines in input file Output without driver failure Output with driver failure
    1101010
    2202020
    3303030
    44040[DRIVER FAILS]
    no output
    55050no output
    66060no output
    77070[DRIVER RECOVERS]
    40, 50, 60, 70
    8808080
    9909090
    10100100100
    -* Recovery from master faults +If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery. -* Current state and future directions \ No newline at end of file +# Where to Go from Here +* Documentation - [Scala and Java](api/streaming/index.html) +* More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) \ No newline at end of file diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 84e4b5bedb8df..e1be5ef51cc9c 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -132,7 +132,7 @@ abstract class DStream[T: ClassManifest] ( // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger if (mustCheckpoint && checkpointDuration == null) { - checkpointDuration = slideDuration.max(Seconds(10)) + checkpointDuration = slideDuration * math.ceil(Seconds(10) / slideDuration).toInt logInfo("Checkpoint interval automatically set to " + checkpointDuration) } diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala index e4dc579a170a4..ee26206e249a8 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/spark/streaming/Duration.scala @@ -16,7 +16,7 @@ case class Duration (private val millis: Long) { def * (times: Int): Duration = new Duration(millis * times) - def / (that: Duration): Long = millis / that.millis + def / (that: Duration): Double = millis.toDouble / that.millis.toDouble def isMultipleOf(that: Duration): Boolean = (this.millis % that.millis == 0) diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala index dc21dfb722288..6a8b81760e35b 100644 --- a/streaming/src/main/scala/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/spark/streaming/Interval.scala @@ -30,6 +30,7 @@ class Interval(val beginTime: Time, val endTime: Time) { override def toString = "[" + beginTime + ", " + endTime + "]" } +private[streaming] object Interval { def currentInterval(duration: Duration): Interval = { val time = new Time(System.currentTimeMillis) diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 5127db3bbcd65..5a2dd46fa0863 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -18,8 +18,8 @@ import org.apache.hadoop.conf.Configuration class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)]) extends Serializable { - - def ssc = self.ssc + + private[streaming] def ssc = self.ssc private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = { new HashPartitioner(numPartitions) @@ -242,7 +242,9 @@ extends Serializable { * Return a new DStream by applying incremental `reduceByKey` over a sliding window. * The reduced value of over a new window is calculated using the old window's reduced value : * 1. reduce the new values that entered the window (e.g., adding new counts) + * * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + * * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. @@ -399,7 +401,7 @@ extends Serializable { } /** - * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this` + * Cogroup `this` DStream with `other` DStream using a partitioner. For each key k in corresponding RDDs of `this` * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that * key in both RDDs. Partitioner is used to partition each generated RDD. */ diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index e70822e5c3bd3..0e21b7480c6c6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -13,6 +13,7 @@ import twitter4j.auth.BasicAuthorization * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. */ +private[streaming] class TwitterInputDStream( @transient ssc_ : StreamingContext, username: String, @@ -26,6 +27,7 @@ class TwitterInputDStream( } } +private[streaming] class TwitterReceiver( username: String, password: String, diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index 5250667bcbc0d..cac86deeaf349 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -50,7 +50,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { val stateStreamCheckpointInterval = Seconds(1) // this ensure checkpointing occurs at least once - val firstNumBatches = (stateStreamCheckpointInterval / batchDuration) * 2 + val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2 val secondNumBatches = firstNumBatches // Setup the streams From 12ea14c211da908a278ab19fd1e9f6acd45daae8 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 18 Feb 2013 15:18:34 -0800 Subject: [PATCH 238/696] Changed networkStream to socketStream and pluggableNetworkStream to become networkStream as a way to create streams from arbitrary network receiver. --- docs/streaming-programming-guide.md | 10 +++---- .../examples/JavaNetworkWordCount.java | 2 +- .../examples/AkkaActorWordCount.scala | 12 ++++----- .../streaming/examples/NetworkWordCount.scala | 2 +- .../examples/clickstream/PageViewStream.scala | 2 +- .../spark/streaming/StreamingContext.scala | 26 ++++++++++--------- .../api/java/JavaStreamingContext.scala | 10 +++---- .../java/spark/streaming/JavaAPISuite.java | 5 ++-- .../spark/streaming/InputStreamsSuite.scala | 2 +- 9 files changed, 36 insertions(+), 35 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index d408e80359c46..71e1bd4aab20f 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -34,16 +34,16 @@ The StreamingContext is used to creating InputDStreams from input sources: {% highlight scala %} // Assuming ssc is the StreamingContext -ssc.networkStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port -ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory +ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory +ssc.socketStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port {% endhighlight %} -A complete list of input sources is available in the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext). Data received from these sources can be processed using DStream operations, which are explained next. +We also provide a input streams for Kafka, Flume, Akka actor, etc. For a complete list of input streams, take a look at the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext). # DStream Operations -Once an input DStream has been created, you can transform it using _DStream operators_. Most of these operators return new DStreams which you can further transform. Eventually, you'll need to call an _output operator_, which forces evaluation of the DStream by writing data out to an external source. +Data received from the input streams can be processed using _DStream operations_. There are two kinds of operations - _transformations_ and _output operations_. Similar to RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams with transformed data. After applying a sequence of transformations to the input streams, you'll need to call the output operations, which writies data out to an external source. ## Transformations @@ -452,4 +452,4 @@ If the driver had crashed in the middle of the processing of time 3, then it wil # Where to Go from Here * Documentation - [Scala and Java](api/streaming/index.html) -* More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) \ No newline at end of file +* More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java index 4299febfd6ca3..07342beb02ea6 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java @@ -35,7 +35,7 @@ public static void main(String[] args) { // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') - JavaDStream lines = ssc.networkTextStream(args[1], Integer.parseInt(args[2])); + JavaDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2])); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { diff --git a/examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala index ff05842c71f0d..553afc2024784 100644 --- a/examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/AkkaActorWordCount.scala @@ -36,8 +36,8 @@ class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) } /** - * A sample word count program demonstrating the use of plugging in - * AkkaActor as Receiver + * A sample word count program demonstrating the use of Akka actor stream. + * */ object AkkaActorWordCount { def main(args: Array[String]) { @@ -56,18 +56,18 @@ object AkkaActorWordCount { Seconds(batchDuration.toLong)) /* - * Following is the use of pluggableActorStream to plug in custom actor as receiver + * Following is the use of actorStream to plug in custom actor as receiver * * An important point to note: * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e type of data received and PluggableInputDstream + * to ensure the type safety, i.e type of data received and actorStream * should be same. * - * For example: Both pluggableActorStream and SampleActorReceiver are parameterized + * For example: Both actorStream and SampleActorReceiver are parameterized * to same type to ensure type safety. */ - val lines = ssc.pluggableActorStream[String]( + val lines = ssc.actorStream[String]( Props(new SampleActorReceiver[String]("akka://spark@%s:%s/user/FeederActor".format( remoteAkkaHost, remoteAkkaPort.toInt))), "SampleReceiver") diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 32f7d57bea485..7ff70ae2e57eb 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -27,7 +27,7 @@ object NetworkWordCount { // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') - val lines = ssc.networkTextStream(args(1), args(2).toInt) + val lines = ssc.socketTextStream(args(1), args(2).toInt) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index 60f228b8adae4..fba72519a9244 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -27,7 +27,7 @@ object PageViewStream { val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1)) // Create a NetworkInputDStream on target host:port and convert each line to a PageView - val pageViews = ssc.networkTextStream(host, port) + val pageViews = ssc.socketTextStream(host, port) .flatMap(_.split("\n")) .map(PageView.fromString(_)) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 48d344f0555d8..a426649726461 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -146,7 +146,7 @@ class StreamingContext private ( * Create an input stream with any arbitrary user implemented network receiver. * @param receiver Custom implementation of NetworkReceiver */ - def pluggableNetworkStream[T: ClassManifest]( + def networkStream[T: ClassManifest]( receiver: NetworkReceiver[T]): DStream[T] = { val inputStream = new PluggableInputDStream[T](this, receiver) @@ -155,15 +155,16 @@ class StreamingContext private ( } /** - * Create an input stream with any arbitrary user implemented akka actor receiver. + * Create an input stream with any arbitrary user implemented Akka actor receiver. * @param props Props object defining creation of the actor * @param name Name of the actor - * @param storageLevel RDD storage level. Defaults to memory-only. + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ - def pluggableActorStream[T: ClassManifest]( + def actorStream[T: ClassManifest]( props: Props, name: String, storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2): DStream[T] = { - pluggableNetworkStream(new ActorReceiver(Settings(props, name, storageLevel))) + networkStream(new ActorReceiver(Settings(props, name, storageLevel))) } /** @@ -174,7 +175,8 @@ class StreamingContext private ( * in its own thread. * @param initialOffsets Optional initial offsets for each of the partitions to consume. * By default the value is pulled from zookeper. - * @param storageLevel RDD storage level. Defaults to memory-only. + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ def kafkaStream[T: ClassManifest]( zkQuorum: String, @@ -189,24 +191,24 @@ class StreamingContext private ( } /** - * Create a input stream from network source hostname:port. Data is received using - * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited + * Create a input stream from TCP source hostname:port. Data is received using + * a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited * lines. * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ - def networkTextStream( + def socketTextStream( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[String] = { - networkStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel) + socketStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel) } /** - * Create a input stream from network source hostname:port. Data is received using + * Create a input stream from TCP source hostname:port. Data is received using * a TCP socket and the receive bytes it interepreted as object using the given * converter. * @param hostname Hostname to connect to for receiving data @@ -215,7 +217,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * @tparam T Type of the objects received (after converting bytes to objects) */ - def networkStream[T: ClassManifest]( + def socketStream[T: ClassManifest]( hostname: String, port: Int, converter: (InputStream) => Iterator[T], diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 03933aae93680..d9a676819aa6a 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -130,7 +130,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { */ def networkTextStream(hostname: String, port: Int, storageLevel: StorageLevel) : JavaDStream[String] = { - ssc.networkTextStream(hostname, port, storageLevel) + ssc.socketTextStream(hostname, port, storageLevel) } /** @@ -140,8 +140,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data */ - def networkTextStream(hostname: String, port: Int): JavaDStream[String] = { - ssc.networkTextStream(hostname, port) + def socketTextStream(hostname: String, port: Int): JavaDStream[String] = { + ssc.socketTextStream(hostname, port) } /** @@ -154,7 +154,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel Storage level to use for storing the received objects * @tparam T Type of the objects received (after converting bytes to objects) */ - def networkStream[T]( + def socketStream[T]( hostname: String, port: Int, converter: JFunction[InputStream, java.lang.Iterable[T]], @@ -163,7 +163,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def fn = (x: InputStream) => converter.apply(x).toIterator implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.networkStream(hostname, port, fn, storageLevel) + ssc.socketStream(hostname, port, fn, storageLevel) } /** diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 16bacffb9285f..5d510fd89f141 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -23,7 +23,6 @@ import spark.streaming.dstream.KafkaPartitionKey; import java.io.*; -import java.text.Collator; import java.util.*; // The test suite itself is Serializable so that anonymous Function implementations can be @@ -984,7 +983,7 @@ public void testKafkaStream() { @Test public void testNetworkTextStream() { - JavaDStream test = ssc.networkTextStream("localhost", 12345); + JavaDStream test = ssc.socketTextStream("localhost", 12345); } @Test @@ -1004,7 +1003,7 @@ public Iterable call(InputStream in) { } } - JavaDStream test = ssc.networkStream( + JavaDStream test = ssc.socketStream( "localhost", 12345, new Converter(), diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 7c1c2e1040073..e6aecfbb76f72 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -41,7 +41,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(master, framework, batchDuration) - val networkStream = ssc.networkTextStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) + val networkStream = ssc.socketTextStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) From 03f45a18d57ea1eceeb688154c01a1d460744600 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 18 Feb 2013 16:56:01 -0800 Subject: [PATCH 239/696] Use port 5080 for httpd/ganglia --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7967bcac50822..89cfbad876c65 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -179,7 +179,7 @@ def launch_cluster(conn, opts, cluster_name): if opts.cluster_type == "mesos": master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') if opts.ganglia: - master_group.authorize('tcp', 80, 80, '0.0.0.0/0') + master_group.authorize('tcp', 5080, 5080, '0.0.0.0/0') if slave_group.rules == []: # Group was just now created slave_group.authorize(src_group=master_group) slave_group.authorize(src_group=slave_group) From e7cdf7a6a496ee76a2c53ac27514c469d929471e Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 18 Feb 2013 17:15:22 -0800 Subject: [PATCH 240/696] Print ganglia url after setup --- ec2/spark_ec2.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 89cfbad876c65..6056498577d56 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -415,6 +415,8 @@ def setup_standalone_cluster(master, slave_nodes, opts): def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") + if opts.ganglia: + print "Ganglia started at http://%s:5080/ganglia" % master # Wait for a whole cluster (masters, slaves and ZooKeeper) to start up From 6cba5a48b0bc1a90dd7a914a82636fcd33294822 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 18 Feb 2013 18:30:36 -0800 Subject: [PATCH 241/696] Print cluster url after setup completes --- ec2/spark_ec2.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 6056498577d56..66b1faf2cd831 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -415,6 +415,11 @@ def setup_standalone_cluster(master, slave_nodes, opts): def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") + if opts.cluster_type == "mesos": + print "Mesos cluster started at http://%s:8080" % master + elif opts.cluster_type == "standalone": + print "Spark standalone cluster started at http://%s:8080" % master + if opts.ganglia: print "Ganglia started at http://%s:5080/ganglia" % master From 7e30c46aaf337eb95c9ec37ddc2ad79439430c96 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 19 Feb 2013 03:05:44 -0800 Subject: [PATCH 242/696] Added comment to the KafkaWordCount, given by Sean McNamara. --- .../spark/streaming/examples/KafkaWordCount.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 65d5da82fcccc..9b135a5c54cf3 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -10,6 +10,18 @@ import spark.streaming.StreamingContext._ import spark.storage.StorageLevel import spark.streaming.util.RawTextHelper._ +/** + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: KafkaWordCount + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * is a list of one or more zookeeper servers that make quorum + * is the name of kafka consumer group + * is a list of one or more kafka topics to consume from + * is the number of threads the kafka consumer should use + * + * Example: + * `./run spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + */ object KafkaWordCount { def main(args: Array[String]) { From 015893f0e8983a7e249709d9820d1bf0dd74d607 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 19 Feb 2013 13:21:33 +0200 Subject: [PATCH 243/696] Adding streaming HyperLogLog example using Algebird --- examples/pom.xml | 5 ++ .../examples/twitter/StreamingHLL.scala | 62 +++++++++++++++++++ project/SparkBuild.scala | 3 +- 3 files changed, 69 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/spark/streaming/examples/twitter/StreamingHLL.scala diff --git a/examples/pom.xml b/examples/pom.xml index f43af670c613f..28da3dbde4a1b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -24,6 +24,11 @@ twitter4j-stream 3.0.3 + + com.twitter + algebird-core_2.9.2 + 0.1.8 + org.scalatest diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/StreamingHLL.scala b/examples/src/main/scala/spark/streaming/examples/twitter/StreamingHLL.scala new file mode 100644 index 0000000000000..f67bb029c6480 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/twitter/StreamingHLL.scala @@ -0,0 +1,62 @@ +package spark.streaming.examples.twitter + +import spark.streaming.{Seconds, StreamingContext} +import spark.storage.StorageLevel +import com.twitter.algebird.HyperLogLog._ +import com.twitter.algebird.HyperLogLogMonoid + +/** + * Example of using HyperLogLog monoid from Twitter's Algebird together with Spark Streaming's + * TwitterInputDStream + */ +object StreamingHLL { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: TwitterStreamingHLL " + + " [filter1] [filter2] ... [filter n]") + System.exit(1) + } + + val Array(master, username, password) = args.slice(0, 3) + val filters = args.slice(3, args.length) + + val ssc = new StreamingContext(master, "TwitterStreamingHLL", Seconds(2)) + val stream = new TwitterInputDStream(ssc, username, password, filters, + StorageLevel.MEMORY_ONLY_SER) + ssc.registerInputStream(stream) + + val users = stream.map(status => status.getUser.getId) + + val globalHll = new HyperLogLogMonoid(12) + var userSet: Set[Long] = Set() + + val approxUsers = users.mapPartitions(ids => { + val hll = new HyperLogLogMonoid(12) + ids.map(id => hll(id)) + }).reduce(_ + _) + + val exactUsers = users.map(id => Set(id)).reduce(_ ++ _) + + var h = globalHll.zero + approxUsers.foreach(rdd => { + if (rdd.count() != 0) { + val partial = rdd.first() + h += partial + println("Approx distinct users this batch: %d".format(partial.estimatedSize.toInt)) + println("Approx distinct users overall: %d".format(globalHll.estimateSize(h).toInt)) + } + }) + + exactUsers.foreach(rdd => { + if (rdd.count() != 0) { + val partial = rdd.first() + userSet ++= partial + println("Exact distinct users this batch: %d".format(partial.size)) + println("Exact distinct users overall: %d".format(userSet.size)) + println("Error rate: %2.5f%%".format(((globalHll.estimateSize(h) / userSet.size.toDouble) - 1) * 100)) + } + }) + + ssc.start() + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index af8b5ba01745b..18cc9ea90e069 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -156,7 +156,8 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", libraryDependencies ++= Seq( - "org.twitter4j" % "twitter4j-stream" % "3.0.3" + "org.twitter4j" % "twitter4j-stream" % "3.0.3", + "com.twitter" % "algebird-core_2.9.2" % "0.1.8" ) ) From f7d3e309cb76ef208ab51f23c90c5e891fb333a3 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 8 Feb 2013 16:56:42 +0530 Subject: [PATCH 244/696] ZeroMQ stream as receiver --- docs/zeroMQ-intro.md | 59 +++++++++++++++++++ project/SparkBuild.scala | 1 + .../spark/streaming/StreamingContext.scala | 22 +++++++ .../streaming/receivers/ZeroMQReceiver.scala | 33 +++++++++++ 4 files changed, 115 insertions(+) create mode 100644 docs/zeroMQ-intro.md create mode 100644 streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala diff --git a/docs/zeroMQ-intro.md b/docs/zeroMQ-intro.md new file mode 100644 index 0000000000000..0365bc08fd6ff --- /dev/null +++ b/docs/zeroMQ-intro.md @@ -0,0 +1,59 @@ +--- +layout: global +title: ZeroMQ Stream setup guide +--- + +## Install ZeroMQ (using JNA) + +To work with zeroMQ, some native libraries have to be installed. + +* Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide](http://www.zeromq.org/intro:get-the-software) + + Typically if you are using ubuntu 12.04, you can do: + + `$ sudo apt-get install libzmq1` + + __To work with akka-zeromq, zmq 2.1 version is supported via [JNA](https://github.com/twall/jna). Incase you want to switch to zeromq 3.0, please install [JZMQ](http://www.zeromq.org/bindings:java) which uses [JNI](http://docs.oracle.com/javase/6/docs/technotes/guides/jni/) and drop in jzmq jar__ + +## Sample scala code + +A publisher is an entity assumed to be outside the spark ecosystem. A sample zeroMQ publisher is provided to try out the sample spark ZeroMQ application. + +1. Start the sample publisher. + +{% highlight scala %} + + + val acs: ActorSystem = ActorSystem() + + val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) + + pubSocket ! ZMQMessage(Seq(Frame("topic"), Frame("My message".getBytes))) + + + +{% endhighlight %} + +A typical zeromq url looks like `tcp://:` + +It does nothing more than publishing the message on the specified topic and url. + +2. Start the spark application by plugging the zeroMQ stream receiver. + +{% highlight scala %} + + val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToObjectsIterator) + +{% endhighlight %} + +bytesToObjectsIterator is going to be a function for decoding the Frame data. + +_For example: For decoding into strings using default charset:_ + + +{% highlight scala %} + + + def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator + +{% endhighlight %} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c6d3cc8b1540f..5fe85a28c3cf7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -134,6 +134,7 @@ object SparkBuild extends Build { "com.typesafe.akka" % "akka-actor" % "2.0.3", "com.typesafe.akka" % "akka-remote" % "2.0.3", "com.typesafe.akka" % "akka-slf4j" % "2.0.3", + "com.typesafe.akka" % "akka-zeromq" % "2.0.3", "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "cc.spray" % "spray-can" % "1.0-M2.1", diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index a9684c577299b..8c772aec6ebb8 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -2,12 +2,14 @@ package spark.streaming import akka.actor.Props import akka.actor.SupervisorStrategy +import akka.zeromq.Subscribe import spark.streaming.dstream._ import spark.{RDD, Logging, SparkEnv, SparkContext} import spark.streaming.receivers.ActorReceiver import spark.streaming.receivers.ReceiverSupervisorStrategy +import spark.streaming.receivers.ZeroMQReceiver import spark.storage.StorageLevel import spark.util.MetadataCleaner import spark.streaming.receivers.ActorReceiver @@ -174,6 +176,26 @@ class StreamingContext private ( networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) } + /** + * ZeroMQ stream receiver + * @param publisherUrl Url of remote zeromq publisher + * @param zeroMQ topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. Defaults to memory-only. + */ + def zeroMQStream[T: ClassManifest](publisherUrl:String, + subscribe: Subscribe, + bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T], + storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = { + + actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)), + "ZeroMQReceiver", storageLevel, supervisorStrategy) + } + /** * Create an input stream that pulls messages form a Kafka Broker. * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala new file mode 100644 index 0000000000000..5533c3cf1ef8b --- /dev/null +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -0,0 +1,33 @@ +package spark.streaming.receivers + +import akka.actor.Actor +import akka.zeromq._ + +import spark.Logging + +/** + * A receiver to subscribe to ZeroMQ stream. + */ +private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T]) + extends Actor with Receiver with Logging { + + override def preStart() = context.system.newSocket(SocketType.Sub, Listener(self), + Connect(publisherUrl), subscribe) + + def receive: Receive = { + + case Connecting ⇒ logInfo("connecting ...") + + case m: ZMQMessage ⇒ + logDebug("Received message for:" + m.firstFrameAsString) + + //We ignore first frame for processing as it is the topic + val bytes = m.frames.tail.map(_.payload) + pushBlock(bytesToObjects(bytes)) + + case Closed ⇒ logInfo("received closed ") + + } +} From 8d44480d840079cb444b5e19511e5027dedd7f77 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 19 Feb 2013 19:42:14 +0530 Subject: [PATCH 245/696] example for demonstrating ZeroMQ stream --- .../streaming/examples/ZeroMQWordCount.scala | 70 +++++++++++++++++++ .../spark/streaming/StreamingContext.scala | 15 ++-- 2 files changed, 77 insertions(+), 8 deletions(-) create mode 100644 examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala new file mode 100644 index 0000000000000..ab7b67ed4b55f --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -0,0 +1,70 @@ +package spark.streaming.examples + +import akka.actor.ActorSystem +import akka.actor.actorRef2Scala +import akka.zeromq._ +import spark.streaming.{ Seconds, StreamingContext } +import spark.streaming.StreamingContext._ +import akka.zeromq.Subscribe + +/** + * A simple publisher for demonstration purposes, repeatedly publishes random Messages + * every one second. + */ +object SimpleZeroMQPublisher { + + def main(args: Array[String]) = { + if (args.length < 2) { + System.err.println("Usage: SimpleZeroMQPublisher ") + System.exit(1) + } + + val Seq(url, topic) = args.toSeq + val acs: ActorSystem = ActorSystem() + + val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) + val messages: Array[String] = Array("words ", "may ", "count ") + while (true) { + Thread.sleep(1000) + pubSocket ! ZMQMessage(Frame(topic) :: messages.map(x => Frame(x.getBytes)).toList) + } + acs.awaitTermination() + } +} + +/** + * A sample wordcount with ZeroMQStream stream + * + * Usage: WordCountZeroMQ + * In local mode, should be 'local[n]' with n > 1 + * and describe where zeroMq publisher is running. + * + * To run this example locally, you may run publisher as + * `$ ./run spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * and then run the example + * `$ ./run spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + */ +object ZeroMQWordCount { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println( + "Usage: WordCountZeroMQ " + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + val Seq(master, url, topic) = args.toSeq + + // Create the context and set the batch size + val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2)) + + def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator + + //For this stream, a zeroMQ publisher should be running. + val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + wordCounts.print() + ssc.start() + } + +} \ No newline at end of file diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 8c772aec6ebb8..f15e6bd23d79d 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -163,7 +163,7 @@ class StreamingContext private ( * @param props Props object defining creation of the actor * @param name Name of the actor * @param storageLevel RDD storage level. Defaults to memory-only. - * + * * @note An important point to note: * Since Actor may exist outside the spark framework, It is thus user's responsibility * to ensure the type safety, i.e parametrized type of data received and actorStream @@ -181,9 +181,9 @@ class StreamingContext private ( * @param publisherUrl Url of remote zeromq publisher * @param zeroMQ topic to subscribe to * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. * @param storageLevel RDD storage level. Defaults to memory-only. */ def zeroMQStream[T: ClassManifest](publisherUrl:String, @@ -191,11 +191,11 @@ class StreamingContext private ( bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T], storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = { - - actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)), + + actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)), "ZeroMQReceiver", storageLevel, supervisorStrategy) } - + /** * Create an input stream that pulls messages form a Kafka Broker. * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). @@ -500,4 +500,3 @@ object StreamingContext { new Path(sscCheckpointDir, UUID.randomUUID.toString).toString } } - From d8ee184d950ac91ad4f617fc0d7b2f02ed6b74d3 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 19 Feb 2013 17:42:57 +0200 Subject: [PATCH 246/696] Dependencies and refactoring for streaming HLL example, and using context.twitterStream method --- examples/pom.xml | 6 ---- ...mingHLL.scala => TwitterAlgebirdHLL.scala} | 29 +++++++++---------- streaming/pom.xml | 10 +++++++ 3 files changed, 24 insertions(+), 21 deletions(-) rename examples/src/main/scala/spark/streaming/examples/{twitter/StreamingHLL.scala => TwitterAlgebirdHLL.scala} (64%) diff --git a/examples/pom.xml b/examples/pom.xml index 28da3dbde4a1b..7d975875fac3a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -19,17 +19,11 @@ org.eclipse.jetty jetty-server - - org.twitter4j - twitter4j-stream - 3.0.3 - com.twitter algebird-core_2.9.2 0.1.8 - org.scalatest scalatest_${scala.version} diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/StreamingHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala similarity index 64% rename from examples/src/main/scala/spark/streaming/examples/twitter/StreamingHLL.scala rename to examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 023a0add804d2..c2095f5b9459b 100644 --- a/examples/src/main/scala/spark/streaming/examples/twitter/StreamingHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -1,4 +1,4 @@ -package spark.streaming.examples.twitter +package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} import spark.storage.StorageLevel @@ -7,44 +7,43 @@ import com.twitter.algebird.HyperLogLogMonoid import spark.streaming.dstream.TwitterInputDStream /** - * Example of using HyperLogLog monoid from Twitter's Algebird together with Spark Streaming's - * TwitterInputDStream + * Example using HyperLogLog monoid from Twitter's Algebird together with Spark Streaming's + * TwitterInputDStream to compute approximate distinct counts of userids. */ -object StreamingHLL { +object TwitterAlgebirdHLL { def main(args: Array[String]) { if (args.length < 3) { - System.err.println("Usage: TwitterStreamingHLL " + + System.err.println("Usage: TwitterAlgebirdHLL " + " [filter1] [filter2] ... [filter n]") System.exit(1) } + /** Bit size parameter for HyperLogLog */ + val BIT_SIZE = 12 val Array(master, username, password) = args.slice(0, 3) val filters = args.slice(3, args.length) - val ssc = new StreamingContext(master, "TwitterStreamingHLL", Seconds(2)) - val stream = new TwitterInputDStream(ssc, username, password, filters, - StorageLevel.MEMORY_ONLY_SER) - ssc.registerInputStream(stream) + val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5)) + val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) - val globalHll = new HyperLogLogMonoid(12) + var globalHll = new HyperLogLogMonoid(BIT_SIZE).zero var userSet: Set[Long] = Set() val approxUsers = users.mapPartitions(ids => { - val hll = new HyperLogLogMonoid(12) + val hll = new HyperLogLogMonoid(BIT_SIZE) ids.map(id => hll(id)) }).reduce(_ + _) val exactUsers = users.map(id => Set(id)).reduce(_ ++ _) - var h = globalHll.zero approxUsers.foreach(rdd => { if (rdd.count() != 0) { val partial = rdd.first() - h += partial + globalHll += partial println("Approx distinct users this batch: %d".format(partial.estimatedSize.toInt)) - println("Approx distinct users overall: %d".format(globalHll.estimateSize(h).toInt)) + println("Approx distinct users overall: %d".format(globalHll.estimatedSize.toInt)) } }) @@ -54,7 +53,7 @@ object StreamingHLL { userSet ++= partial println("Exact distinct users this batch: %d".format(partial.size)) println("Exact distinct users overall: %d".format(userSet.size)) - println("Error rate: %2.5f%%".format(((globalHll.estimateSize(h) / userSet.size.toDouble) - 1) * 100)) + println("Error rate: %2.5f%%".format(((globalHll.estimatedSize / userSet.size.toDouble) - 1) * 100)) } }) diff --git a/streaming/pom.xml b/streaming/pom.xml index 6ee7e59df39d1..d78c39da0d1a6 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -47,6 +47,16 @@ zkclient 0.1 + + org.twitter4j + twitter4j-stream + 3.0.3 + + + org.twitter4j + twitter4j-core + 3.0.3 + org.scalatest From 8a281399f970db761ea05baf07972fff1c5bd058 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 19 Feb 2013 17:56:02 +0200 Subject: [PATCH 247/696] Streaming example using Twitter Algebird's Count Min Sketch monoid --- .../examples/TwitterAlgebirdCMS.scala | 78 +++++++++++++++++++ 1 file changed, 78 insertions(+) create mode 100644 examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala new file mode 100644 index 0000000000000..d50e86aff5d71 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -0,0 +1,78 @@ +package spark.streaming.examples + +import spark.streaming.{Seconds, StreamingContext} +import spark.storage.StorageLevel +import com.twitter.algebird._ +import spark.streaming.StreamingContext._ +import spark.SparkContext._ + +/** + * Example of using CountMinSketch monoid from Twitter's Algebird together with Spark Streaming's + * TwitterInputDStream + */ +object TwitterAlgebirdCMS { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: TwitterAlgebirdCMS " + + " [filter1] [filter2] ... [filter n]") + System.exit(1) + } + + val DELTA = 1E-3 + val EPS = 0.01 + val SEED = 1 + val PERC = 0.001 + val TOPK = 10 + + val Array(master, username, password) = args.slice(0, 3) + val filters = args.slice(3, args.length) + + val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10)) + val stream = ssc.twitterStream(username, password, filters, + StorageLevel.MEMORY_ONLY_SER) + + val users = stream.map(status => status.getUser.getId) + + var globalCMS = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC).zero + var globalExact = Map[Long, Int]() + val mm = new MapMonoid[Long, Int]() + + val approxTopUsers = users.mapPartitions(ids => { + val cms = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC) + ids.map(id => cms.create(id)) + }).reduce(_ ++ _) + + val exactTopUsers = users.map(id => (id, 1)) + .reduceByKey((a, b) => a + b) + + approxTopUsers.foreach(rdd => { + if (rdd.count() != 0) { + val partial = rdd.first() + val partialTopK = partial.heavyHitters.map(id => + (id, partial.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK) + globalCMS ++= partial + val globalTopK = globalCMS.heavyHitters.map(id => + (id, globalCMS.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK) + println("Approx heavy hitters at %2.2f%% threshold this batch: %s".format(PERC, + partialTopK.mkString("[", ",", "]"))) + println("Approx heavy hitters at %2.2f%% threshold overall: %s".format(PERC, + globalTopK.mkString("[", ",", "]"))) + } + }) + + exactTopUsers.foreach(rdd => { + if (rdd.count() != 0) { + val partialMap = rdd.collect().toMap + val partialTopK = rdd.map( + {case (id, count) => (count, id)}) + .sortByKey(ascending = false).take(TOPK) + globalExact = mm.plus(globalExact.toMap, partialMap) + val globalTopK = globalExact.toSeq.sortBy(_._2).reverse.slice(0, TOPK) + println("Exact heavy hitters this batch: %s".format(partialTopK.mkString("[", ",", "]"))) + println("Exact heavy hitters overall: %s".format(globalTopK.mkString("[", ",", "]"))) + } + }) + + ssc.start() + } +} From 9d49a6b03fb91d516bf40e50f67e87155c69dba1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 16 Feb 2013 15:56:04 -0800 Subject: [PATCH 248/696] Use RDD type for `foreach` operator in Java. --- .../spark/streaming/api/java/JavaDStream.scala | 5 ++++- .../streaming/api/java/JavaDStreamLike.scala | 15 +++++++++------ .../streaming/api/java/JavaPairDStream.scala | 4 +++- .../streaming/api/java/JavaStreamingContext.scala | 2 +- .../test/java/spark/streaming/JavaAPISuite.java | 1 + .../test/java/spark/streaming/JavaTestUtils.scala | 5 +++-- 6 files changed, 21 insertions(+), 11 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 30985b4ebc13b..51efe6cae8494 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -4,6 +4,7 @@ import spark.streaming.{Duration, Time, DStream} import spark.api.java.function.{Function => JFunction} import spark.api.java.JavaRDD import spark.storage.StorageLevel +import spark.RDD /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -26,7 +27,9 @@ import spark.storage.StorageLevel * - A function that is used to generate an RDD after each time interval */ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassManifest[T]) - extends JavaDStreamLike[T, JavaDStream[T]] { + extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] { + + override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) /** Return a new DStream containing only the elements that satisfy a predicate. */ def filter(f: JFunction[T, java.lang.Boolean]): JavaDStream[T] = diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 1c1ba05ff98b5..4e1458ca9e36e 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -6,17 +6,20 @@ import java.lang.{Long => JLong} import scala.collection.JavaConversions._ import spark.streaming._ -import spark.api.java.JavaRDD +import spark.api.java.{JavaRDDLike, JavaRDD} import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import java.util import spark.RDD import JavaDStream._ -trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable { +trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] + extends Serializable { implicit val classManifest: ClassManifest[T] def dstream: DStream[T] + def wrapRDD(in: RDD[T]): R + implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[JLong] = { in.map(new JLong(_)) } @@ -220,16 +223,16 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable * Apply a function to each RDD in this DStream. This is an output operator, so * this DStream will be registered as an output stream and therefore materialized. */ - def foreach(foreachFunc: JFunction[JavaRDD[T], Void]) { - dstream.foreach(rdd => foreachFunc.call(new JavaRDD(rdd))) + def foreach(foreachFunc: JFunction[R, Void]) { + dstream.foreach(rdd => foreachFunc.call(wrapRDD(rdd))) } /** * Apply a function to each RDD in this DStream. This is an output operator, so * this DStream will be registered as an output stream and therefore materialized. */ - def foreach(foreachFunc: JFunction2[JavaRDD[T], Time, Void]) { - dstream.foreach((rdd, time) => foreachFunc.call(new JavaRDD(rdd), time)) + def foreach(foreachFunc: JFunction2[R, Time, Void]) { + dstream.foreach((rdd, time) => foreachFunc.call(wrapRDD(rdd), time)) } /** diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 952ca657bf77b..de3e802300628 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -19,7 +19,9 @@ import com.google.common.base.Optional class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifiest: ClassManifest[K], implicit val vManifest: ClassManifest[V]) - extends JavaDStreamLike[(K, V), JavaPairDStream[K, V]] { + extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] { + + override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) // ======================================================================= // Methods common to all DStream's diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index d9a676819aa6a..878e179589c0b 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -254,7 +254,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Registers an output stream that will be computed every interval */ - def registerOutputStream(outputStream: JavaDStreamLike[_, _]) { + def registerOutputStream(outputStream: JavaDStreamLike[_, _, _]) { ssc.registerOutputStream(outputStream.dstream) } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 5d510fd89f141..4fe2de5a1a764 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -12,6 +12,7 @@ import scala.Tuple2; import spark.HashPartitioner; import spark.api.java.JavaRDD; +import spark.api.java.JavaRDDLike; import spark.api.java.JavaSparkContext; import spark.api.java.function.*; import spark.storage.StorageLevel; diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala index 52ea28732ada8..64a7e7cbf9a36 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala @@ -31,8 +31,9 @@ trait JavaTestBase extends TestSuiteBase { * Attach a provided stream to it's associated StreamingContext as a * [[spark.streaming.TestOutputStream]]. **/ - def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T,This]]( - dstream: JavaDStreamLike[T, This]) = { + def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T, This, R], + R <: spark.api.java.JavaRDDLike[T, R]]( + dstream: JavaDStreamLike[T, This, R]) = { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] val ostream = new TestOutputStream(dstream.dstream, From 35880de42edb30cf705036083710c85a74a351fa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 16 Feb 2013 16:36:12 -0800 Subject: [PATCH 249/696] Use RDD type for `transform` operator in Java. This is an improved implementation of the `transform` operator in Java. The main difference is that this allows all four possible types of transform functions 1. JavaRDD -> JavaRDD 2. JavaRDD -> JavaPairRDD 3. JavaPairRDD -> JavaPairRDD 4. JavaPairRDD -> JavaRDD whereas previously only (1) and (3) were possible. Conflicts: streaming/src/test/java/spark/streaming/JavaAPISuite.java --- .../streaming/api/java/JavaDStreamLike.scala | 40 +++++++-- .../java/spark/streaming/JavaAPISuite.java | 89 ++++++++++++++++++- 2 files changed, 122 insertions(+), 7 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 4e1458ca9e36e..f7b1704884b32 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -6,7 +6,7 @@ import java.lang.{Long => JLong} import scala.collection.JavaConversions._ import spark.streaming._ -import spark.api.java.{JavaRDDLike, JavaRDD} +import spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import java.util import spark.RDD @@ -239,11 +239,11 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD is generated by applying a function * on each RDD of this DStream. */ - def transform[U](transformFunc: JFunction[JavaRDD[T], JavaRDD[U]]): JavaDStream[U] = { + def transform[U](transformFunc: JFunction[R, JavaRDD[U]]): JavaDStream[U] = { implicit val cm: ClassManifest[U] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] def scalaTransform (in: RDD[T]): RDD[U] = - transformFunc.call(new JavaRDD[T](in)).rdd + transformFunc.call(wrapRDD(in)).rdd dstream.transform(scalaTransform(_)) } @@ -251,11 +251,41 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD is generated by applying a function * on each RDD of this DStream. */ - def transform[U](transformFunc: JFunction2[JavaRDD[T], Time, JavaRDD[U]]): JavaDStream[U] = { + def transform[U](transformFunc: JFunction2[R, Time, JavaRDD[U]]): JavaDStream[U] = { implicit val cm: ClassManifest[U] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] def scalaTransform (in: RDD[T], time: Time): RDD[U] = - transformFunc.call(new JavaRDD[T](in), time).rdd + transformFunc.call(wrapRDD(in), time).rdd + dstream.transform(scalaTransform(_, _)) + } + + /** + * Return a new DStream in which each RDD is generated by applying a function + * on each RDD of this DStream. + */ + def transform[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]): + JavaPairDStream[K2, V2] = { + implicit val cmk: ClassManifest[K2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]] + implicit val cmv: ClassManifest[V2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]] + def scalaTransform (in: RDD[T]): RDD[(K2, V2)] = + transformFunc.call(wrapRDD(in)).rdd + dstream.transform(scalaTransform(_)) + } + + /** + * Return a new DStream in which each RDD is generated by applying a function + * on each RDD of this DStream. + */ + def transform[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]): + JavaPairDStream[K2, V2] = { + implicit val cmk: ClassManifest[K2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]] + implicit val cmv: ClassManifest[V2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]] + def scalaTransform (in: RDD[T], time: Time): RDD[(K2, V2)] = + transformFunc.call(wrapRDD(in), time).rdd dstream.transform(scalaTransform(_, _)) } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 4fe2de5a1a764..9be680dbdcba1 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -294,8 +294,9 @@ public void testTransform() { Arrays.asList(6,7,8), Arrays.asList(9,10,11)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream transformed = stream.transform(new Function, JavaRDD>() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream transformed = + stream.transform(new Function, JavaRDD>() { @Override public JavaRDD call(JavaRDD in) throws Exception { return in.map(new Function() { @@ -742,6 +743,90 @@ public void testCountByValueAndWindow() { } @Test + public void testPairTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2(3, 5), + new Tuple2(1, 5), + new Tuple2(4, 5), + new Tuple2(2, 5)), + Arrays.asList( + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5), + new Tuple2(1, 5))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, 5), + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5)), + Arrays.asList( + new Tuple2(1, 5), + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream sorted = pairStream.transform( + new Function, JavaPairRDD>() { + @Override + public JavaPairRDD call(JavaPairRDD in) throws Exception { + return in.sortByKey(); + } + }); + + JavaTestUtils.attachTestOutputStream(sorted); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairToNormalRDDTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2(3, 5), + new Tuple2(1, 5), + new Tuple2(4, 5), + new Tuple2(2, 5)), + Arrays.asList( + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5), + new Tuple2(1, 5))); + + List> expected = Arrays.asList( + Arrays.asList(3,1,4,2), + Arrays.asList(2,3,4,1)); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaDStream firstParts = pairStream.transform( + new Function, JavaRDD>() { + @Override + public JavaRDD call(JavaPairRDD in) throws Exception { + return in.map(new Function, Integer>() { + @Override + public Integer call(Tuple2 in) { + return in._1(); + } + }); + } + }); + + JavaTestUtils.attachTestOutputStream(firstParts); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + public void testMapValues() { List>> inputData = stringStringKVStream; From fed1122d74c9d92ce26c28b0bf429e556dcc9bdd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 16 Feb 2013 16:43:23 -0800 Subject: [PATCH 250/696] Use RDD type for `slice` operator in Java. This commit uses the RDD type in `slice`, making it available to both normal and pair RDD's in java. It also updates the signature for `slice` to match changes in the Scala API. --- .../main/scala/spark/streaming/api/java/JavaDStreamLike.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index f7b1704884b32..8be36200ece42 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -215,8 +215,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T /** * Return all the RDDs between 'fromDuration' to 'toDuration' (both included) */ - def slice(fromDuration: Duration, toDuration: Duration): JList[JavaRDD[T]] = { - new util.ArrayList(dstream.slice(fromDuration, toDuration).map(new JavaRDD(_)).toSeq) + def slice(fromTime: Time, toTime: Time): JList[R] = { + new util.ArrayList(dstream.slice(fromTime, toTime).map(wrapRDD(_)).toSeq) } /** From 041c19e5f0309e7a667faab5fee9f9081db58237 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 19 Feb 2013 08:44:20 -0800 Subject: [PATCH 251/696] Small changes that were missing in merge --- .../main/scala/spark/streaming/api/java/JavaPairDStream.scala | 1 + streaming/src/test/java/spark/streaming/JavaAPISuite.java | 1 + 2 files changed, 2 insertions(+) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index de3e802300628..c1c878355976c 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -15,6 +15,7 @@ import org.apache.hadoop.conf.Configuration import spark.api.java.JavaPairRDD import spark.storage.StorageLevel import com.google.common.base.Optional +import spark.RDD class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifiest: ClassManifest[K], diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 9be680dbdcba1..53fac143868a4 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -13,6 +13,7 @@ import spark.HashPartitioner; import spark.api.java.JavaRDD; import spark.api.java.JavaRDDLike; +import spark.api.java.JavaPairRDD; import spark.api.java.JavaSparkContext; import spark.api.java.function.*; import spark.storage.StorageLevel; From 687581c3ec2b6b8310bd5be9f2d15b25b9051aac Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 19 Feb 2013 11:52:35 -0800 Subject: [PATCH 252/696] Paranoid uncaught exception handling for exceptions during shutdown --- .../main/scala/spark/executor/Executor.scala | 29 +++++++++++++++---- 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index bd21ba719a77c..b63bec11adfce 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -50,14 +50,31 @@ private[spark] class Executor extends Logging { override def uncaughtException(thread: Thread, exception: Throwable) { try { logError("Uncaught exception in thread " + thread, exception) - if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) - } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + + // We may have been called from a shutdown hook. If so, we must not call System.exit(). + // (If we do, we will deadlock.) Runtime#addShutdownHook should fail if we are shutting + // down, which would either occur if we were called from a shutdown hook or if + // a System.exit() occured concurrently. + var shuttingDown = false + try { + val hook = new Thread { + override def run() {} + } + Runtime.getRuntime.addShutdownHook(hook) + Runtime.getRuntime.removeShutdownHook(hook) + } catch { + case ise: IllegalStateException => shuttingDown = true + } + if (!shuttingDown) { + if (exception.isInstanceOf[OutOfMemoryError]) { + System.exit(ExecutorExitCode.OOM) + } else { + System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + } } } catch { - case oom: OutOfMemoryError => System.exit(ExecutorExitCode.OOM) - case t: Throwable => System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) } } } From d0588bd6d7da3ba5adaba24303ad8616bdc2484f Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 19 Feb 2013 11:53:01 -0800 Subject: [PATCH 253/696] Catch/log errors deleting temp dirs --- core/src/main/scala/spark/storage/DiskStore.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 7e5b820cbbdc6..ddbf8821ad15a 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -178,7 +178,11 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - localDirs.foreach(localDir => Utils.deleteRecursively(localDir)) + try { + localDirs.foreach(localDir => Utils.deleteRecursively(localDir)) + } catch { + case t: Throwable => logError("Exception while deleting local spark dirs", t) + } } }) } From 130f704bafe9e327e8974f6ed3a4e00c478f6279 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Feb 2013 16:03:52 -0800 Subject: [PATCH 254/696] Added a method to create PartitionPruningRDD. --- .../main/scala/spark/rdd/PartitionPruningRDD.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index f2f4fd56d1349..41ff62dd22857 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -40,3 +40,15 @@ class PartitionPruningRDD[T: ClassManifest]( override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions } + + +object PartitionPruningRDD { + + /** + * Create a PartitionPruningRDD. This function can be used to create the PartitionPruningRDD + * when its type T is not known at compile time. + */ + def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = { + new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassManifest) + } +} From ecd137a72da189c52b92a1286b004740706bd936 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Tue, 19 Feb 2013 16:58:02 -0800 Subject: [PATCH 255/696] Fixes link to issue tracker in documentation page "Contributing to Spark". --- docs/_config.yml | 1 + docs/contributing-to-spark.md | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 2bd2eecc863e4..09617e4a1efb6 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,3 +7,4 @@ SPARK_VERSION: 0.7.0-SNAPSHOT SPARK_VERSION_SHORT: 0.7.0 SCALA_VERSION: 2.9.2 MESOS_VERSION: 0.9.0-incubating +SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md index 14d0dc856b2f7..50feeb2d6c42a 100644 --- a/docs/contributing-to-spark.md +++ b/docs/contributing-to-spark.md @@ -15,7 +15,7 @@ The Spark team welcomes contributions in the form of GitHub pull requests. Here But first, make sure that you have [configured a spark-env.sh](configuration.html) with at least `SCALA_HOME`, as some of the tests try to spawn subprocesses using this. - Add new unit tests for your code. We use [ScalaTest](http://www.scalatest.org/) for testing. Just add a new Suite in `core/src/test`, or methods to an existing Suite. -- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issue tracker](https://spark-project.atlassian.net), or email the [mailing list](http://www.spark-project.org/mailing-lists.html). +- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issue tracker]({{site.SPARK_ISSUE_TRACKER_URL}}), or email the [mailing list](http://www.spark-project.org/mailing-lists.html). # Licensing of Contributions From 092c631fa8da6381b814f4d262c884ba08629b39 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 19 Feb 2013 17:49:55 -0800 Subject: [PATCH 256/696] Pull detection of being in a shutdown hook into utility function. --- core/src/main/scala/spark/Utils.scala | 21 +++++++++++++++++++ .../main/scala/spark/executor/Executor.scala | 16 ++------------ 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 28d643abca8f4..81daacf958b5a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -454,4 +454,25 @@ private object Utils extends Logging { def clone[T](value: T, serializer: SerializerInstance): T = { serializer.deserialize[T](serializer.serialize(value)) } + + /** + * Detect whether this thread might be executing a shutdown hook. Will always return true if + * the current thread is a running a shutdown hook but may spuriously return true otherwise (e.g. + * if System.exit was just called by a concurrent thread). + * + * Currently, this detects whether the JVM is shutting down by Runtime#addShutdownHook throwing + * an IllegalStateException. + */ + def inShutdown(): Boolean = { + try { + val hook = new Thread { + override def run() {} + } + Runtime.getRuntime.addShutdownHook(hook) + Runtime.getRuntime.removeShutdownHook(hook) + } catch { + case ise: IllegalStateException => return true + } + return false + } } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index b63bec11adfce..5de09030aa1b3 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -52,20 +52,8 @@ private[spark] class Executor extends Logging { logError("Uncaught exception in thread " + thread, exception) // We may have been called from a shutdown hook. If so, we must not call System.exit(). - // (If we do, we will deadlock.) Runtime#addShutdownHook should fail if we are shutting - // down, which would either occur if we were called from a shutdown hook or if - // a System.exit() occured concurrently. - var shuttingDown = false - try { - val hook = new Thread { - override def run() {} - } - Runtime.getRuntime.addShutdownHook(hook) - Runtime.getRuntime.removeShutdownHook(hook) - } catch { - case ise: IllegalStateException => shuttingDown = true - } - if (!shuttingDown) { + // (If we do, we will deadlock.) + if (!Utils.inShutdown()) { if (exception.isInstanceOf[OutOfMemoryError]) { System.exit(ExecutorExitCode.OOM) } else { From 05dc385649277836962a512a83195083990a7134 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 20 Feb 2013 15:28:12 +0530 Subject: [PATCH 257/696] A bug fix post merge, following changes to AkkaUtils --- .../main/scala/spark/streaming/examples/ActorWordCount.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 71b4e5bf1a80c..346151c147483 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -147,7 +147,7 @@ object ActorWordCount { */ val lines = ssc.actorStream[String]( - Props(new SampleActorReceiver[String]("akka://spark@%s:%s/user/FeederActor".format( + Props(new SampleActorReceiver[String]("akka://test@%s:%s/user/FeederActor".format( host, port.toInt))), "SampleReceiver") //compute wordcount From 4e5b09664cdf95effff61c042b6243107355b55c Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 20 Feb 2013 12:33:37 +0530 Subject: [PATCH 258/696] fixes corresponding to review feedback at pull request #479 --- .../scala/spark/streaming/examples/ZeroMQWordCount.scala | 9 ++++++--- project/SparkBuild.scala | 4 ++-- .../main/scala/spark/streaming/StreamingContext.scala | 2 +- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala index ab7b67ed4b55f..5ed9b7cb76875 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -35,20 +35,23 @@ object SimpleZeroMQPublisher { /** * A sample wordcount with ZeroMQStream stream * - * Usage: WordCountZeroMQ + * To work with zeroMQ, some native libraries have to be installed. + * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide](http://www.zeromq.org/intro:get-the-software) + * + * Usage: ZeroMQWordCount * In local mode, should be 'local[n]' with n > 1 * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as * `$ ./run spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` - * and then run the example + * and run the example as * `$ ./run spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ object ZeroMQWordCount { def main(args: Array[String]) { if (args.length < 3) { System.err.println( - "Usage: WordCountZeroMQ " + + "Usage: ZeroMQWordCount " + "In local mode, should be 'local[n]' with n > 1") System.exit(1) } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5fe85a28c3cf7..7f432b60dbc1a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -134,7 +134,6 @@ object SparkBuild extends Build { "com.typesafe.akka" % "akka-actor" % "2.0.3", "com.typesafe.akka" % "akka-remote" % "2.0.3", "com.typesafe.akka" % "akka-slf4j" % "2.0.3", - "com.typesafe.akka" % "akka-zeromq" % "2.0.3", "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "cc.spray" % "spray-can" % "1.0-M2.1", @@ -165,7 +164,8 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "com.github.sgroschupf" % "zkclient" % "0.1", - "org.twitter4j" % "twitter4j-stream" % "3.0.3" + "org.twitter4j" % "twitter4j-stream" % "3.0.3", + "com.typesafe.akka" % "akka-zeromq" % "2.0.3" ) ) ++ assemblySettings ++ extraAssemblySettings diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index f15e6bd23d79d..2ca7dcc218222 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -177,7 +177,7 @@ class StreamingContext private ( } /** - * ZeroMQ stream receiver + * Create an input stream that receives messages pushed by a zeromq publisher. * @param publisherUrl Url of remote zeromq publisher * @param zeroMQ topic to subscribe to * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence From ee885611ff14dbf1bcc218e72da2728fe653e213 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 20 Feb 2013 19:18:24 +0530 Subject: [PATCH 259/696] removed document on zeromq --- docs/zeroMQ-intro.md | 59 -------------------------------------------- 1 file changed, 59 deletions(-) delete mode 100644 docs/zeroMQ-intro.md diff --git a/docs/zeroMQ-intro.md b/docs/zeroMQ-intro.md deleted file mode 100644 index 0365bc08fd6ff..0000000000000 --- a/docs/zeroMQ-intro.md +++ /dev/null @@ -1,59 +0,0 @@ ---- -layout: global -title: ZeroMQ Stream setup guide ---- - -## Install ZeroMQ (using JNA) - -To work with zeroMQ, some native libraries have to be installed. - -* Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide](http://www.zeromq.org/intro:get-the-software) - - Typically if you are using ubuntu 12.04, you can do: - - `$ sudo apt-get install libzmq1` - - __To work with akka-zeromq, zmq 2.1 version is supported via [JNA](https://github.com/twall/jna). Incase you want to switch to zeromq 3.0, please install [JZMQ](http://www.zeromq.org/bindings:java) which uses [JNI](http://docs.oracle.com/javase/6/docs/technotes/guides/jni/) and drop in jzmq jar__ - -## Sample scala code - -A publisher is an entity assumed to be outside the spark ecosystem. A sample zeroMQ publisher is provided to try out the sample spark ZeroMQ application. - -1. Start the sample publisher. - -{% highlight scala %} - - - val acs: ActorSystem = ActorSystem() - - val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) - - pubSocket ! ZMQMessage(Seq(Frame("topic"), Frame("My message".getBytes))) - - - -{% endhighlight %} - -A typical zeromq url looks like `tcp://:` - -It does nothing more than publishing the message on the specified topic and url. - -2. Start the spark application by plugging the zeroMQ stream receiver. - -{% highlight scala %} - - val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToObjectsIterator) - -{% endhighlight %} - -bytesToObjectsIterator is going to be a function for decoding the Frame data. - -_For example: For decoding into strings using default charset:_ - - -{% highlight scala %} - - - def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator - -{% endhighlight %} From 334ab9244113e4b792fd51697ef80ab0d3b3de25 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 20 Feb 2013 10:26:36 -0800 Subject: [PATCH 260/696] Fixed bug in CheckpointSuite --- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 8 ++++---- core/src/test/scala/spark/CheckpointSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 3328477959507..9e37bdf659201 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -22,10 +22,10 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri override def getPartitions: Array[Partition] = { val dirContents = fs.listStatus(new Path(checkpointPath)) - val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted - val numPartitions = splitFiles.size - if (numPartitions > 0 && !splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || - !splitFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1))) { + val partitionFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted + val numPartitions = partitionFiles.size + if (numPartitions > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || + ! partitionFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1)))) { throw new SparkException("Invalid checkpoint directory: " + checkpointPath) } Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i)) diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 1935ac9e49776..ca385972fb2eb 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -164,12 +164,12 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("CheckpointRDD with zero partitions") { val rdd = new BlockRDD[Int](sc, Array[String]()) - assert(rdd.splits.size === 0) + assert(rdd.partitions.size === 0) assert(rdd.isCheckpointed === false) rdd.checkpoint() assert(rdd.count() === 0) assert(rdd.isCheckpointed === true) - assert(rdd.splits.size === 0) + assert(rdd.partitions.size === 0) } /** From 718474b9c6ab985833af9cde69f13c4d33498f6d Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 21 Feb 2013 12:11:31 +0200 Subject: [PATCH 261/696] Bumping Algebird to 0.1.9 --- examples/pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 7d975875fac3a..f6125444e2a46 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -22,7 +22,7 @@ com.twitter algebird-core_2.9.2 - 0.1.8 + 0.1.9 org.scalatest diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 090fd65bcb1da..030a60f5bda3e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -155,7 +155,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8") + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.9") ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From d9bdae8cc249ee8f595a849c5a751caef75140c5 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 21 Feb 2013 12:31:31 +0200 Subject: [PATCH 262/696] Adding documentation for HLL and CMS examples. More efficient and clear use of the monoids. --- .../examples/TwitterAlgebirdCMS.scala | 29 ++++++++++++++----- .../examples/TwitterAlgebirdHLL.scala | 19 ++++++++---- 2 files changed, 36 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index d50e86aff5d71..39a1a702eeae9 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -7,8 +7,22 @@ import spark.streaming.StreamingContext._ import spark.SparkContext._ /** - * Example of using CountMinSketch monoid from Twitter's Algebird together with Spark Streaming's - * TwitterInputDStream + * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute + * windowed and global Top-K estimates of user IDs occurring in a Twitter stream. + *
    + * Note that since Algebird's implementation currently only supports Long inputs, + * the example operates on Long IDs. Once the implementation supports other inputs (such as String), + * the same approach could be used for computing popular topics for example. + *

    + *

    + * + * This blog post has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure + * for approximate frequency estimation in data streams (e.g. Top-K elements, frequency of any given element, etc), + * that uses space sub-linear in the number of elements in the stream. Once elements are added to the CMS, the + * estimated count of an element can be computed, as well as "heavy-hitters" that occur more than a threshold + * percentage of the overall total count. + *

    + * Algebird's implementation is a monoid, so we can succinctly merge two CMS instances in the reduce operation. */ object TwitterAlgebirdCMS { def main(args: Array[String]) { @@ -18,27 +32,28 @@ object TwitterAlgebirdCMS { System.exit(1) } + // CMS parameters val DELTA = 1E-3 val EPS = 0.01 val SEED = 1 val PERC = 0.001 + // K highest frequency elements to take val TOPK = 10 val Array(master, username, password) = args.slice(0, 3) val filters = args.slice(3, args.length) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10)) - val stream = ssc.twitterStream(username, password, filters, - StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) - var globalCMS = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC).zero - var globalExact = Map[Long, Int]() + val cms = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC) + var globalCMS = cms.zero val mm = new MapMonoid[Long, Int]() + var globalExact = Map[Long, Int]() val approxTopUsers = users.mapPartitions(ids => { - val cms = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC) ids.map(id => cms.create(id)) }).reduce(_ ++ _) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index c2095f5b9459b..914fba4ca22c5 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -7,8 +7,17 @@ import com.twitter.algebird.HyperLogLogMonoid import spark.streaming.dstream.TwitterInputDStream /** - * Example using HyperLogLog monoid from Twitter's Algebird together with Spark Streaming's - * TwitterInputDStream to compute approximate distinct counts of userids. + * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute + * a windowed and global estimate of the unique user IDs occurring in a Twitter stream. + *

    + *

    + * This + * blog post and this + * blog post + * have good overviews of HyperLogLog (HLL). HLL is a memory-efficient datastructure for estimating + * the cardinality of a data stream, i.e. the number of unique elements. + *

    + * Algebird's implementation is a monoid, so we can succinctly merge two HLL instances in the reduce operation. */ object TwitterAlgebirdHLL { def main(args: Array[String]) { @@ -18,7 +27,7 @@ object TwitterAlgebirdHLL { System.exit(1) } - /** Bit size parameter for HyperLogLog */ + /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ val BIT_SIZE = 12 val Array(master, username, password) = args.slice(0, 3) val filters = args.slice(3, args.length) @@ -28,11 +37,11 @@ object TwitterAlgebirdHLL { val users = stream.map(status => status.getUser.getId) - var globalHll = new HyperLogLogMonoid(BIT_SIZE).zero + val hll = new HyperLogLogMonoid(BIT_SIZE) + var globalHll = hll.zero var userSet: Set[Long] = Set() val approxUsers = users.mapPartitions(ids => { - val hll = new HyperLogLogMonoid(BIT_SIZE) ids.map(id => hll(id)) }).reduce(_ + _) From baab23abdfc94af32c35d5fc2035382d3faa0ec4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 14:13:01 -0800 Subject: [PATCH 263/696] TaskContext does not hold a reference to Task; instead, it has a shared instance of TaskMetrics with Task --- core/src/main/scala/spark/TaskContext.scala | 9 ++++++-- .../main/scala/spark/executor/Executor.scala | 2 +- .../scala/spark/executor/TaskMetrics.scala | 21 ++++++++----------- .../main/scala/spark/rdd/CoGroupedRDD.scala | 14 ++++++------- .../scala/spark/scheduler/DAGScheduler.scala | 2 +- .../scala/spark/scheduler/ResultTask.scala | 3 ++- .../spark/scheduler/ShuffleMapTask.scala | 5 +++-- .../src/main/scala/spark/scheduler/Task.scala | 14 +++---------- 8 files changed, 33 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index 6b67b17b8ac48..5aadd5e261da1 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,9 +1,14 @@ package spark +import executor.TaskMetrics import scala.collection.mutable.ArrayBuffer -import spark.scheduler.Task -class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long, val task: Task[_]) extends Serializable { +class TaskContext( + val stageId: Int, + val splitId: Int, + val attemptId: Long, + val taskMetrics: TaskMetrics = TaskMetrics.empty() +) extends Serializable { //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to // be separate diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 0906702a3dff7..7f2c16b43a281 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -95,7 +95,7 @@ private[spark] class Executor extends Logging { env.mapOutputTracker.updateGeneration(task.generation) val value = task.run(taskId.toInt) val accumUpdates = Accumulators.values - val result = new TaskResult(value, accumUpdates, TaskMetrics(task)) + val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null)) val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index b005b9ff27a94..525abf8893435 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,19 +1,16 @@ package spark.executor -import spark.scheduler.Task - case class TaskMetrics( - val totalBlocksFetched : Option[Int], - val remoteBlocksFetched: Option[Int], - val localBlocksFetched: Option[Int], - val remoteFetchWaitTime: Option[Long], - val remoteBytesRead: Option[Long], - val shuffleBytesWritten: Option[Long] + var shuffleReadMillis: Option[Long], + var totalBlocksFetched : Option[Int], + var remoteBlocksFetched: Option[Int], + var localBlocksFetched: Option[Int], + var remoteFetchWaitTime: Option[Long], + var remoteFetchTime: Option[Long], + var remoteBytesRead: Option[Long], + var shuffleBytesWritten: Option[Long] ) object TaskMetrics { - private[spark] def apply(task: Task[_]) : TaskMetrics = { - TaskMetrics(task.totalBlocksFetched, task.remoteBlocksFetched, task.localBlocksFetched, - task.remoteFetchWaitTime, task.remoteReadBytes, task.shuffleBytesWritten) - } + private[spark] def empty() : TaskMetrics = TaskMetrics(None,None,None,None,None,None,None,None) } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index a32a83ccaeba7..66e3cb4c32935 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -106,13 +106,13 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) for ((k, vs) <- fetchItr) { getSeq(k)(depNum) ++= vs } - context.task.shuffleReadMillis = Some(fetchItr.getNetMillis) - context.task.remoteFetchTime = Some(fetchItr.remoteFetchTime) - context.task.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) - context.task.remoteReadBytes = Some(fetchItr.remoteBytesRead) - context.task.totalBlocksFetched = Some(fetchItr.totalBlocks) - context.task.localBlocksFetched = Some(fetchItr.numLocalBlocks) - context.task.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) + context.taskMetrics.shuffleReadMillis = Some(fetchItr.getNetMillis) + context.taskMetrics.remoteFetchTime = Some(fetchItr.remoteFetchTime) + context.taskMetrics.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) + context.taskMetrics.remoteBytesRead = Some(fetchItr.remoteBytesRead) + context.taskMetrics.totalBlocksFetched = Some(fetchItr.totalBlocks) + context.taskMetrics.localBlocksFetched = Some(fetchItr.numLocalBlocks) + context.taskMetrics.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 608962b5f174f..04128f89fd0a9 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -393,7 +393,7 @@ class DAGScheduler( SparkEnv.set(env) val rdd = job.finalStage.rdd val split = rdd.splits(job.partitions(0)) - val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0, null) //TODO figure out what to do about Task here + val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index e1b26a0cf5fea..585e4a28996d9 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -71,7 +71,8 @@ private[spark] class ResultTask[T, U]( } override def run(attemptId: Long): U = { - val context = new TaskContext(stageId, partition, attemptId, this) + val context = new TaskContext(stageId, partition, attemptId) + metrics = Some(context.taskMetrics) try { func(context, rdd.iterator(split, context)) } finally { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 744f046b0e87a..486dba0bb039d 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -118,7 +118,8 @@ private[spark] class ShuffleMapTask( override def run(attemptId: Long): MapStatus = { val numOutputSplits = dep.partitioner.numPartitions - val taskContext = new TaskContext(stageId, partition, attemptId, this) + val taskContext = new TaskContext(stageId, partition, attemptId) + metrics = Some(taskContext.taskMetrics) try { // Partition the map output. val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) @@ -141,7 +142,7 @@ private[spark] class ShuffleMapTask( totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) } - shuffleBytesWritten = Some(totalBytes) + metrics.get.shuffleBytesWritten = Some(totalBytes) return new MapStatus(blockManager.blockManagerId, compressedSizes) } finally { diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index 84fc3bc9889b8..a6462c6968b67 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -1,12 +1,12 @@ package spark.scheduler -import scala.collection.mutable.HashMap -import spark.serializer.{SerializerInstance, Serializer} +import spark.serializer.SerializerInstance import java.io.{DataInputStream, DataOutputStream} import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import spark.util.ByteBufferInputStream import scala.collection.mutable.HashMap +import spark.executor.TaskMetrics /** * A task to execute on a worker node. @@ -17,15 +17,7 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. - var shuffleReadMillis : Option[Long] = None - var remoteReadBytes : Option[Long] = None - var remoteFetchWaitTime : Option[Long] = None - var remoteFetchTime : Option[Long] = None - var totalBlocksFetched : Option[Int] = None - var remoteBlocksFetched: Option[Int] = None - var localBlocksFetched: Option[Int] = None - - var shuffleBytesWritten : Option[Long] = None + var metrics: Option[TaskMetrics] = None } From 69f9a7035f7ccd0e779fa7986c0107cc386b6a6f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 15:07:10 -0800 Subject: [PATCH 264/696] fully revert change to addOnCompleteCallback -- missed this in e9f53ec --- core/src/test/scala/spark/scheduler/TaskContextSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index 4af5de6cf66a1..a5db7103f5ce8 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -16,7 +16,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte val rdd = new RDD[String](sc, List()) { override def getSplits = Array[Split](StubSplit(0)) override def compute(split: Split, context: TaskContext) = { - context.addOnCompleteCallback(_ => completed = true) + context.addOnCompleteCallback(() => completed = true) sys.error("failed") } } From f2fcabf2eaba4b87e4e57edfb651212687796487 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 10:00:13 -0800 Subject: [PATCH 265/696] add timing around parts of executor & track result size --- core/src/main/scala/spark/executor/Executor.scala | 10 ++++++++++ core/src/main/scala/spark/executor/TaskMetrics.scala | 7 +++++-- .../scala/spark/scheduler/cluster/TaskSetManager.scala | 4 +++- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 3e3daa29f76e2..4474ef4593703 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -85,6 +85,7 @@ private[spark] class Executor extends Logging { extends Runnable { override def run() { + val startTime = System.currentTimeMillis() SparkEnv.set(env) Thread.currentThread.setContextClassLoader(urlClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() @@ -98,7 +99,16 @@ private[spark] class Executor extends Logging { val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) logInfo("Its generation is " + task.generation) env.mapOutputTracker.updateGeneration(task.generation) + val taskStart = System.currentTimeMillis() val value = task.run(taskId.toInt) + val taskFinish = System.currentTimeMillis() + task.metrics.foreach{ m => + m.executorDeserializeTime = (taskStart - startTime).toInt + m.executorRunTime = (taskFinish - taskStart).toInt + } + //TODO I'd also like to track the time it takes to serialize the task results, but that is huge headache, b/c + // we need to serialize the task metrics first. If TaskMetrics had a custom serialized format, we could + // just change the relevants bytes in the byte buffer val accumUpdates = Accumulators.values val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null)) val serializedResult = ser.serialize(result) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 525abf8893435..8db71bdc17f7f 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -8,9 +8,12 @@ case class TaskMetrics( var remoteFetchWaitTime: Option[Long], var remoteFetchTime: Option[Long], var remoteBytesRead: Option[Long], - var shuffleBytesWritten: Option[Long] + var shuffleBytesWritten: Option[Long], + var executorDeserializeTime: Int, + var executorRunTime:Int, + var resultSize: Long ) object TaskMetrics { - private[spark] def empty() : TaskMetrics = TaskMetrics(None,None,None,None,None,None,None,None) + private[spark] def empty() : TaskMetrics = TaskMetrics(None,None,None,None,None,None,None,None, -1, -1, -1) } \ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 9ed098e501c7f..48876ffd79547 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -259,7 +259,9 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + //lame way to get size into final metrics + val metricsWithSize = result.metrics.copy(resultSize = serializedData.limit()) + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, metricsWithSize) // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { From 176cb20703ffc344a232fb725d50975c31495bcd Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 18:10:54 -0800 Subject: [PATCH 266/696] add task result size; better formatting for time interval distributions; cleanup distribution formatting --- .../scala/spark/scheduler/SparkListener.scala | 46 +++++++++++++++++-- .../scala/spark/scheduler/StageInfo.scala | 4 ++ 2 files changed, 45 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 92d9c0330ae69..951fe1ae3bc2b 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -15,13 +15,19 @@ case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { - println("Finished stage: " + stageCompleted.stageInfo) - showDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution, "%4.0f") - showDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution, d => Utils.memoryBytesToString(d.toLong)) - showDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution, "%4.0f") - showDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution, d => Utils.memoryBytesToString(d.toLong)) + import spark.scheduler.StatsReportListener._ + logInfo("Finished stage: " + stageCompleted.stageInfo) + showMillisDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution) + showBytesDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution) + showMillisDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution) + showBytesDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution) + showBytesDistribution("task result size:", stageCompleted.stageInfo.getTaskResultSizeDistribution) } +} + +object StatsReportListener { + //for profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) val probabilities = percentiles.map{_ / 100.0} @@ -32,6 +38,14 @@ class StatsReportListener extends SparkListener with Logging { showDistribution(heading, dOpt, f _) } + def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { + showDistribution(heading, dOpt, d => Utils.memoryBytesToString(d.toLong)) + } + + def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { + showDistribution(heading, dOpt, d => StatsReportListener.millisToString(d.toLong)) + } + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { dOpt.foreach { d => val stats = d.statCounter @@ -41,4 +55,26 @@ class StatsReportListener extends SparkListener with Logging { logInfo("\t" + quantiles.mkString("\t")) } } + + + val seconds = 1000L + val minutes = seconds * 60 + val hours = minutes * 60 + + /** + * reformat a time interval in milliseconds to a prettier format for output + */ + def millisToString(ms: Long) = { + val (size, units) = + if (ms > hours) { + (ms.toDouble / hours, "hours") + } else if (ms > minutes) { + (ms.toDouble / minutes, "min") + } else if (ms > seconds) { + (ms.toDouble / seconds, "s") + } else { + (ms.toDouble, "ms") + } + "%.1f %s".format(size, units) + } } \ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index df3d1ec4dc062..299f43d1c529a 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -28,4 +28,8 @@ case class StageInfo( def getRemoteBytesReadDistribution = { Distribution(taskMetrics.flatMap{_.remoteBytesRead.map{_.toDouble}}) } + + def getTaskResultSizeDistribution = { + Distribution(taskMetrics.flatMap{_.resultSize.map{_.toDouble}}) + } } From 6f62a5785872baeae5032c7a85bfcc05409f03d7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Feb 2013 18:37:07 -0800 Subject: [PATCH 267/696] add runtime breakdowns --- .../scala/spark/scheduler/SparkListener.scala | 11 +++++- .../scala/spark/scheduler/StageInfo.scala | 39 ++++++++++++++++++- 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 951fe1ae3bc2b..f97164669e092 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -16,17 +16,24 @@ case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ - logInfo("Finished stage: " + stageCompleted.stageInfo) + this.logInfo("Finished stage: " + stageCompleted.stageInfo) showMillisDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution) showBytesDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution) + + //fetch & some io info showMillisDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution) showBytesDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution) showBytesDistribution("task result size:", stageCompleted.stageInfo.getTaskResultSizeDistribution) + + //runtime breakdown + showDistribution("executor (non-fetch) time pct: ", stageCompleted.stageInfo.getExectuorRuntimePercentage, "%2.0f \\%") + showDistribution("fetch wait time pct: ", stageCompleted.stageInfo.getFetchRuntimePercentage, "%2.0f \\%") + showDistribution("other time pct: ", stageCompleted.stageInfo.getOtherRuntimePercentage, "%2.0f \\%") } } -object StatsReportListener { +object StatsReportListener extends Logging { //for profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 299f43d1c529a..ac02d3445ce33 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -30,6 +30,43 @@ case class StageInfo( } def getTaskResultSizeDistribution = { - Distribution(taskMetrics.flatMap{_.resultSize.map{_.toDouble}}) + Distribution(taskMetrics.map{_.resultSize.toDouble}) + } + + lazy val runtimePercentages = taskMetrics.zip(taskInfos).map{ + case (metrics, info) => RuntimePercentage(info.duration, metrics) + } + + /** + * distribution of the percentage of task runtime of the executor itself, excluding time spent waiting on a fetch + */ + def getExectuorRuntimePercentage = { + Distribution(runtimePercentages.map{_.executorPct}) + } + + /** + * distribution of the percentage of task runtime spent waiting on a fetch + */ + def getFetchRuntimePercentage = { + Distribution(runtimePercentages.flatMap{_.fetchPct}) + } + + /** + * distribution of the percentage of task runtime spent not waiting on fetch, and not actively executing on + * a remote machine (eg., serializing task, sending it over network, sending results back over network) + */ + def getOtherRuntimePercentage = { + Distribution(runtimePercentages.map{_.other}) + } +} + +private[spark] case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) +private[spark] object RuntimePercentage { + def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { + val denom = totalTime.toDouble + val fetch = metrics.remoteFetchWaitTime.map{_ / denom} + val exec = (metrics.executorRunTime - metrics.remoteFetchWaitTime.getOrElse(0l)) / denom + val other = 1.0 - (exec + fetch.getOrElse(0d)) + RuntimePercentage(exec, fetch, other) } } From d0bfac3eeda09c18c7af521584cfe9db9a5a10c3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 Feb 2013 22:27:36 -0800 Subject: [PATCH 268/696] taskInfo tracks if a task is run on a preferred host --- core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala | 3 ++- .../main/scala/spark/scheduler/cluster/TaskSetManager.scala | 2 +- core/src/main/scala/spark/scheduler/local/LocalScheduler.scala | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index a7e14094fb70a..53a3c5dc4d3cb 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -11,7 +11,8 @@ class TaskInfo( val index: Int, val launchTime: Long, val executorId: String, - val host: String) { + val host: String, + val preferred: Boolean) { var finishTime: Long = 0 var failed = false diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 48876ffd79547..236f81bb9f3a7 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -208,7 +208,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe taskSet.id, index, taskId, execId, host, prefStr)) // Do various bookkeeping copiesRunning(index) += 1 - val info = new TaskInfo(taskId, index, time, execId, host) + val info = new TaskInfo(taskId, index, time, execId, host, preferred) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) if (preferred) { diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 6b91728b74f2f..a76253ea14d3e 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -53,7 +53,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon def runTask(task: Task[_], idInJob: Int, attemptId: Int) { logInfo("Running " + task) - val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local") + val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local", true) // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { From 7960927cf4ae2c9bb543c34e1f66f525fb828263 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 Feb 2013 23:14:36 -0800 Subject: [PATCH 269/696] get rid of a bunch of boilerplate; more formatting happens in Listener, not StageInfo --- .../scala/spark/scheduler/SparkListener.scala | 62 ++++++++++++++++--- .../scala/spark/scheduler/StageInfo.scala | 60 +----------------- 2 files changed, 54 insertions(+), 68 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index f97164669e092..0a57b946362de 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,7 +1,9 @@ package spark.scheduler +import spark.scheduler.cluster.TaskInfo import spark.util.Distribution import spark.{Utils, Logging} +import spark.executor.TaskMetrics trait SparkListener { def onStageCompleted(stageCompleted: StageCompleted) @@ -16,19 +18,23 @@ case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ + implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stageInfo) - showMillisDistribution("task runtime:", stageCompleted.stageInfo.getTaskRuntimeDistribution) - showBytesDistribution("shuffle bytes written:", stageCompleted.stageInfo.getShuffleBytesWrittenDistribution) + showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) + showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleBytesWritten) //fetch & some io info - showMillisDistribution("fetch wait time:",stageCompleted.stageInfo.getRemoteFetchWaitTimeDistribution) - showBytesDistribution("remote bytes read:", stageCompleted.stageInfo.getRemoteBytesReadDistribution) - showBytesDistribution("task result size:", stageCompleted.stageInfo.getTaskResultSizeDistribution) + showMillisDistribution("fetch wait time:",(_, metric) => metric.remoteFetchWaitTime) + showBytesDistribution("remote bytes read:", (_, metric) => metric.remoteBytesRead) + showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown - showDistribution("executor (non-fetch) time pct: ", stageCompleted.stageInfo.getExectuorRuntimePercentage, "%2.0f \\%") - showDistribution("fetch wait time pct: ", stageCompleted.stageInfo.getFetchRuntimePercentage, "%2.0f \\%") - showDistribution("other time pct: ", stageCompleted.stageInfo.getOtherRuntimePercentage, "%2.0f \\%") + val runtimePcts = stageCompleted.stageInfo.taskInfos.zip(stageCompleted.stageInfo.taskMetrics).map{ + case (info, metrics) => RuntimePercentage(info.duration, metrics) + } + showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") + showDistribution("fetch wait time pct: ", Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") + showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%") } } @@ -40,11 +46,31 @@ object StatsReportListener extends Logging { val probabilities = percentiles.map{_ / 100.0} val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" + def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = { + Distribution(stage.stageInfo.taskInfos.zip(stage.stageInfo.taskMetrics).flatMap{ + case ((info,metric)) => getMetric(info, metric)}) + } + + //is there some way to setup the types that I can get rid of this completely? + def extractLongDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Long]): Option[Distribution] = { + extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) + } + def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { def f(d:Double) = format.format(d) showDistribution(heading, dOpt, f _) } + def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double]) + (implicit stage: StageCompleted) { + showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) + } + + def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long]) + (implicit stage: StageCompleted) { + showBytesDistribution(heading, extractLongDistribution(stage, getMetric)) + } + def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { showDistribution(heading, dOpt, d => Utils.memoryBytesToString(d.toLong)) } @@ -53,6 +79,11 @@ object StatsReportListener extends Logging { showDistribution(heading, dOpt, d => StatsReportListener.millisToString(d.toLong)) } + def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) + (implicit stage: StageCompleted) { + showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) + } + def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { dOpt.foreach { d => val stats = d.statCounter @@ -84,4 +115,17 @@ object StatsReportListener extends Logging { } "%.1f %s".format(size, units) } -} \ No newline at end of file +} + + + +case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) +object RuntimePercentage { + def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { + val denom = totalTime.toDouble + val fetch = metrics.remoteFetchWaitTime.map{_ / denom} + val exec = (metrics.executorRunTime - metrics.remoteFetchWaitTime.getOrElse(0l)) / denom + val other = 1.0 - (exec + fetch.getOrElse(0d)) + RuntimePercentage(exec, fetch, other) + } +} diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index ac02d3445ce33..179e384cf9e2c 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -10,63 +10,5 @@ case class StageInfo( val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), val taskMetrics: mutable.Buffer[TaskMetrics] = mutable.Buffer[TaskMetrics]() ) { - override def toString = stage.rdd.toString - - def getTaskRuntimeDistribution = { - Distribution(taskInfos.map{_.duration.toDouble}) - } - - def getShuffleBytesWrittenDistribution = { - Distribution(taskMetrics.flatMap{_.shuffleBytesWritten.map{_.toDouble}}) - } - - def getRemoteFetchWaitTimeDistribution = { - Distribution(taskMetrics.flatMap{_.remoteFetchWaitTime.map{_.toDouble}}) - } - - def getRemoteBytesReadDistribution = { - Distribution(taskMetrics.flatMap{_.remoteBytesRead.map{_.toDouble}}) - } - - def getTaskResultSizeDistribution = { - Distribution(taskMetrics.map{_.resultSize.toDouble}) - } - - lazy val runtimePercentages = taskMetrics.zip(taskInfos).map{ - case (metrics, info) => RuntimePercentage(info.duration, metrics) - } - - /** - * distribution of the percentage of task runtime of the executor itself, excluding time spent waiting on a fetch - */ - def getExectuorRuntimePercentage = { - Distribution(runtimePercentages.map{_.executorPct}) - } - - /** - * distribution of the percentage of task runtime spent waiting on a fetch - */ - def getFetchRuntimePercentage = { - Distribution(runtimePercentages.flatMap{_.fetchPct}) - } - - /** - * distribution of the percentage of task runtime spent not waiting on fetch, and not actively executing on - * a remote machine (eg., serializing task, sending it over network, sending results back over network) - */ - def getOtherRuntimePercentage = { - Distribution(runtimePercentages.map{_.other}) - } -} - -private[spark] case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) -private[spark] object RuntimePercentage { - def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { - val denom = totalTime.toDouble - val fetch = metrics.remoteFetchWaitTime.map{_ / denom} - val exec = (metrics.executorRunTime - metrics.remoteFetchWaitTime.getOrElse(0l)) / denom - val other = 1.0 - (exec + fetch.getOrElse(0d)) - RuntimePercentage(exec, fetch, other) - } -} +} \ No newline at end of file From 394d3acc3e66df9377819505824c57008d78a228 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 Feb 2013 12:15:45 -0800 Subject: [PATCH 270/696] store taskInfo & metrics together in a tuple --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 +-- core/src/main/scala/spark/scheduler/SparkListener.scala | 4 ++-- core/src/main/scala/spark/scheduler/StageInfo.scala | 4 +--- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f2b518e72db71..ce3b62cc4d768 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -489,8 +489,7 @@ class DAGScheduler( Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task - stageToInfos(stage).taskInfos += event.taskInfo - stageToInfos(stage).taskMetrics += event.taskMetrics + stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 0a57b946362de..f844a9068016b 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -29,7 +29,7 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown - val runtimePcts = stageCompleted.stageInfo.taskInfos.zip(stageCompleted.stageInfo.taskMetrics).map{ + val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") @@ -47,7 +47,7 @@ object StatsReportListener extends Logging { val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = { - Distribution(stage.stageInfo.taskInfos.zip(stage.stageInfo.taskMetrics).flatMap{ + Distribution(stage.stageInfo.taskInfos.flatMap{ case ((info,metric)) => getMetric(info, metric)}) } diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 179e384cf9e2c..8d83ff10c420d 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -2,13 +2,11 @@ package spark.scheduler import spark.scheduler.cluster.TaskInfo import scala.collection._ -import spark.util.Distribution import spark.executor.TaskMetrics case class StageInfo( val stage: Stage, - val taskInfos: mutable.Buffer[TaskInfo] = mutable.Buffer[TaskInfo](), - val taskMetrics: mutable.Buffer[TaskMetrics] = mutable.Buffer[TaskMetrics]() + val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() ) { override def toString = stage.rdd.toString } \ No newline at end of file From 796e934d316750aa39c7fbb6b99cbb25d3aa2236 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 15:07:29 -0800 Subject: [PATCH 271/696] add some docs & some cleanup --- .../main/scala/spark/executor/TaskMetrics.scala | 17 ++++++++++++++++- .../scala/spark/scheduler/SparkListener.scala | 7 ++++++- .../spark/scheduler/cluster/TaskInfo.scala | 2 -- 3 files changed, 22 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 8db71bdc17f7f..c66abdf2cac50 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,10 +1,25 @@ package spark.executor +/** + * + * @param totalBlocksFetched total number of blocks fetched in a shuffle (remote or local) + * @param remoteBlocksFetched number of remote blocks fetched in a shuffle + * @param localBlocksFetched local blocks fetched in a shuffle + * @param shuffleReadMillis total time to read shuffle data + * @param remoteFetchWaitTime total time that is spent blocked waiting for shuffle to fetch remote data + * @param remoteFetchTime the total amount of time for all the shuffle fetches. This adds up time from overlapping + * shuffles, so can be longer than task time + * @param remoteBytesRead total number of remote bytes read from a shuffle + * @param shuffleBytesWritten number of bytes written for a shuffle + * @param executorDeserializeTime time taken on the executor to deserialize this task + * @param executorRunTime time the executor spends actually running the task (including fetching shuffle data) + * @param resultSize the number of bytes this task transmitted back to the driver as the TaskResult + */ case class TaskMetrics( - var shuffleReadMillis: Option[Long], var totalBlocksFetched : Option[Int], var remoteBlocksFetched: Option[Int], var localBlocksFetched: Option[Int], + var shuffleReadMillis: Option[Long], var remoteFetchWaitTime: Option[Long], var remoteFetchTime: Option[Long], var remoteBytesRead: Option[Long], diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index f844a9068016b..0915b3eb5b2a1 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -6,6 +6,9 @@ import spark.{Utils, Logging} import spark.executor.TaskMetrics trait SparkListener { + /** + * called when a stage is completed, with information on the completed stage + */ def onStageCompleted(stageCompleted: StageCompleted) } @@ -14,7 +17,9 @@ sealed trait SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents - +/** + * Simple SparkListener that logs a few summary statistics when each stage completes + */ class StatsReportListener extends SparkListener with Logging { def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 53a3c5dc4d3cb..dfe3c5a85bc25 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,7 +1,5 @@ package spark.scheduler.cluster -import spark.executor.TaskMetrics - /** * Information about a running task attempt inside a TaskSet. */ From 81bd07da265a9fdaad366d8fd46a70df6d5c9806 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 15:21:45 -0800 Subject: [PATCH 272/696] sparkListeners should be a val --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index ce3b62cc4d768..1bf5054f4d794 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -76,7 +76,7 @@ class DAGScheduler( private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] - private[spark] var sparkListeners = ArrayBuffer[SparkListener]() + private[spark] val sparkListeners = ArrayBuffer[SparkListener]() var cacheLocs = new HashMap[Int, Array[List[String]]] From 208edaac1bdb4b0520e5fb204f7bc96079bfa3ed Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 21 Feb 2013 15:22:26 -0800 Subject: [PATCH 273/696] Fixed condition in InputDStream isTimeValid. --- .../src/main/scala/spark/streaming/dstream/InputDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala index a4db44a608e08..3c5d43a60955a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala @@ -29,7 +29,7 @@ abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContex false // Time not valid } else { // Time is valid, but check it it is more than lastValidTime - if (lastValidTime == null || lastValidTime <= time) { + if (lastValidTime != null && time < lastValidTime) { logWarning("isTimeValid called with " + time + " where as last valid time is " + lastValidTime) } lastValidTime = time From 9230617f238b4aab8de95173d9f1cdc0b18cdb43 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 16:55:14 -0800 Subject: [PATCH 274/696] add cleanup iterator --- .../scala/spark/util/CleanupIterator.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 core/src/main/scala/spark/util/CleanupIterator.scala diff --git a/core/src/main/scala/spark/util/CleanupIterator.scala b/core/src/main/scala/spark/util/CleanupIterator.scala new file mode 100644 index 0000000000000..d2093c023011b --- /dev/null +++ b/core/src/main/scala/spark/util/CleanupIterator.scala @@ -0,0 +1,25 @@ +package spark.util + +/** + * Wrapper around an iterator which calls a cleanup method when its finished iterating through its elements + */ +abstract class CleanupIterator[+A, +I <: Iterator[A]](sub: I) extends Iterator[A]{ + def next = sub.next + def hasNext = { + val r = sub.hasNext + if (!r) { + cleanup + } + r + } + + def cleanup +} + +object CleanupIterator { + def apply[A, I <: Iterator[A]](sub: I, cleanupFunction: => Unit) : CleanupIterator[A,I] = { + new CleanupIterator[A,I](sub) { + def cleanup = cleanupFunction + } + } +} \ No newline at end of file From 0f37b43b402ab706a75af30e8912d6a4bd1bd38e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Feb 2013 16:56:28 -0800 Subject: [PATCH 275/696] make the ShuffleFetcher responsible for collecting shuffle metrics, which gives us metrics for CoGroupedRDD and ShuffledRDD --- .../scala/spark/BlockStoreShuffleFetcher.scala | 16 +++++++++++++--- core/src/main/scala/spark/ShuffleFetcher.scala | 5 ++--- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 9 +-------- core/src/main/scala/spark/rdd/ShuffledRDD.scala | 2 +- 4 files changed, 17 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 25532dae28b14..9f5ebe3fd1899 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,13 +1,14 @@ package spark +import executor.TaskMetrics import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} -import spark.util.TimedIterator +import util.{CleanupIterator, TimedIterator} private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { - override def fetch[K, V](shuffleId: Int, reduceId: Int) = { + override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager @@ -46,9 +47,18 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } } + val blockFetcherItr = blockManager.getMultiple(blocksByAddress) val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker itr.setDelegate(blockFetcherItr) - itr + CleanupIterator[(K,V), Iterator[(K,V)]](itr, { + metrics.shuffleReadMillis = Some(itr.getNetMillis) + metrics.remoteFetchTime = Some(itr.remoteFetchTime) + metrics.remoteFetchWaitTime = Some(itr.remoteFetchWaitTime) + metrics.remoteBytesRead = Some(itr.remoteBytesRead) + metrics.totalBlocksFetched = Some(itr.totalBlocks) + metrics.localBlocksFetched = Some(itr.numLocalBlocks) + metrics.remoteBlocksFetched = Some(itr.numRemoteBlocks) + }) } } diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 828cd943f4366..442e9f0269dc4 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,14 +1,13 @@ package spark -import spark.storage.BlockFetchTracker -import spark.util.TimedIterator +import executor.TaskMetrics private[spark] abstract class ShuffleFetcher { /** * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int) : TimedIterator[(K,V)] with BlockFetchTracker + def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) : Iterator[(K,V)] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 38efaa12b5342..65b4621b87ed0 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -102,17 +102,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index) + val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics) for ((k, vs) <- fetchItr) { getSeq(k)(depNum) ++= vs } - context.taskMetrics.shuffleReadMillis = Some(fetchItr.getNetMillis) - context.taskMetrics.remoteFetchTime = Some(fetchItr.remoteFetchTime) - context.taskMetrics.remoteFetchWaitTime = Some(fetchItr.remoteFetchWaitTime) - context.taskMetrics.remoteBytesRead = Some(fetchItr.remoteBytesRead) - context.taskMetrics.totalBlocksFetched = Some(fetchItr.totalBlocks) - context.taskMetrics.localBlocksFetched = Some(fetchItr.numLocalBlocks) - context.taskMetrics.remoteBlocksFetched = Some(fetchItr.numRemoteBlocks) } } JavaConversions.mapAsScalaMap(map).iterator diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index c2f118305f33f..51f02409b6a75 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -28,6 +28,6 @@ class ShuffledRDD[K, V]( override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index) + SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics) } } From f8c3a03d553c7c2818a9c13a7b4f70fe6a9d5afa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 22 Feb 2013 12:54:15 -0800 Subject: [PATCH 276/696] SPARK-702: Replace Function --> JFunction in JavaAPI Suite. In a few places the Scala (rather than Java) function class is used. --- core/src/main/scala/spark/api/java/JavaPairRDD.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index df3af3817dc81..c41207773e92c 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -59,7 +59,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** * Return a new RDD containing only the elements that satisfy a predicate. */ - def filter(f: Function[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] = + def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue())) /** @@ -102,7 +102,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). */ - def combineByKey[C](createCombiner: Function[V, C], + def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner): JavaPairRDD[K, C] = { @@ -309,7 +309,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U](f: Function[V, U]): JavaPairRDD[K, U] = { + def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = { implicit val cm: ClassManifest[U] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] fromRDD(rdd.mapValues(f)) From d4d7993bf5106545ae1056fb6e8d7e2601f60535 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 22 Feb 2013 15:51:37 -0800 Subject: [PATCH 277/696] Several fixes to the work to log when no resources can be used by a job. Fixed some of the messages as well as code style. --- core/src/main/scala/spark/deploy/master/Master.scala | 8 ++++---- .../spark/scheduler/cluster/ClusterScheduler.scala | 12 ++++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index dda25463c7208..b7f167425f5a7 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -205,10 +205,6 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } } } - if (workers.toArray.filter(_.state == WorkerState.ALIVE).size > 0 && - firstApp != None && firstApp.get.executors.size == 0) { - logWarning("Could not find any machines with enough memory. Ensure that SPARK_WORKER_MEM > SPARK_MEM.") - } } def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) { @@ -254,6 +250,10 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor if (firstApp == None) { firstApp = Some(app) } + val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray + if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) { + logWarning("Could not find any workers with enough memory for " + firstApp.get.id) + } return app } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 04d01e9ce8d36..d9c2f9517be50 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -24,7 +24,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // How often to check for speculative tasks val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "5000").toLong + val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong val activeTaskSets = new HashMap[String, TaskSetManager] var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -106,8 +106,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext) starvationTimer.scheduleAtFixedRate(new TimerTask() { override def run() { if (!hasLaunchedTask) { - logWarning("Initial TaskSet has not accepted any offers. " + - "Check the scheduler UI to ensure slaves are registered.") + logWarning("Initial job has not accepted any resources; " + + "check your cluster UI to ensure that workers are registered") + } else { + this.cancel() } } }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) @@ -169,7 +171,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } while (launchedTask) } - if (tasks.size > 0) hasLaunchedTask = true + if (tasks.size > 0) { + hasLaunchedTask = true + } return tasks } } From c8a788692185326c001233bb249d2ed046cd7319 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Fri, 22 Feb 2013 15:16:03 -0800 Subject: [PATCH 278/696] Detect when SendingConnections drop by trying to read them. Comment fix --- .../main/scala/spark/network/Connection.scala | 24 +++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index cd5b7d57f32f5..d1451bc2124c5 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -198,7 +198,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { outbox.synchronized { outbox.addMessage(message) if (channel.isConnected) { - changeConnectionKeyInterest(SelectionKey.OP_WRITE) + changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ) } } } @@ -219,7 +219,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { def finishConnect() { try { channel.finishConnect - changeConnectionKeyInterest(SelectionKey.OP_WRITE) + changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ) logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending") } catch { case e: Exception => { @@ -239,8 +239,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { currentBuffers ++= chunk.buffers } case None => { - changeConnectionKeyInterest(0) - /*key.interestOps(0)*/ + changeConnectionKeyInterest(SelectionKey.OP_READ) return } } @@ -267,6 +266,23 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } } } + + override def read() { + // We don't expect the other side to send anything; so, we just read to detect an error or EOF. + try { + val length = channel.read(ByteBuffer.allocate(1)) + if (length == -1) { // EOF + close() + } else if (length > 0) { + logWarning("Unexpected data read from SendingConnection to " + remoteConnectionManagerId) + } + } catch { + case e: Exception => + logError("Exception while reading SendingConnection to " + remoteConnectionManagerId, e) + callOnExceptionCallback(e) + close() + } + } } From 50cf8c8b79222e2b56dc5c28992adb08bb9c602b Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Fri, 22 Feb 2013 15:23:58 -0800 Subject: [PATCH 279/696] Add fault tolerance test that uses replicated RDDs. --- .../test/scala/spark/DistributedSuite.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 0e2585daa434c..caa4ba3a3705a 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -217,6 +217,27 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter assert(grouped.collect.size === 1) } } + + test("recover from node failures with replication") { + import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity} + DistributedSuite.amMaster = true + // Using more than two nodes so we don't have a symmetric communication pattern and might + // cache a partially correct list of peers. + sc = new SparkContext("local-cluster[3,1,512]", "test") + for (i <- 1 to 3) { + val data = sc.parallelize(Seq(true, false, false, false), 4) + data.persist(StorageLevel.MEMORY_ONLY_2) + + assert(data.count === 4) + assert(data.map(markNodeIfIdentity).collect.size === 4) + assert(data.map(failOnMarkedIdentity).collect.size === 4) + + // Create a new replicated RDD to make sure that cached peer information doesn't cause + // problems. + val data2 = sc.parallelize(Seq(true, true), 2).persist(StorageLevel.MEMORY_ONLY_2) + assert(data2.count === 2) + } + } } object DistributedSuite { From 24c0cd616827b7d7ecdd2c7d770aedd020c006a1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 22 Feb 2013 18:23:29 -0800 Subject: [PATCH 280/696] Fixed resolver for akka-zeromq --- project/SparkBuild.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index fcaa7d3cee9bd..5e7c3b5e3aca2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,6 +162,9 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", + resolvers ++= Seq( + "Akka Repository" at "http://repo.akka.io/releases" + ), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "com.github.sgroschupf" % "zkclient" % "0.1", From d942d3907241d50b693a316785af56023ec218b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 23 Feb 2013 11:19:07 -0800 Subject: [PATCH 281/696] Handle exceptions in RecordReader.close() better (suggested by Jim Donahue) --- core/src/main/scala/spark/rdd/HadoopRDD.scala | 17 +++++++++++------ .../src/main/scala/spark/rdd/NewHadoopRDD.scala | 15 ++++++++++++--- 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 8139a2a40c66f..78097502bca48 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -15,7 +15,7 @@ import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils -import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext} +import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} /** @@ -42,7 +42,7 @@ class HadoopRDD[K, V]( keyClass: Class[K], valueClass: Class[V], minSplits: Int) - extends RDD[(K, V)](sc, Nil) { + extends RDD[(K, V)](sc, Nil) with Logging { // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) @@ -71,7 +71,7 @@ class HadoopRDD[K, V]( reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => reader.close()) + context.addOnCompleteCallback{ () => close() } val key: K = reader.createKey() val value: V = reader.createValue() @@ -88,9 +88,6 @@ class HadoopRDD[K, V]( } gotNext = true } - if (finished) { - reader.close() - } !finished } @@ -104,6 +101,14 @@ class HadoopRDD[K, V]( gotNext = false (key, value) } + + private def close() { + try { + reader.close() + } catch { + case e: Exception => logWarning("Exception in RecordReader.close()", e) + } + } } override def getPreferredLocations(split: Partition): Seq[String] = { diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index ebd4c3f0e2d86..df2361025c753 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -7,7 +7,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext} +import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} private[spark] @@ -26,7 +26,8 @@ class NewHadoopRDD[K, V]( valueClass: Class[V], @transient conf: Configuration) extends RDD[(K, V)](sc, Nil) - with HadoopMapReduceUtil { + with HadoopMapReduceUtil + with Logging { // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) @@ -61,7 +62,7 @@ class NewHadoopRDD[K, V]( reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => reader.close()) + context.addOnCompleteCallback(() => close()) var havePair = false var finished = false @@ -81,6 +82,14 @@ class NewHadoopRDD[K, V]( havePair = false return (reader.getCurrentKey, reader.getCurrentValue) } + + private def close() { + try { + reader.close() + } catch { + case e: Exception => logWarning("Exception in RecordReader.close()", e) + } + } } override def getPreferredLocations(split: Partition): Seq[String] = { From f51b0f93f20e23804b2f95edfb1d86b9c9cee493 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 23 Feb 2013 13:26:59 -0800 Subject: [PATCH 282/696] Adding Java-accessible methods to Vector.scala This is needed for the Strata machine learning tutorial (and also is generally helpful). --- core/src/main/scala/spark/util/Vector.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala index 03559751bc46b..d03cebeea9426 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/spark/util/Vector.scala @@ -10,12 +10,14 @@ class Vector(val elements: Array[Double]) extends Serializable { throw new IllegalArgumentException("Vectors of different length") return Vector(length, i => this(i) + other(i)) } + def add(other: Vector) = +(other) def - (other: Vector): Vector = { if (length != other.length) throw new IllegalArgumentException("Vectors of different length") return Vector(length, i => this(i) - other(i)) } + def subtract(other: Vector) = -(other) def dot(other: Vector): Double = { if (length != other.length) @@ -60,10 +62,13 @@ class Vector(val elements: Array[Double]) extends Serializable { } this } + def addInPlace(other: Vector) = +=(other) def * (scale: Double): Vector = Vector(length, i => this(i) * scale) + def multiply (d: Double) = *(d) def / (d: Double): Vector = this * (1 / d) + def divide (d: Double) = /(d) def unary_- = this * -1 From 931f439be9048fde244b81e3eae4ad5ff9de4adf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 23 Feb 2013 15:40:41 -0800 Subject: [PATCH 283/696] Responding to code review --- core/src/main/scala/spark/util/Vector.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala index d03cebeea9426..835822edb2300 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/spark/util/Vector.scala @@ -10,14 +10,16 @@ class Vector(val elements: Array[Double]) extends Serializable { throw new IllegalArgumentException("Vectors of different length") return Vector(length, i => this(i) + other(i)) } - def add(other: Vector) = +(other) + + def add(other: Vector) = this + other def - (other: Vector): Vector = { if (length != other.length) throw new IllegalArgumentException("Vectors of different length") return Vector(length, i => this(i) - other(i)) } - def subtract(other: Vector) = -(other) + + def subtract(other: Vector) = this - other def dot(other: Vector): Double = { if (length != other.length) @@ -62,13 +64,16 @@ class Vector(val elements: Array[Double]) extends Serializable { } this } - def addInPlace(other: Vector) = +=(other) + + def addInPlace(other: Vector) = this +=other def * (scale: Double): Vector = Vector(length, i => this(i) * scale) - def multiply (d: Double) = *(d) + + def multiply (d: Double) = this * d def / (d: Double): Vector = this * (1 / d) - def divide (d: Double) = /(d) + + def divide (d: Double) = this / d def unary_- = this * -1 From 41285eaae3642b73b3ac5007a35cc4e8f1d7d084 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 23 Feb 2013 16:25:07 -0800 Subject: [PATCH 284/696] Fixed differences in APIs of StreamingContext and JavaStreamingContext. Change rawNetworkStream to rawSocketStream, and added twitter, actor, zeroMQ streams to JavaStreamingContext. Also added them to JavaAPISuite. --- .../streaming/examples/RawNetworkGrep.scala | 2 +- .../spark/streaming/StreamingContext.scala | 16 +- .../api/java/JavaStreamingContext.scala | 201 ++++++++++++++++-- .../java/spark/streaming/JavaAPISuite.java | 37 +++- .../spark/streaming/InputStreamsSuite.scala | 2 +- 5 files changed, 232 insertions(+), 26 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala index 2eec777c54e8d..66e709b7a3338 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala @@ -37,7 +37,7 @@ object RawNetworkGrep { RawTextHelper.warmUp(ssc.sc) val rawStreams = (1 to numStreams).map(_ => - ssc.rawNetworkStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray + ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray val union = ssc.union(rawStreams) union.filter(_.contains("the")).count().foreach(r => println("Grep count: " + r.collect().mkString)) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index d0430b3f3eef8..25c67b279b7d2 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -170,7 +170,8 @@ class StreamingContext private ( * should be same. */ def actorStream[T: ClassManifest]( - props: Props, name: String, + props: Props, + name: String, storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = { networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) @@ -179,19 +180,20 @@ class StreamingContext private ( /** * Create an input stream that receives messages pushed by a zeromq publisher. * @param publisherUrl Url of remote zeromq publisher - * @param zeroMQ topic to subscribe to + * @param subscribe topic to subscribe to * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence * of byte thus it needs the converter(which might be deserializer of bytes) * to translate from sequence of sequence of bytes, where sequence refer to a frame * and sub sequence refer to its payload. * @param storageLevel RDD storage level. Defaults to memory-only. */ - def zeroMQStream[T: ClassManifest](publisherUrl:String, + def zeroMQStream[T: ClassManifest]( + publisherUrl:String, subscribe: Subscribe, bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T], storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = { - + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy + ): DStream[T] = { actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)), "ZeroMQReceiver", storageLevel, supervisorStrategy) } @@ -283,7 +285,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * @tparam T Type of the objects in the received blocks */ - def rawNetworkStream[T: ClassManifest]( + def rawSocketStream[T: ClassManifest]( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 @@ -352,7 +354,7 @@ class StreamingContext private ( def twitterStream( username: String, password: String, - filters: Seq[String], + filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[Status] = { val inputStream = new TwitterInputDStream(this, username, password, filters, storageLevel) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index d2a0ba725fdbf..f3b40b5b88ef9 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -1,16 +1,26 @@ package spark.streaming.api.java -import scala.collection.JavaConversions._ -import java.lang.{Long => JLong, Integer => JInt} - import spark.streaming._ -import dstream._ +import receivers.{ActorReceiver, ReceiverSupervisorStrategy} +import spark.streaming.dstream._ import spark.storage.StorageLevel + import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import spark.api.java.{JavaSparkContext, JavaRDD} + import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} + +import twitter4j.Status + +import akka.actor.Props +import akka.actor.SupervisorStrategy +import akka.zeromq.Subscribe + +import scala.collection.JavaConversions._ + +import java.lang.{Long => JLong, Integer => JInt} import java.io.InputStream import java.util.{Map => JMap} -import spark.api.java.{JavaSparkContext, JavaRDD} /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -128,7 +138,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ - def networkTextStream(hostname: String, port: Int, storageLevel: StorageLevel) + def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel) : JavaDStream[String] = { ssc.socketTextStream(hostname, port, storageLevel) } @@ -186,13 +196,13 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel Storage level to use for storing the received objects * @tparam T Type of the objects in the received blocks */ - def rawNetworkStream[T]( + def rawSocketStream[T]( hostname: String, port: Int, storageLevel: StorageLevel): JavaDStream[T] = { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - JavaDStream.fromDStream(ssc.rawNetworkStream(hostname, port, storageLevel)) + JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel)) } /** @@ -204,10 +214,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param port Port to connect to for receiving data * @tparam T Type of the objects in the received blocks */ - def rawNetworkStream[T](hostname: String, port: Int): JavaDStream[T] = { + def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - JavaDStream.fromDStream(ssc.rawNetworkStream(hostname, port)) + JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port)) } /** @@ -246,11 +256,178 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param hostname Hostname of the slave machine to which the flume data will be sent * @param port Port of the slave machine to which the flume data will be sent */ - def flumeStream(hostname: String, port: Int): - JavaDStream[SparkFlumeEvent] = { + def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = { ssc.flumeStream(hostname, port) } + /** + * Create a input stream that returns tweets received from Twitter. + * @param username Twitter username + * @param password Twitter password + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + username: String, + password: String, + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + ssc.twitterStream(username, password, filters, storageLevel) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param username Twitter username + * @param password Twitter password + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream( + username: String, + password: String, + filters: Array[String] + ): JavaDStream[Status] = { + ssc.twitterStream(username, password, filters) + } + + /** + * Create a input stream that returns tweets received from Twitter. + * @param username Twitter username + * @param password Twitter password + */ + def twitterStream( + username: String, + password: String + ): JavaDStream[Status] = { + ssc.twitterStream(username, password) + } + + /** + * Create an input stream with any arbitrary user implemented actor receiver. + * @param props Props object defining creation of the actor + * @param name Name of the actor + * @param storageLevel Storage level to use for storing the received objects + * + * @note An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e parametrized type of data received and actorStream + * should be same. + */ + def actorStream[T]( + props: Props, + name: String, + storageLevel: StorageLevel, + supervisorStrategy: SupervisorStrategy + ): JavaDStream[T] = { + implicit val cm: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + ssc.actorStream[T](props, name, storageLevel, supervisorStrategy) + } + + /** + * Create an input stream with any arbitrary user implemented actor receiver. + * @param props Props object defining creation of the actor + * @param name Name of the actor + * @param storageLevel Storage level to use for storing the received objects + * + * @note An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e parametrized type of data received and actorStream + * should be same. + */ + def actorStream[T]( + props: Props, + name: String, + storageLevel: StorageLevel + ): JavaDStream[T] = { + implicit val cm: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + ssc.actorStream[T](props, name, storageLevel) + } + + /** + * Create an input stream with any arbitrary user implemented actor receiver. + * @param props Props object defining creation of the actor + * @param name Name of the actor + * + * @note An important point to note: + * Since Actor may exist outside the spark framework, It is thus user's responsibility + * to ensure the type safety, i.e parametrized type of data received and actorStream + * should be same. + */ + def actorStream[T]( + props: Props, + name: String + ): JavaDStream[T] = { + implicit val cm: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + ssc.actorStream[T](props, name) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel Storage level to use for storing the received objects + */ + def zeroMQStream[T]( + publisherUrl:String, + subscribe: Subscribe, + bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T], + storageLevel: StorageLevel, + supervisorStrategy: SupervisorStrategy + ): JavaDStream[T] = { + implicit val cm: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + * @param storageLevel RDD storage level. Defaults to memory-only. + */ + def zeroMQStream[T]( + publisherUrl:String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], + storageLevel: StorageLevel + ): JavaDStream[T] = { + implicit val cm: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) + } + + /** + * Create an input stream that receives messages pushed by a zeromq publisher. + * @param publisherUrl Url of remote zeromq publisher + * @param subscribe topic to subscribe to + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence + * of byte thus it needs the converter(which might be deserializer of bytes) + * to translate from sequence of sequence of bytes, where sequence refer to a frame + * and sub sequence refer to its payload. + */ + def zeroMQStream[T]( + publisherUrl:String, + subscribe: Subscribe, + bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] + ): JavaDStream[T] = { + implicit val cm: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + ssc.zeroMQStream[T](publisherUrl, subscribe, fn) + } + /** * Registers an output stream that will be computed every interval */ diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 4530af5f6af50..3bed500f73e84 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -24,10 +24,16 @@ import spark.streaming.JavaTestUtils; import spark.streaming.JavaCheckpointTestUtils; import spark.streaming.dstream.KafkaPartitionKey; +import spark.streaming.InputStreamsSuite; import java.io.*; import java.util.*; +import akka.actor.Props; +import akka.zeromq.Subscribe; + + + // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. @@ -1205,12 +1211,12 @@ public void testKafkaStream() { } @Test - public void testNetworkTextStream() { + public void testSocketTextStream() { JavaDStream test = ssc.socketTextStream("localhost", 12345); } @Test - public void testNetworkString() { + public void testSocketString() { class Converter extends Function> { public Iterable call(InputStream in) { BufferedReader reader = new BufferedReader(new InputStreamReader(in)); @@ -1239,13 +1245,13 @@ public void testTextFileStream() { } @Test - public void testRawNetworkStream() { - JavaDStream test = ssc.rawNetworkStream("localhost", 12345); + public void testRawSocketStream() { + JavaDStream test = ssc.rawSocketStream("localhost", 12345); } @Test public void testFlumeStream() { - JavaDStream test = ssc.flumeStream("localhost", 12345); + JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); } @Test @@ -1253,4 +1259,25 @@ public void testFileStream() { JavaPairDStream foo = ssc.fileStream("/tmp/foo"); } + + @Test + public void testTwitterStream() { + String[] filters = new String[] { "good", "bad", "ugly" }; + JavaDStream test = ssc.twitterStream("username", "password", filters, StorageLevel.MEMORY_ONLY()); + } + + @Test + public void testActorStream() { + JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY()); + } + + @Test + public void testZeroMQStream() { + JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function>() { + @Override + public Iterable call(byte[][] b) throws Exception { + return null; + } + }); + } } diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index c9f941c5b8b64..1024d3ac9790e 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -42,7 +42,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } - test("network input stream") { + test("socket input stream") { // Start the server val testServer = new TestServer(testPort) testServer.start() From d853aa9658a87d644d483b1fa9d41c29e3ac0673 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 23 Feb 2013 17:42:26 -0800 Subject: [PATCH 285/696] Change spark.cleaner.delay to spark.cleaner.ttl. Updated docs. --- core/src/main/scala/spark/util/MetadataCleaner.scala | 4 ++-- docs/configuration.md | 6 +++--- docs/streaming-programming-guide.md | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index a342d378ffab2..dafa90671214b 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -38,7 +38,7 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging object MetadataCleaner { - def getDelaySeconds = System.getProperty("spark.cleaner.delay", "-1").toInt - def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.delay", delay.toString) } + def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt + def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.ttl", delay.toString) } } diff --git a/docs/configuration.md b/docs/configuration.md index f1ca77aa7826a..04eb6daaa5d01 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -183,7 +183,7 @@ Apart from these, the following properties are also available, and may be useful spark.broadcast.factory - spark.broadcast. HttpBroadcastFactory + spark.broadcast.HttpBroadcastFactory Which broadcast implementation to use. @@ -244,10 +244,10 @@ Apart from these, the following properties are also available, and may be useful - spark.cleaner.delay + spark.cleaner.ttl (disable) - Duration (minutes) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). + Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 71e1bd4aab20f..4a5e3e36a544a 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -335,7 +335,7 @@ For a Spark Streaming application running on a cluster to be stable, the process A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). ## 24/7 Operation -By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.delay` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.delay` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created. +By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created. This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so. From f442e7d83c93c894215427f5ef86c96d61160e0e Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sun, 24 Feb 2013 00:27:14 -0600 Subject: [PATCH 286/696] Update for split->partition rename. --- core/src/main/scala/spark/RDD.scala | 8 +++---- .../main/scala/spark/rdd/SubtractedRDD.scala | 24 +++++++++---------- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 27a4d2d287208..9e8eaee756c56 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -397,17 +397,17 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD with the elements from `this` that are not in `other`. * - * Uses `this` partitioner/split size, because even if `other` is huge, the resulting + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ def subtract(other: RDD[T]): RDD[T] = - subtract(other, partitioner.getOrElse(new HashPartitioner(splits.size))) + subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], numSplits: Int): RDD[T] = - subtract(other, new HashPartitioner(numSplits)) + def subtract(other: RDD[T], numPartitions: Int): RDD[T] = + subtract(other, new HashPartitioner(numPartitions)) /** * Return an RDD with the elements from `this` that are not in `other`. diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 244874e4e0d6b..daf9cc993cf42 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -6,7 +6,7 @@ import spark.RDD import spark.Partitioner import spark.Dependency import spark.TaskContext -import spark.Split +import spark.Partition import spark.SparkEnv import spark.ShuffleDependency import spark.OneToOneDependency @@ -24,7 +24,7 @@ import spark.OneToOneDependency * touch each once to decide if the value needs to be removed. * * This is particularly helpful when `rdd1` is much smaller than `rdd2`, as - * you can use `rdd1`'s partitioner/split size and not worry about running + * you can use `rdd1`'s partitioner/partition size and not worry about running * out of memory because of the size of `rdd2`. */ private[spark] class SubtractedRDD[T: ClassManifest]( @@ -63,16 +63,16 @@ private[spark] class SubtractedRDD[T: ClassManifest]( } } - override def getSplits: Array[Split] = { - val array = new Array[Split](part.numPartitions) + override def getPartitions: Array[Partition] = { + val array = new Array[Partition](part.numPartitions) for (i <- 0 until array.size) { - // Each CoGroupSplit will dependend on rdd1 and rdd2 - array(i) = new CoGroupSplit(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) => + // Each CoGroupPartition will depend on rdd1 and rdd2 + array(i) = new CoGroupPartition(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) => dependencies(j) match { case s: ShuffleDependency[_, _] => new ShuffleCoGroupSplitDep(s.shuffleId) case _ => - new NarrowCoGroupSplitDep(rdd, i, rdd.splits(i)) + new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } }.toList) } @@ -81,21 +81,21 @@ private[spark] class SubtractedRDD[T: ClassManifest]( override val partitioner = Some(part) - override def compute(s: Split, context: TaskContext): Iterator[T] = { - val split = s.asInstanceOf[CoGroupSplit] + override def compute(p: Partition, context: TaskContext): Iterator[T] = { + val partition = p.asInstanceOf[CoGroupPartition] val set = new JHashSet[T] def integrate(dep: CoGroupSplitDep, op: T => Unit) = dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => for (k <- rdd.iterator(itsSplit, context)) op(k.asInstanceOf[T]) case ShuffleCoGroupSplitDep(shuffleId) => - for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, split.index)) + for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index)) op(k.asInstanceOf[T]) } // the first dep is rdd1; add all keys to the set - integrate(split.deps(0), set.add) + integrate(partition.deps(0), set.add) // the second dep is rdd2; remove all of its keys from the set - integrate(split.deps(1), set.remove) + integrate(partition.deps(1), set.remove) set.iterator } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 05fb280d0acd1..77e0eab8299b9 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -241,7 +241,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val b = sc.parallelize(Array(2, 3, 4), 4) val c = a.subtract(b) assert(c.collect().toSet === Set(1)) - assert(c.splits.size === a.splits.size) + assert(c.partitions.size === a.partitions.size) } test("subtract with narrow dependency") { From 37c7a71f9cc574e0a17579e9cf1651daa92553c7 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sun, 24 Feb 2013 00:27:53 -0600 Subject: [PATCH 287/696] Add subtract to JavaRDD, JavaDoubleRDD, and JavaPairRDD. --- .../scala/spark/api/java/JavaDoubleRDD.scala | 23 ++++++++++++++++++- .../scala/spark/api/java/JavaPairRDD.scala | 21 +++++++++++++++++ .../main/scala/spark/api/java/JavaRDD.scala | 20 ++++++++++++++++ 3 files changed, 63 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index da3cb2cd31395..ba00b6a8448f1 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -6,8 +6,8 @@ import spark.api.java.function.{Function => JFunction} import spark.util.StatCounter import spark.partial.{BoundedDouble, PartialResult} import spark.storage.StorageLevel - import java.lang.Double +import spark.Partitioner class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] { @@ -57,6 +57,27 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav */ def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions)) + /** + * Return an RDD with the elements from `this` that are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtract(other: JavaDoubleRDD): JavaDoubleRDD = + fromRDD(srdd.subtract(other)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaDoubleRDD, numPartitions: Int): JavaDoubleRDD = + fromRDD(srdd.subtract(other, numPartitions)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaDoubleRDD, p: Partitioner): JavaDoubleRDD = + fromRDD(srdd.subtract(other, p)) + /** * Return a sampled subset of this RDD. */ diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c41207773e92c..cfbdda88c0417 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -181,6 +181,27 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) + /** + * Return an RDD with the elements from `this` that are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtract(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] = + fromRDD(rdd.subtract(other)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaPairRDD[K, V], numPartitions: Int): JavaPairRDD[K, V] = + fromRDD(rdd.subtract(other, numPartitions)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaPairRDD[K, V], p: Partitioner): JavaPairRDD[K, V] = + fromRDD(rdd.subtract(other, p)) + /** * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine` * is true, Spark will group values of the same key together on the map side before the diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 3ccd6f055ebef..301688889898e 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -55,6 +55,26 @@ JavaRDDLike[T, JavaRDD[T]] { */ def union(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.union(other.rdd)) + /** + * Return an RDD with the elements from `this` that are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtract(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.subtract(other)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaRDD[T], numPartitions: Int): JavaRDD[T] = + wrapRDD(rdd.subtract(other, numPartitions)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = + wrapRDD(rdd.subtract(other, p)) + } object JavaRDD { From b4eb24de9692d05357766b8159ab4d5ec17f89da Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 23 Feb 2013 23:59:45 -0800 Subject: [PATCH 288/696] Updated streaming programming guide with Java API info, and comments from Patrick. --- docs/streaming-programming-guide.md | 85 ++++++++++++++++--- .../streaming/api/java/JavaDStream.scala | 4 +- .../streaming/dstream/QueueInputDStream.scala | 1 + 3 files changed, 76 insertions(+), 14 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 4a5e3e36a544a..ded43e67cd707 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -207,7 +207,7 @@ ssc.stop() {% endhighlight %} # Example -A simple example to start off is the [NetworkWordCount](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/spark/streaming/examples/WordCountNetwork.scala` . +A simple example to start off is the [NetworkWordCount](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` . {% highlight scala %} import spark.streaming.{Seconds, StreamingContext} @@ -216,7 +216,7 @@ import spark.streaming.StreamingContext._ // Create the context and set up a network input stream to receive from a host:port val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1)) -val lines = ssc.networkTextStream(args(1), args(2).toInt) +val lines = ssc.socketTextStream(args(1), args(2).toInt) // Split the lines into words, count them, and print some of the counts on the master val words = lines.flatMap(_.split(" ")) @@ -227,6 +227,8 @@ wordCounts.print() ssc.start() {% endhighlight %} +The `socketTextStream` returns a DStream of lines received from a TCP socket-based source. The `lines` DStream is _transformed_ into a DStream using the `flatMap` operation, where each line is split into words. This `words` DStream is then mapped to a DStream of `(word, 1)` pairs, which is finally reduced to get the word counts. `wordCounts.print()` will print 10 of the counts generated every second. + To run this example on your local machine, you need to first run a Netcat server by using {% highlight bash %} @@ -347,15 +349,19 @@ Tuning the memory usage and GC behavior of Spark applications have been discusse * **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times. # Fault-tolerance Properties -There are two aspects to fault-tolerance - failure of a worker node and that of a driver node. In this section, we are going to discuss the fault-tolerance behavior and the semantics of the processed data. +In this section, we are going to discuss the behavior of Spark Streaming application in the event of a node failure. To understand this, let us remember the basic fault-tolerance properties of Spark's RDDs. + + 1. An RDD is an immutable, and deterministically re-computable, distributed dataset. Each RDD remembers the lineage of deterministic operations that were used on a fault-tolerant input dataset to create it. + 1. If any partition of an RDD is lost due to a worker node failure, then that partition can be re-computed from the original fault-tolerant dataset using the lineage of operations. + +Since all data transformations in Spark Streaming are based on RDD operations, as long as the input dataset is present, all intermediate data can recomputed. Keeping these properties in mind, we are going to discuss the failure semantics in more detail. ## Failure of a Worker Node -In case of the worker node failure, none of the processed data will be lost because -1. All the input data is fault-tolerant (either the data is on HDFS, or it replicated Spark Streaming if received from the network) -1. All intermediate data is expressed as RDDs with their lineage to the input data, which allows Spark to recompute any part of the intermediate data is lost to worker node failure. +There are two failure behaviors based on which input sources are used. -If the worker node where a network data receiver is running fails, then the receiver will be restarted on a different node and it will continue to receive data. However, data that was accepted by the receiver but not yet replicated to other Spark nodes may be lost, which is a fraction of a second of data. +1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure. +1. _Using any input source that receives data through a network_ - For network-based data sources like Kafka and Flume, the received input data is replicated in memory between nodes of the cluster (default replication factor is 2). So if a worker node fails, then the system can recompute the lost from the the left over copy of the input data. However, if the worker node where a network receiver was running fails, then a tiny bit of data may be lost, that is, the data received by the system but not yet replicated to other node(s). The receiver will be started on a different node and it will continue to receive data. Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation always leads to the same result. As a result, all DStream transformations are guaranteed to have _exactly-once_ semantics. That is, the final transformed result will be same even if there were was a worker node failure. However, output operations (like `foreach`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to HDFS using the `saveAs*Files` operations (as the file will simply get over-written by the same data), additional transactions-like mechanisms may be necessary to achieve exactly-once semantics for output operations. @@ -372,11 +378,19 @@ All this is periodically saved in the file `/graph` where val ssc = new StreamingContext(checkpointDirectory) {% endhighlight %} -Calling `ssc.start()` on this new context will restart the receivers and the stream computations. +On calling `ssc.start()` on this new context, the following steps are taken by the system + +1. Schedule the transformations and output operations for all the time steps between the time when the driver failed and when it was restarted. This is also done for those time steps that were scheduled but not processed due to the failure. This will make the system recompute all the intermediate data from the checkpointed RDD files, etc. +1. Restart the network receivers, if any, and continue receiving new data. + +In the current _alpha_ release, there are two different failure behaviors based on which input sources are used. + +1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure. +1. _Using any input source that receives data through a network_ - As aforesaid, the received input data is replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state will not be recovered completely. -In case of stateful operations (that is, `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the intermediate data at the time of failure also needs to be recomputed.This requires two things - (i) the RDD checkpoints and (ii) the data received since the checkpoints. In the current _alpha_ release, the input data received from the network is not saved durably across driver failures (the data is only replicated in memory of the worker processes and gets lost when the driver fails). Only with file input streams (where the data is already durably stored) is the recovery from driver failure complete and all intermediate data is recomputed. In a future release, this will be true for all input streams. Note that for non-stateful operations, with _all_ input streams, the system will recover and continue receiving and processing new data. +In future releases, this behaviour will be fixed for all input sources, that is, all data will be recovered irrespective of which input sources are used. Note that for non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams, the system, upon restarting, will continue to receive and process new data. -To understand the behavior of the system under driver failure, lets consider what will happen with a file input stream Specifically, in the case of the file input stream, it will correctly identify new files that were created while the driver was down and process them in the same way as it would have if the driver had not failed. To explain further in the case of file input stream, we shall use an example. Lets say, files are being generated every second, and a Spark Streaming program reads every new file and output the number of lines in the file. This is what the sequence of outputs would be with and without a driver failure. +To better understand the behavior of the system under driver failure with a HDFS source, lets consider what will happen with a file input stream Specifically, in the case of the file input stream, it will correctly identify new files that were created while the driver was down and process them in the same way as it would have if the driver had not failed. To explain further in the case of file input stream, we shall use an example. Lets say, files are being generated every second, and a Spark Streaming program reads every new file and output the number of lines in the file. This is what the sequence of outputs would be with and without a driver failure. @@ -450,6 +464,55 @@ To understand the behavior of the system under driver failure, lets consider wha If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery. +# Java API + +Similar to [Spark's Java API](java-programming-guide.html), we also provide a Java API for Spark Streaming which allows all its features to be accessible from a Java program. This is defined in [spark.streaming.api.java] (api/streaming/index.html#spark.streaming.api.java.package) package and includes [JavaStreamingContext](api/streaming/index.html#spark.streaming.api.java.JavaStreamingContext) and [JavaDStream](api/streaming/index.html#spark.streaming.api.java.JavaDStream) classes that provide the same methods as their Scala counterparts, but take Java functions (that is, Function, and Function2) and return Java data and collection types. Some of the key points to note are: + +1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#spark.api.java.function.Function) and [Function2](api/core/index.html#spark.api.java.function.Function2) +1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types. + +Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java` + +The streaming context and the socket stream from input source is started by using a `JavaStreamingContext`, that has the same parameters and provides the same input streams as its Scala counterpart. + +{% highlight java %} +JavaStreamingContext ssc = new JavaStreamingContext(mesosUrl, "NetworkWordCount", Seconds(1)); +JavaDStream lines = ssc.socketTextStream(ip, port); +{% endhighlight %} + + +Then the `lines` are split into words by using the `flatMap` function and [FlatMapFunction](api/core/index.html#spark.api.java.function.FlatMapFunction). + +{% highlight java %} +JavaDStream words = lines.flatMap( + new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(x.split(" ")); + } + }); +{% endhighlight %} + +The `words` is then mapped to a [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) of `(word, 1)` pairs using `map` and [PairFunction](api/core/index.html#spark.api.java.function.PairFunction). This is reduced by using `reduceByKey` and [Function2](api/core/index.html#spark.api.java.function.Function2). + +{% highlight java %} +JavaPairDStream wordCounts = words.map( + new PairFunction() { + @Override + public Tuple2 call(String s) throws Exception { + return new Tuple2(s, 1); + } + }).reduceByKey( + new Function2() { + @Override + public Integer call(Integer i1, Integer i2) throws Exception { + return i1 + i2; + } + }); +{% endhighlight %} + + + # Where to Go from Here -* Documentation - [Scala and Java](api/streaming/index.html) +* Documentation - [Scala](api/streaming/index.html#spark.streaming.package) and [Java](api/streaming/index.html#spark.streaming.api.java.package) * More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 51efe6cae8494..4d93f0a5f729e 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -17,9 +17,7 @@ import spark.RDD * * This class contains the basic operations available on all DStreams, such as `map`, `filter` and * `window`. In addition, [[spark.streaming.api.java.JavaPairDStream]] contains operations available - * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. These operations - * are automatically available on any DStream of the right type (e.g., DStream[(Int, Int)] through - * implicit conversions when `spark.streaming.StreamingContext._` is imported. + * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. * * DStreams internally is characterized by a few basic properties: * - A list of other DStreams that the DStream depends on diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala index 024bf3bea4783..6b310bc0b611c 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala @@ -7,6 +7,7 @@ import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer import spark.streaming.{Time, StreamingContext} +private[streaming] class QueueInputDStream[T: ClassManifest]( @transient ssc: StreamingContext, val queue: Queue[RDD[T]], From c1a040db3a82fcd9b5da8a1af3a79d19ace99005 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 24 Feb 2013 11:00:30 -0800 Subject: [PATCH 289/696] Fixed bugs in examples. --- .../java/spark/streaming/examples/JavaNetworkWordCount.java | 2 +- .../main/scala/spark/streaming/examples/ActorWordCount.scala | 3 +-- .../main/scala/spark/streaming/examples/NetworkWordCount.scala | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java index 07342beb02ea6..0e9eadd01b711 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java @@ -23,7 +23,7 @@ */ public class JavaNetworkWordCount { public static void main(String[] args) { - if (args.length < 2) { + if (args.length < 3) { System.err.println("Usage: NetworkWordCount \n" + "In local mode, should be 'local[n]' with n > 1"); System.exit(1); diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 346151c147483..76293fbb96bf7 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -131,8 +131,7 @@ object ActorWordCount { val Seq(master, host, port) = args.toSeq // Create the context and set the batch size - val ssc = new StreamingContext(master, "ActorWordCount", - Seconds(10)) + val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2)) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 7ff70ae2e57eb..5ac6d19b349da 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -16,7 +16,7 @@ import spark.streaming.StreamingContext._ */ object NetworkWordCount { def main(args: Array[String]) { - if (args.length < 2) { + if (args.length < 3) { System.err.println("Usage: NetworkWordCount \n" + "In local mode, should be 'local[n]' with n > 1") System.exit(1) From 68c7934b1a442d7c83fd7d8f70bc03eee1c61c76 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 24 Feb 2013 11:51:45 -0800 Subject: [PATCH 290/696] Fixed missing dependencies in streaming/pom.xml --- streaming/pom.xml | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/streaming/pom.xml b/streaming/pom.xml index d78c39da0d1a6..92b17fc3af6f4 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -20,6 +20,17 @@ lib file://${project.basedir}/lib + + akka-repo + Akka Repository + http://repo.akka.io/releases + + true + + + false + + @@ -53,11 +64,10 @@ 3.0.3 - org.twitter4j - twitter4j-core - 3.0.3 + com.typesafe.akka + akka-zeromq + 2.0.3 - org.scalatest scalatest_${scala.version} From f282bc496062377ba17b6cf4fefc653ebe70b360 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 24 Feb 2013 12:44:12 -0800 Subject: [PATCH 291/696] Changed Algebird from 0.1.9 to 0.1.8 --- examples/pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index f6125444e2a46..7d975875fac3a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -22,7 +22,7 @@ com.twitter algebird-core_2.9.2 - 0.1.9 + 0.1.8 org.scalatest diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5e7c3b5e3aca2..22bdc93602ea5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -155,7 +155,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.9") + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8") ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From 28f8b721f65fc8e699f208c5dc64d90822a85d91 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 24 Feb 2013 13:01:54 -0800 Subject: [PATCH 292/696] Added back the initial spark job before starting streaming receivers --- .../src/main/scala/spark/streaming/NetworkInputTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index 64972fd5cd79f..b159d26c02b2d 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -141,7 +141,7 @@ class NetworkInputTracker( } // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. - //ssc.sparkContext.makeRDD(1 to 100, 100).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() + ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() // Distribute the receivers and start them ssc.sparkContext.runJob(tempRDD, startReceiver) From 2c966c98fbcfe5633d8af20fa7020a2ec8d7ad70 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 24 Feb 2013 13:25:09 -0800 Subject: [PATCH 293/696] Change numSplits to numPartitions in PySpark. --- python/pyspark/join.py | 20 +++++++-------- python/pyspark/rdd.py | 56 +++++++++++++++++++++--------------------- 2 files changed, 38 insertions(+), 38 deletions(-) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 7036c47980667..5f4294fb1b777 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -32,13 +32,13 @@ """ -def _do_python_join(rdd, other, numSplits, dispatch): +def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numSplits).flatMapValues(dispatch) + return vs.union(ws).groupByKey(numPartitions).flatMapValues(dispatch) -def python_join(rdd, other, numSplits): +def python_join(rdd, other, numPartitions): def dispatch(seq): vbuf, wbuf = [], [] for (n, v) in seq: @@ -47,10 +47,10 @@ def dispatch(seq): elif n == 2: wbuf.append(v) return [(v, w) for v in vbuf for w in wbuf] - return _do_python_join(rdd, other, numSplits, dispatch) + return _do_python_join(rdd, other, numPartitions, dispatch) -def python_right_outer_join(rdd, other, numSplits): +def python_right_outer_join(rdd, other, numPartitions): def dispatch(seq): vbuf, wbuf = [], [] for (n, v) in seq: @@ -61,10 +61,10 @@ def dispatch(seq): if not vbuf: vbuf.append(None) return [(v, w) for v in vbuf for w in wbuf] - return _do_python_join(rdd, other, numSplits, dispatch) + return _do_python_join(rdd, other, numPartitions, dispatch) -def python_left_outer_join(rdd, other, numSplits): +def python_left_outer_join(rdd, other, numPartitions): def dispatch(seq): vbuf, wbuf = [], [] for (n, v) in seq: @@ -75,10 +75,10 @@ def dispatch(seq): if not wbuf: wbuf.append(None) return [(v, w) for v in vbuf for w in wbuf] - return _do_python_join(rdd, other, numSplits, dispatch) + return _do_python_join(rdd, other, numPartitions, dispatch) -def python_cogroup(rdd, other, numSplits): +def python_cogroup(rdd, other, numPartitions): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) def dispatch(seq): @@ -89,4 +89,4 @@ def dispatch(seq): elif n == 2: wbuf.append(v) return (vbuf, wbuf) - return vs.union(ws).groupByKey(numSplits).mapValues(dispatch) + return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6b6ab6abd97e9..172ed85fab926 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -215,7 +215,7 @@ def unpack_batches(pair): yield pair return java_cartesian.flatMap(unpack_batches) - def groupBy(self, f, numSplits=None): + def groupBy(self, f, numPartitions=None): """ Return an RDD of grouped items. @@ -224,7 +224,7 @@ def groupBy(self, f, numSplits=None): >>> sorted([(x, sorted(y)) for (x, y) in result]) [(0, [2, 8]), (1, [1, 1, 3, 5])] """ - return self.map(lambda x: (f(x), x)).groupByKey(numSplits) + return self.map(lambda x: (f(x), x)).groupByKey(numPartitions) def pipe(self, command, env={}): """ @@ -274,7 +274,7 @@ def _collect_iterator_through_file(self, iterator): def reduce(self, f): """ - Reduces the elements of this RDD using the specified commutative and + Reduces the elements of this RDD using the specified commutative and associative binary operator. >>> from operator import add @@ -422,22 +422,22 @@ def collectAsMap(self): """ return dict(self.collect()) - def reduceByKey(self, func, numSplits=None): + def reduceByKey(self, func, numPartitions=None): """ Merge the values for each key using an associative reduce function. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. - Output will be hash-partitioned with C{numSplits} splits, or the - default parallelism level if C{numSplits} is not specified. + Output will be hash-partitioned with C{numPartitions} partitions, or + the default parallelism level if C{numPartitions} is not specified. >>> from operator import add >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> sorted(rdd.reduceByKey(add).collect()) [('a', 2), ('b', 1)] """ - return self.combineByKey(lambda x: x, func, func, numSplits) + return self.combineByKey(lambda x: x, func, func, numPartitions) def reduceByKeyLocally(self, func): """ @@ -474,7 +474,7 @@ def countByKey(self): """ return self.map(lambda x: x[0]).countByValue() - def join(self, other, numSplits=None): + def join(self, other, numPartitions=None): """ Return an RDD containing all pairs of elements with matching keys in C{self} and C{other}. @@ -489,9 +489,9 @@ def join(self, other, numSplits=None): >>> sorted(x.join(y).collect()) [('a', (1, 2)), ('a', (1, 3))] """ - return python_join(self, other, numSplits) + return python_join(self, other, numPartitions) - def leftOuterJoin(self, other, numSplits=None): + def leftOuterJoin(self, other, numPartitions=None): """ Perform a left outer join of C{self} and C{other}. @@ -506,9 +506,9 @@ def leftOuterJoin(self, other, numSplits=None): >>> sorted(x.leftOuterJoin(y).collect()) [('a', (1, 2)), ('b', (4, None))] """ - return python_left_outer_join(self, other, numSplits) + return python_left_outer_join(self, other, numPartitions) - def rightOuterJoin(self, other, numSplits=None): + def rightOuterJoin(self, other, numPartitions=None): """ Perform a right outer join of C{self} and C{other}. @@ -523,10 +523,10 @@ def rightOuterJoin(self, other, numSplits=None): >>> sorted(y.rightOuterJoin(x).collect()) [('a', (2, 1)), ('b', (None, 4))] """ - return python_right_outer_join(self, other, numSplits) + return python_right_outer_join(self, other, numPartitions) # TODO: add option to control map-side combining - def partitionBy(self, numSplits, partitionFunc=hash): + def partitionBy(self, numPartitions, partitionFunc=hash): """ Return a copy of the RDD partitioned using the specified partitioner. @@ -535,22 +535,22 @@ def partitionBy(self, numSplits, partitionFunc=hash): >>> set(sets[0]).intersection(set(sets[1])) set([]) """ - if numSplits is None: - numSplits = self.ctx.defaultParallelism + if numPartitions is None: + numPartitions = self.ctx.defaultParallelism # Transferring O(n) objects to Java is too expensive. Instead, we'll - # form the hash buckets in Python, transferring O(numSplits) objects + # form the hash buckets in Python, transferring O(numPartitions) objects # to Java. Each object is a (splitNumber, [objects]) pair. def add_shuffle_key(split, iterator): buckets = defaultdict(list) for (k, v) in iterator: - buckets[partitionFunc(k) % numSplits].append((k, v)) + buckets[partitionFunc(k) % numPartitions].append((k, v)) for (split, items) in buckets.iteritems(): yield str(split) yield dump_pickle(Batch(items)) keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() - partitioner = self.ctx._jvm.PythonPartitioner(numSplits, + partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, id(partitionFunc)) jrdd = pairRDD.partitionBy(partitioner).values() rdd = RDD(jrdd, self.ctx) @@ -561,7 +561,7 @@ def add_shuffle_key(split, iterator): # TODO: add control over map-side aggregation def combineByKey(self, createCombiner, mergeValue, mergeCombiners, - numSplits=None): + numPartitions=None): """ Generic function to combine the elements for each key using a custom set of aggregation functions. @@ -586,8 +586,8 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, >>> sorted(x.combineByKey(str, add, add).collect()) [('a', '11'), ('b', '1')] """ - if numSplits is None: - numSplits = self.ctx.defaultParallelism + if numPartitions is None: + numPartitions = self.ctx.defaultParallelism def combineLocally(iterator): combiners = {} for (k, v) in iterator: @@ -597,7 +597,7 @@ def combineLocally(iterator): combiners[k] = mergeValue(combiners[k], v) return combiners.iteritems() locally_combined = self.mapPartitions(combineLocally) - shuffled = locally_combined.partitionBy(numSplits) + shuffled = locally_combined.partitionBy(numPartitions) def _mergeCombiners(iterator): combiners = {} for (k, v) in iterator: @@ -609,10 +609,10 @@ def _mergeCombiners(iterator): return shuffled.mapPartitions(_mergeCombiners) # TODO: support variant with custom partitioner - def groupByKey(self, numSplits=None): + def groupByKey(self, numPartitions=None): """ Group the values for each key in the RDD into a single sequence. - Hash-partitions the resulting RDD with into numSplits partitions. + Hash-partitions the resulting RDD with into numPartitions partitions. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> sorted(x.groupByKey().collect()) @@ -630,7 +630,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numSplits) + numPartitions) # TODO: add tests def flatMapValues(self, f): @@ -659,7 +659,7 @@ def groupWith(self, other): return self.cogroup(other) # TODO: add variant with custom parittioner - def cogroup(self, other, numSplits=None): + def cogroup(self, other, numPartitions=None): """ For each key k in C{self} or C{other}, return a resulting RDD that contains a tuple with the list of values for that key in C{self} as well @@ -670,7 +670,7 @@ def cogroup(self, other, numSplits=None): >>> sorted(x.cogroup(y).collect()) [('a', ([1], [2])), ('b', ([4], []))] """ - return python_cogroup(self, other, numSplits) + return python_cogroup(self, other, numPartitions) # TODO: `lookup` is disabled because we can't make direct comparisons based # on the key; we need to compare the hash of the key to the hash of the From 5ab37be9831e8a70b2502b14aed1c87cb002a189 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 24 Feb 2013 16:24:52 -0800 Subject: [PATCH 294/696] Fixed class paths and dependencies based on Matei's comments. --- ...ceiver.md => custom-streaming-receiver.md} | 0 docs/streaming-custom-receivers.md | 101 ++++++++++++++++++ docs/streaming-programming-guide.md | 6 +- pom.xml | 6 +- project/SparkBuild.scala | 5 +- run | 5 +- run2.cmd | 3 + streaming/pom.xml | 11 -- 8 files changed, 113 insertions(+), 24 deletions(-) rename docs/{plugin-custom-receiver.md => custom-streaming-receiver.md} (100%) create mode 100644 docs/streaming-custom-receivers.md diff --git a/docs/plugin-custom-receiver.md b/docs/custom-streaming-receiver.md similarity index 100% rename from docs/plugin-custom-receiver.md rename to docs/custom-streaming-receiver.md diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md new file mode 100644 index 0000000000000..0eb4246158e53 --- /dev/null +++ b/docs/streaming-custom-receivers.md @@ -0,0 +1,101 @@ +--- +layout: global +title: Tutorial - Spark streaming, Plugging in a custom receiver. +--- + +A "Spark streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. + +This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. + + +## A quick and naive walk-through + +### Write a simple receiver + +This starts with implementing [Actor](#References) + +Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api. + +{% highlight scala %} + + class SocketTextStreamReceiver (host:String, + port:Int, + bytesToString: ByteString => String) extends Actor with Receiver { + + override def preStart = IOManager(context.system).connect(host, port) + + def receive = { + case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes)) + } + + } + + +{% endhighlight %} + +All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details. + +### A sample spark application + +* First create a Spark streaming context with master url and batchduration. + +{% highlight scala %} + + val ssc = new StreamingContext(master, "WordCountCustomStreamSource", + Seconds(batchDuration)) + +{% endhighlight %} + +* Plug-in the actor configuration into the spark streaming context and create a DStream. + +{% highlight scala %} + + val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8445, z => z.utf8String)),"SocketReceiver") + +{% endhighlight %} + +* Process it. + +{% highlight scala %} + + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + + wordCounts.print() + ssc.start() + + +{% endhighlight %} + +* After processing it, stream can be tested using the netcat utility. + + $ nc -l localhost 8445 + hello world + hello hello + + +## Multiple homogeneous/heterogeneous receivers. + +A DStream union operation is provided for taking union on multiple input streams. + +{% highlight scala %} + + val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8445, z => z.utf8String)),"SocketReceiver") + + // Another socket stream receiver + val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver( + "localhost",8446, z => z.utf8String)),"SocketReceiver") + + val union = lines.union(lines2) + +{% endhighlight %} + +Above stream can be easily process as described earlier. + +_A more comprehensive example is provided in the spark streaming examples_ + +## References + +1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index ded43e67cd707..0e618a06c796f 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -365,14 +365,14 @@ There are two failure behaviors based on which input sources are used. Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation always leads to the same result. As a result, all DStream transformations are guaranteed to have _exactly-once_ semantics. That is, the final transformed result will be same even if there were was a worker node failure. However, output operations (like `foreach`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to HDFS using the `saveAs*Files` operations (as the file will simply get over-written by the same data), additional transactions-like mechanisms may be necessary to achieve exactly-once semantics for output operations. -## Failure of a Driver Node -A system that is required to operate 24/7 needs to be able tolerate the failure of the drive node as well. Spark Streaming does this by saving the state of the DStream computation periodically to a HDFS file, that can be used to restart the streaming computation in the event of a failure of the driver node. To elaborate, the following state is periodically saved to a file. +## Failure of the Driver Node +A system that is required to operate 24/7 needs to be able tolerate the failure of the driver node as well. Spark Streaming does this by saving the state of the DStream computation periodically to a HDFS file, that can be used to restart the streaming computation in the event of a failure of the driver node. This checkpointing is enabled by setting a HDFS directory for checkpointing using `ssc.checkpoint()` as described [earlier](#rdd-checkpointing-within-dstreams). To elaborate, the following state is periodically saved to a file. 1. The DStream operator graph (input streams, output streams, etc.) 1. The configuration of each DStream (checkpoint interval, etc.) 1. The RDD checkpoint files of each DStream -All this is periodically saved in the file `/graph` where `` is the HDFS path set using `ssc.checkpoint(...)` as described earlier. To recover, a new Streaming Context can be created with this directory by using +All this is periodically saved in the file `/graph`. To recover, a new Streaming Context can be created with this directory by using {% highlight scala %} val ssc = new StreamingContext(checkpointDirectory) diff --git a/pom.xml b/pom.xml index 7e06cae052b58..99eb17856a813 100644 --- a/pom.xml +++ b/pom.xml @@ -84,9 +84,9 @@ - typesafe-repo - Typesafe Repository - http://repo.typesafe.com/typesafe/releases/ + akka-repo + Akka Repository + http://repo.akka.io/releases/ true diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 22bdc93602ea5..b0b6e21681b29 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -114,7 +114,7 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", + "Akka Repository" at "http://repo.akka.io/releases/", "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Spray Repository" at "http://repo.spray.cc/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", @@ -162,9 +162,6 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", - resolvers ++= Seq( - "Akka Repository" at "http://repo.akka.io/releases" - ), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "com.github.sgroschupf" % "zkclient" % "0.1", diff --git a/run b/run index 6b2d84d48dbf0..ecbf7673c660f 100755 --- a/run +++ b/run @@ -111,14 +111,13 @@ CLASSPATH+=":$FWDIR/conf" CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes" if [ -n "$SPARK_TESTING" ] ; then CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" fi CLASSPATH+=":$CORE_DIR/src/main/resources" CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -for jar in `find "$STREAMING_DIR/lib" -name '*jar'`; do - CLASSPATH+=":$jar" -done +CLASSPATH+=":$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH+=":$FWDIR/lib_managed/jars/*" CLASSPATH+=":$FWDIR/lib_managed/bundles/*" diff --git a/run2.cmd b/run2.cmd index c913a5195ef95..705a4d1ff6830 100644 --- a/run2.cmd +++ b/run2.cmd @@ -47,11 +47,14 @@ set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel +set STREAMING_DIR=%FWDIR%streaming set PYSPARK_DIR=%FWDIR%python rem Build up classpath set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources +set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes +set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\* set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\* set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* diff --git a/streaming/pom.xml b/streaming/pom.xml index 92b17fc3af6f4..15523eadcb781 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -20,17 +20,6 @@ lib file://${project.basedir}/lib - - akka-repo - Akka Repository - http://repo.akka.io/releases - - true - - - false - - From abb54718658f8cfa0005fe99f4159eabce074f45 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 24 Feb 2013 16:32:44 -0800 Subject: [PATCH 295/696] Removing duplicate doc. --- docs/custom-streaming-receiver.md | 101 ----------------------------- docs/streaming-custom-receivers.md | 4 +- 2 files changed, 2 insertions(+), 103 deletions(-) delete mode 100644 docs/custom-streaming-receiver.md diff --git a/docs/custom-streaming-receiver.md b/docs/custom-streaming-receiver.md deleted file mode 100644 index 0eb4246158e53..0000000000000 --- a/docs/custom-streaming-receiver.md +++ /dev/null @@ -1,101 +0,0 @@ ---- -layout: global -title: Tutorial - Spark streaming, Plugging in a custom receiver. ---- - -A "Spark streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. - -This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. - - -## A quick and naive walk-through - -### Write a simple receiver - -This starts with implementing [Actor](#References) - -Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api. - -{% highlight scala %} - - class SocketTextStreamReceiver (host:String, - port:Int, - bytesToString: ByteString => String) extends Actor with Receiver { - - override def preStart = IOManager(context.system).connect(host, port) - - def receive = { - case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes)) - } - - } - - -{% endhighlight %} - -All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details. - -### A sample spark application - -* First create a Spark streaming context with master url and batchduration. - -{% highlight scala %} - - val ssc = new StreamingContext(master, "WordCountCustomStreamSource", - Seconds(batchDuration)) - -{% endhighlight %} - -* Plug-in the actor configuration into the spark streaming context and create a DStream. - -{% highlight scala %} - - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8445, z => z.utf8String)),"SocketReceiver") - -{% endhighlight %} - -* Process it. - -{% highlight scala %} - - val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) - - wordCounts.print() - ssc.start() - - -{% endhighlight %} - -* After processing it, stream can be tested using the netcat utility. - - $ nc -l localhost 8445 - hello world - hello hello - - -## Multiple homogeneous/heterogeneous receivers. - -A DStream union operation is provided for taking union on multiple input streams. - -{% highlight scala %} - - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8445, z => z.utf8String)),"SocketReceiver") - - // Another socket stream receiver - val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8446, z => z.utf8String)),"SocketReceiver") - - val union = lines.union(lines2) - -{% endhighlight %} - -Above stream can be easily process as described earlier. - -_A more comprehensive example is provided in the spark streaming examples_ - -## References - -1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 0eb4246158e53..5476c00d020cb 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -1,9 +1,9 @@ --- layout: global -title: Tutorial - Spark streaming, Plugging in a custom receiver. +title: Tutorial - Spark Streaming, Plugging in a custom receiver. --- -A "Spark streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. +A "Spark Streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. From c44ccf2862e8be183ccecac3bf61f9651b21984a Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sun, 24 Feb 2013 23:54:03 -0600 Subject: [PATCH 296/696] Use default parallelism if its set. --- core/src/main/scala/spark/Partitioner.scala | 23 +++++++++++++++----- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 03966f1c96c01..eec0e8dd79da4 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -10,12 +10,21 @@ abstract class Partitioner extends Serializable { } object Partitioner { + + private val useDefaultParallelism = System.getProperty("spark.default.parallelism") != null + /** - * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. If any of - * the RDDs already has a partitioner, choose that one, otherwise use a default HashPartitioner. + * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. + * + * If any of the RDDs already has a partitioner, choose that one. * - * The number of partitions will be the same as the number of partitions in the largest upstream - * RDD, as this should be least likely to cause out-of-memory errors. + * Otherwise, we use a default HashPartitioner. For the number of partitions, if + * spark.default.parallelism is set, then we'll use the value from SparkContext + * defaultParallelism, otherwise we'll use the max number of upstream partitions. + * + * Unless spark.default.parallelism is set, He number of partitions will be the + * same as the number of partitions in the largest upstream RDD, as this should + * be least likely to cause out-of-memory errors. * * We use two method parameters (rdd, others) to enforce callers passing at least 1 RDD. */ @@ -24,7 +33,11 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - return new HashPartitioner(bySize.head.partitions.size) + if (useDefaultParallelism) { + return new HashPartitioner(rdd.context.defaultParallelism) + } else { + return new HashPartitioner(bySize.head.partitions.size) + } } } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 2099999ed7ee3..8411291b2caa3 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -235,7 +235,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(rdd.values.collect().toList === List("a", "b")) } - test("default partitioner uses split size") { + test("default partitioner uses partition size") { sc = new SparkContext("local", "test") // specify 2000 partitions val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) From 254acb16662cd1d35ab6a7a9f7f69494e5330db3 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 25 Feb 2013 13:37:07 +0530 Subject: [PATCH 297/696] Moving akka dependency resolver to shared. --- project/SparkBuild.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b0b6e21681b29..25c232837358d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,6 +44,9 @@ object SparkBuild extends Build { transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), + // shared between both core and streaming. + resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), + // For Sonatype publishing resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"), @@ -114,7 +117,6 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - "Akka Repository" at "http://repo.akka.io/releases/", "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Spray Repository" at "http://repo.spray.cc/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", From 6ae9a22c3e15b3a4e2b0445cb89d855091746a18 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 10:28:08 -0800 Subject: [PATCH 298/696] Get spark.default.paralellism on each call to defaultPartitioner, instead of only once, in case the user changes it across Spark uses --- core/src/main/scala/spark/Partitioner.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index eec0e8dd79da4..6f8cd17c88220 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -10,9 +10,6 @@ abstract class Partitioner extends Serializable { } object Partitioner { - - private val useDefaultParallelism = System.getProperty("spark.default.parallelism") != null - /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. * @@ -33,7 +30,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (useDefaultParallelism) { + if (System.getProperty("spark.default.parallelism") != null) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) From 8f17387d9723c1359c86ff1773cf6613f02bb9c6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 25 Feb 2013 10:31:06 -0800 Subject: [PATCH 299/696] remove bogus comment --- core/src/main/scala/spark/TaskContext.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index 5aadd5e261da1..dd0609026ace3 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -9,8 +9,6 @@ class TaskContext( val attemptId: Long, val taskMetrics: TaskMetrics = TaskMetrics.empty() ) extends Serializable { - //by adding Task here, I'm destroying the separation between Task & TaskContext ... not sure why they need to - // be separate @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit] From 25f737804ac1f4a925aaf6b1a147e63bc08a75f4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 11:53:55 -0800 Subject: [PATCH 300/696] Change tabs to spaces --- run | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/run b/run index ecbf7673c660f..fd06fbe7c7e8b 100755 --- a/run +++ b/run @@ -28,21 +28,21 @@ fi # Add java opts for master, worker, executor. The opts maybe null case "$1" in - 'spark.deploy.master.Master') - SPARK_JAVA_OPTS+=" $SPARK_MASTER_OPTS" - ;; - 'spark.deploy.worker.Worker') - SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS" - ;; - 'spark.executor.StandaloneExecutorBackend') - SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" - ;; - 'spark.executor.MesosExecutorBackend') - SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" - ;; - 'spark.repl.Main') - SPARK_JAVA_OPTS+=" $SPARK_REPL_OPTS" - ;; + 'spark.deploy.master.Master') + SPARK_JAVA_OPTS+=" $SPARK_MASTER_OPTS" + ;; + 'spark.deploy.worker.Worker') + SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS" + ;; + 'spark.executor.StandaloneExecutorBackend') + SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + ;; + 'spark.executor.MesosExecutorBackend') + SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + ;; + 'spark.repl.Main') + SPARK_JAVA_OPTS+=" $SPARK_REPL_OPTS" + ;; esac if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then From 85a85646d99bdb256f1f667d81647eab45bc6e28 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Feb 2013 12:25:43 -0800 Subject: [PATCH 301/696] Disable remote lifecycle logging from Akka. This changes the default setting to `off` for remote lifecycle events. When this is on, it is very chatty at the INFO level. It also prints out several ERROR messages sometimes when sc.stop() is called. --- core/src/main/scala/spark/util/AkkaUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 30aec5a663040..f7b9e2d4ce986 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -37,7 +37,7 @@ private[spark] object AkkaUtils { akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" - akka.remote.log-remote-lifecycle-events = on + akka.remote.log-remote-lifecycle-events = off akka.remote.netty.hostname = "%s" akka.remote.netty.port = %d akka.remote.netty.connection-timeout = %ds From 3c7dcb61ab335ea19107354f091249ca8ca9d930 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 13:15:12 -0800 Subject: [PATCH 302/696] Use a single setting for disabling API doc build --- docs/README.md | 2 +- docs/_plugins/copy_api_dirs.rb | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/README.md b/docs/README.md index 887f407f18c66..c2b3497bb374e 100644 --- a/docs/README.md +++ b/docs/README.md @@ -33,4 +33,4 @@ Similarly, you can build just the PySpark epydoc by running `epydoc --config epy When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). -NOTE: To skip the step of building and copying over the scaladoc when you build the docs, run `SKIP_SCALADOC=1 jekyll`. Similarly, `SKIP_EPYDOC=1 jekyll` will skip PySpark API doc generation. +NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index e400dec619f77..d77e53963c2f7 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -1,7 +1,8 @@ require 'fileutils' include FileUtils -if ENV['SKIP_SCALADOC'] != '1' +if ENV['SKIP_API'] != '1' + # Build Scaladoc for Java/Scala projects = ["core", "examples", "repl", "bagel", "streaming"] puts "Moving to project root and building scaladoc." @@ -27,9 +28,8 @@ puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) end -end -if ENV['SKIP_EPYDOC'] != '1' + # Build Epydoc for Python puts "Moving to python directory and building epydoc." cd("../python") puts `epydoc --config epydoc.conf` From 848321f9101987a25dfae000fb2ebf72ec203f69 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 13:15:30 -0800 Subject: [PATCH 303/696] Change doc color scheme slightly for Spark 0.7 (to differ from 0.6) --- docs/css/bootstrap.css | 30 +++++++++++++++--------------- docs/css/bootstrap.min.css | 2 +- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/css/bootstrap.css b/docs/css/bootstrap.css index 0664207aeebb7..b51ef7b42b438 100644 --- a/docs/css/bootstrap.css +++ b/docs/css/bootstrap.css @@ -2719,16 +2719,16 @@ table .span24 { .dropdown-menu .active > a:hover { color: #ffffff; text-decoration: none; - background-color: #0088cc; - background-color: #0081c2; - background-image: linear-gradient(to bottom, #0088cc, #0077b3); - background-image: -moz-linear-gradient(top, #0088cc, #0077b3); - background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3)); - background-image: -webkit-linear-gradient(top, #0088cc, #0077b3); - background-image: -o-linear-gradient(top, #0088cc, #0077b3); + background-color: #0098cc; + background-color: #0098cc; + background-image: linear-gradient(to bottom, #0098cc, #0087b3); + background-image: -moz-linear-gradient(top, #0098cc, #0087b3); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0098cc), to(#0087b3)); + background-image: -webkit-linear-gradient(top, #0098cc, #0087b3); + background-image: -o-linear-gradient(top, #0098cc, #0087b3); background-repeat: repeat-x; outline: 0; - filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0); + filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc', endColorstr='#ff0087b3', GradientType=0); } .dropdown-menu .disabled > a, @@ -4025,18 +4025,18 @@ input[type="submit"].btn.btn-mini { min-height: 40px; padding-right: 20px; padding-left: 20px; - background-color: #fafafa; - background-image: -moz-linear-gradient(top, #ffffff, #f2f2f2); - background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#f2f2f2)); - background-image: -webkit-linear-gradient(top, #ffffff, #f2f2f2); - background-image: -o-linear-gradient(top, #ffffff, #f2f2f2); - background-image: linear-gradient(to bottom, #ffffff, #f2f2f2); + background-color: #fadafa; + background-image: -moz-linear-gradient(top, #ffddff, #f2d2f2); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffddff), to(#f2d2f2)); + background-image: -webkit-linear-gradient(top, #ffddff, #f2d2f2); + background-image: -o-linear-gradient(top, #ffddff, #f2d2f2); + background-image: linear-gradient(to bottom, #ffddff, #f2d2f2); background-repeat: repeat-x; border: 1px solid #d4d4d4; -webkit-border-radius: 4px; -moz-border-radius: 4px; border-radius: 4px; - filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff2f2f2', GradientType=0); + filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff', endColorstr='#fff2d2f2', GradientType=0); -webkit-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); -moz-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css index 3119038a6b7ab..eb48138e08d68 100644 --- a/docs/css/bootstrap.min.css +++ b/docs/css/bootstrap.min.css @@ -6,4 +6,4 @@ * http://www.apache.org/licenses/LICENSE-2.0 * * Designed and built with all the love in the world @twitter by @mdo and @fat. - */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#08c;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#08c;background-color:#0081c2;background-image:linear-gradient(to bottom,#08c,#0077b3);background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#f2f2f2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f2f2f2));background-image:-webkit-linear-gradient(top,#fff,#f2f2f2);background-image:-o-linear-gradient(top,#fff,#f2f2f2);background-image:linear-gradient(to bottom,#fff,#f2f2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fff2f2f2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} + */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0098cc;background-color:#0098cc;background-image:-moz-linear-gradient(top,#0098cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0098cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0098cc,#0087b3);background-image:-o-linear-gradient(top,#0098cc,#0087b3);background-image:linear-gradient(to bottom,#0098cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0098cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0098cc,#0087b3);background-image:-moz-linear-gradient(top,#0098cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0098cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0098cc,#0087b3);background-image:-o-linear-gradient(top,#0098cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#e2f2e2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e2f2e2));background-image:-webkit-linear-gradient(top,#fff,#e2f2e2);background-image:-o-linear-gradient(top,#fff,#e2f2e2);background-image:linear-gradient(to bottom,#fff,#e2f2e2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff',endColorstr='#ffe2f2e2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} From 01bd136ba5c5f47a82b67b8d34df5a6aeaf79080 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 13:27:11 -0800 Subject: [PATCH 304/696] Use public method sparkContext instead of protected sc in streaming examples --- .../src/main/scala/spark/streaming/examples/QueueStream.scala | 4 ++-- .../main/scala/spark/streaming/examples/RawNetworkGrep.scala | 2 +- .../spark/streaming/examples/clickstream/PageViewStream.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala index 2a265d021d2a0..d83db7784d4ab 100644 --- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala @@ -30,10 +30,10 @@ object QueueStream { // Create and push some RDDs into for (i <- 1 to 30) { - rddQueue += ssc.sc.makeRDD(1 to 1000, 10) + rddQueue += sc.sparkContext.makeRDD(1 to 1000, 10) Thread.sleep(1000) } ssc.stop() System.exit(0) } -} \ No newline at end of file +} diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala index 66e709b7a3338..cce00014261ee 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala @@ -34,7 +34,7 @@ object RawNetworkGrep { val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis)) // Warm up the JVMs on master and slave for JIT compilation to kick in - RawTextHelper.warmUp(ssc.sc) + RawTextHelper.warmUp(sc.sparkContext) val rawStreams = (1 to numStreams).map(_ => ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index fba72519a9244..9a2ba30ee4baf 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -60,7 +60,7 @@ object PageViewStream { .map("Unique active users: " + _) // An external dataset we want to join to this stream - val userList = ssc.sc.parallelize( + val userList = ssc.sparkContext.parallelize( Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq) metric match { From 6b87ef7c8625260e6b99cd6c6ebd16761efd5cda Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 14:01:16 -0800 Subject: [PATCH 305/696] Fix compile error --- .../src/main/scala/spark/streaming/examples/QueueStream.scala | 2 +- .../main/scala/spark/streaming/examples/RawNetworkGrep.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala index d83db7784d4ab..e9cb7b55ea132 100644 --- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala @@ -30,7 +30,7 @@ object QueueStream { // Create and push some RDDs into for (i <- 1 to 30) { - rddQueue += sc.sparkContext.makeRDD(1 to 1000, 10) + rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) Thread.sleep(1000) } ssc.stop() diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala index cce00014261ee..49b3223eecd19 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala @@ -34,7 +34,7 @@ object RawNetworkGrep { val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis)) // Warm up the JVMs on master and slave for JIT compilation to kick in - RawTextHelper.warmUp(sc.sparkContext) + RawTextHelper.warmUp(ssc.sparkContext) val rawStreams = (1 to numStreams).map(_ => ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray From 1ef58dadccb878f70ffd638b2d0f260c2939eda9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 14:01:24 -0800 Subject: [PATCH 306/696] Add a config property for Akka lifecycle event logging --- core/src/main/scala/spark/util/AkkaUtils.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index f7b9e2d4ce986..3e805b78314c7 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -31,20 +31,22 @@ private[spark] object AkkaUtils { val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt + val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" - akka.remote.log-remote-lifecycle-events = off akka.remote.netty.hostname = "%s" akka.remote.netty.port = %d akka.remote.netty.connection-timeout = %ds akka.remote.netty.message-frame-size = %d MiB akka.remote.netty.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d - """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize)) + akka.remote.log-remote-lifecycle-events = %s + """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, + if (lifecycleEvents) "on" else "off")) val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader) From 5d4a0ac794bff3bc3c579ab6e0658b17bdeeab43 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 14:23:03 -0800 Subject: [PATCH 307/696] Some tweaks to docs --- docs/_layouts/global.html | 6 +++--- docs/index.md | 2 +- docs/streaming-programming-guide.md | 10 +++++----- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 94baa634aaf4e..4b3da7ef573de 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -53,11 +53,11 @@ diff --git a/docs/index.md b/docs/index.md index c6ef507cb0997..fd74a051e0d52 100644 --- a/docs/index.md +++ b/docs/index.md @@ -58,9 +58,9 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). * [Quick Start](quick-start.html): a quick introduction to the Spark API; start here! * [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API -* [Streaming Programming Guide](streaming-programming-guide.html): an API preview of Spark Streaming * [Java Programming Guide](java-programming-guide.html): using Spark from Java * [Python Programming Guide](python-programming-guide.html): using Spark from Python +* [Spark Streaming Guide](streaming-programming-guide.html): using the alpha release of Spark Streaming **API Docs:** diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 0e618a06c796f..7e913b783d89e 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -7,9 +7,9 @@ title: Spark Streaming Programming Guide {:toc} # Overview -A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collection of elements) representing a continuous stream of data. DStreams can created from live incoming data (such as data from a socket, Kafka, etc.) or it can be generated by transformation of existing DStreams using parallel operators like map, reduce, and window. The basic processing model is as follows: -(i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides the data into batches. Each batch of data is treated as a RDD, that is a immutable and parallel collection of data. These input data RDDs are automatically persisted in memory (serialized by default) and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively referred to as an InputDStream. -(ii) Data received by InputDStreams are processed processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures. +A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transformong existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: +(i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides it into batches. Each batch of data is treated as an RDD, that is, an immutable parallel collection of data. These input RDDs are saved in memory and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively called an InputDStream. +(ii) Data received by InputDStreams are processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures. This guide shows some how to start programming with DStreams. @@ -20,7 +20,7 @@ The first thing a Spark Streaming program must do is create a `StreamingContext` new StreamingContext(master, jobName, batchDuration) {% endhighlight %} -The `master` parameter is either the [Mesos master URL](running-on-mesos.html) (for running on a cluster)or the special "local" string (for local mode) that is used to create a Spark Context. For more information about this please refer to the [Spark programming guide](scala-programming-guide.html). The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the Mesos web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Starting with something conservative like 5 seconds maybe a good start. See [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. +The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing mode. The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the cluster UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Starting with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. This constructor creates a SparkContext object using the given `master` and `jobName` parameters. However, if you already have a SparkContext or you need to create a custom SparkContext by specifying list of JARs, then a StreamingContext can be created from the existing SparkContext, by using {% highlight scala %} @@ -514,5 +514,5 @@ JavaPairDStream wordCounts = words.map( # Where to Go from Here -* Documentation - [Scala](api/streaming/index.html#spark.streaming.package) and [Java](api/streaming/index.html#spark.streaming.api.java.package) +* API docs - [Scala](api/streaming/index.html#spark.streaming.package) and [Java](api/streaming/index.html#spark.streaming.api.java.package) * More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) From 568bdaf8ae784c9b832f564cb99f1b81ad487f73 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 14:34:55 -0800 Subject: [PATCH 308/696] Set spark.deploy.spreadOut to true by default in 0.7 (improves locality) --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index b7f167425f5a7..d5188a4779e3c 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -43,7 +43,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "false").toBoolean + val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at spark://" + ip + ":" + port) From 50ce0516e6bce2a528a28e4719713802923f68ab Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Feb 2013 14:28:23 -0800 Subject: [PATCH 309/696] Some changes to streaming failure docs. TD gave me the go-ahead to just make these changes: - Define stateful dstream - Some minor wording fixes --- docs/streaming-programming-guide.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 0e618a06c796f..d58d030ebd45c 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -286,7 +286,9 @@ For input streams that receive data from the network (that is, subclasses of Net Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence). # RDD Checkpointing within DStreams -DStreams created by stateful operations like `updateStateByKey` require the RDDs in the DStream to be periodically saved to HDFS files for checkpointing. This is because, unless checkpointed, the lineage of operations of the state RDDs can increase indefinitely (since each RDD in the DStream depends on the previous RDD). This leads to two problems - (i) the size of Spark tasks increase proportionally with the RDD lineage leading higher task launch times, (ii) no limit on the amount of recomputation required on failure. Checkpointing RDDs at some interval by writing them to HDFS allows the lineage to be truncated. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. +A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. Because stateful operations have an infinitely growing lineage, their state must be periodically checkpointed to HDFS. + +Checkpointing prevents extra computation during both failure recovery and normal operations by transparently truncating this lineage. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using @@ -380,15 +382,15 @@ val ssc = new StreamingContext(checkpointDirectory) On calling `ssc.start()` on this new context, the following steps are taken by the system -1. Schedule the transformations and output operations for all the time steps between the time when the driver failed and when it was restarted. This is also done for those time steps that were scheduled but not processed due to the failure. This will make the system recompute all the intermediate data from the checkpointed RDD files, etc. +1. Schedule the transformations and output operations for all the time steps between the time when the driver failed and when it last checkpointed. This is also done for those time steps that were previously scheduled but not processed due to the failure. This will make the system recompute all the intermediate data from the checkpointed RDD files, etc. 1. Restart the network receivers, if any, and continue receiving new data. In the current _alpha_ release, there are two different failure behaviors based on which input sources are used. 1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure. -1. _Using any input source that receives data through a network_ - As aforesaid, the received input data is replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state will not be recovered completely. +1. _Using any input source that receives data through a network_ - The received input data is replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state will not be recovered completely. -In future releases, this behaviour will be fixed for all input sources, that is, all data will be recovered irrespective of which input sources are used. Note that for non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams, the system, upon restarting, will continue to receive and process new data. +In future releases, we will support full recoverability for all input sources. Note that for non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams, the system, upon restarting, will continue to receive and process new data. To better understand the behavior of the system under driver failure with a HDFS source, lets consider what will happen with a file input stream Specifically, in the case of the file input stream, it will correctly identify new files that were created while the driver was down and process them in the same way as it would have if the driver had not failed. To explain further in the case of file input stream, we shall use an example. Lets say, files are being generated every second, and a Spark Streaming program reads every new file and output the number of lines in the file. This is what the sequence of outputs would be with and without a driver failure. From 07f261876930a07161a7bcdb713430113e9e9ec8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Feb 2013 15:09:46 -0800 Subject: [PATCH 310/696] Minor changes based on feedback --- docs/streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index d58d030ebd45c..4b351d07e51f7 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -286,9 +286,9 @@ For input streams that receive data from the network (that is, subclasses of Net Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence). # RDD Checkpointing within DStreams -A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. Because stateful operations have an infinitely growing lineage, their state must be periodically checkpointed to HDFS. +A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. -Checkpointing prevents extra computation during both failure recovery and normal operations by transparently truncating this lineage. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. +Because stateful operations have an infinitely growing lineage, their state must be periodically checkpointed to HDFS. Checkpointing truncates the lineage graph and stores the of the DStream in HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using From 490f056cddc3dc02066a1e2414be6576d6441d51 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 15:13:30 -0800 Subject: [PATCH 311/696] Allow passing sparkHome and JARs to StreamingContext constructor Also warns if spark.cleaner.ttl is not set in the version where you pass your own SparkContext. --- core/src/main/scala/spark/SparkContext.scala | 2 +- .../spark/api/java/JavaSparkContext.scala | 4 +- docs/scala-programming-guide.md | 4 +- docs/streaming-programming-guide.md | 10 ++--- .../scala/spark/streaming/Checkpoint.scala | 1 + .../spark/streaming/StreamingContext.scala | 40 ++++++++++++++----- .../api/java/JavaStreamingContext.scala | 36 +++++++++++++++++ .../streaming/dstream/KafkaInputDStream.scala | 1 + 8 files changed, 76 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index df23710d469f2..7503b1a5ea30e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -64,7 +64,7 @@ class SparkContext( val appName: String, val sparkHome: String = null, val jars: Seq[String] = Nil, - environment: Map[String, String] = Map()) + val environment: Map[String, String] = Map()) extends Logging { // Ensure logging is initialized before we spawn any threads diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index f75fc27c7b2f6..5f18b1e15bd69 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -31,8 +31,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the slave nodes - * @param jars Collection of JARs to send to the cluster. These can be paths on the local file - * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param jarFile JAR file to send to the cluster. This can be a path on the local file system + * or an HDFS, HTTP, HTTPS, or FTP URL. */ def this(master: String, appName: String, sparkHome: String, jarFile: String) = this(new SparkContext(master, appName, sparkHome, Seq(jarFile))) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index b98718a5532e0..2315aadbdf17c 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -38,10 +38,10 @@ The first thing a Spark program must do is to create a `SparkContext` object, wh This is done through the following constructor: {% highlight scala %} -new SparkContext(master, jobName, [sparkHome], [jars]) +new SparkContext(master, appName, [sparkHome], [jars]) {% endhighlight %} -The `master` parameter is a string specifying a [Mesos](running-on-mesos.html) cluster to connect to, or a special "local" string to run in local mode, as described below. `jobName` is a name for your job, which will be shown in the Mesos web UI when running on a cluster. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. +The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable. For example, to run on four cores, use diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 7e913b783d89e..42a4a5619de7c 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -17,16 +17,12 @@ This guide shows some how to start programming with DStreams. The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using {% highlight scala %} -new StreamingContext(master, jobName, batchDuration) +new StreamingContext(master, appName, batchDuration, [sparkHome], [jars]) {% endhighlight %} -The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing mode. The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the cluster UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Starting with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. - -This constructor creates a SparkContext object using the given `master` and `jobName` parameters. However, if you already have a SparkContext or you need to create a custom SparkContext by specifying list of JARs, then a StreamingContext can be created from the existing SparkContext, by using -{% highlight scala %} -new StreamingContext(sparkContext, batchDuration) -{% endhighlight %} +The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). +This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`. # Attaching Input Sources - InputDStreams diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index e7a392fbbf346..e303e33e5e401 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -17,6 +17,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val framework = ssc.sc.appName val sparkHome = ssc.sc.sparkHome val jars = ssc.sc.jars + val environment = ssc.sc.environment val graph = ssc.graph val checkpointDir = ssc.checkpointDir val checkpointDuration = ssc.checkpointDuration diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 25c67b279b7d2..31b5d2c8bcff1 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -6,7 +6,7 @@ import akka.zeromq.Subscribe import spark.streaming.dstream._ -import spark.{RDD, Logging, SparkEnv, SparkContext} +import spark._ import spark.streaming.receivers.ActorReceiver import spark.streaming.receivers.ReceiverSupervisorStrategy import spark.streaming.receivers.ZeroMQReceiver @@ -14,18 +14,18 @@ import spark.storage.StorageLevel import spark.util.MetadataCleaner import spark.streaming.receivers.ActorReceiver - import scala.collection.mutable.Queue +import scala.collection.Map import java.io.InputStream import java.util.concurrent.atomic.AtomicInteger +import java.util.UUID import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path -import java.util.UUID import twitter4j.Status /** @@ -44,7 +44,9 @@ class StreamingContext private ( * @param sparkContext Existing SparkContext * @param batchDuration The time interval at which streaming data will be divided into batches */ - def this(sparkContext: SparkContext, batchDuration: Duration) = this(sparkContext, null, batchDuration) + def this(sparkContext: SparkContext, batchDuration: Duration) = { + this(sparkContext, null, batchDuration) + } /** * Create a StreamingContext by providing the details necessary for creating a new SparkContext. @@ -52,8 +54,17 @@ class StreamingContext private ( * @param appName A name for your job, to display on the cluster web UI * @param batchDuration The time interval at which streaming data will be divided into batches */ - def this(master: String, appName: String, batchDuration: Duration) = - this(StreamingContext.createNewSparkContext(master, appName), null, batchDuration) + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String = null, + jars: Seq[String] = Nil, + environment: Map[String, String] = Map()) = { + this(StreamingContext.createNewSparkContext(master, appName, sparkHome, jars, environment), + null, batchDuration) + } + /** * Re-create a StreamingContext from a checkpoint file. @@ -65,15 +76,20 @@ class StreamingContext private ( initLogging() if (sc_ == null && cp_ == null) { - throw new Exception("Streaming Context cannot be initilalized with " + + throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") } + if (MetadataCleaner.getDelaySeconds < 0) { + throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; " + + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)") + } + protected[streaming] val isCheckpointPresent = (cp_ != null) protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars) + new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment) } else { sc_ } @@ -478,8 +494,12 @@ object StreamingContext { new PairDStreamFunctions[K, V](stream) } - protected[streaming] def createNewSparkContext(master: String, appName: String): SparkContext = { - + protected[streaming] def createNewSparkContext( + master: String, + appName: String, + sparkHome: String, + jars: Seq[String], + environment: Map[String, String]): SparkContext = { // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second interval. if (MetadataCleaner.getDelaySeconds < 0) { diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index f3b40b5b88ef9..b528ebbc1992a 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -45,6 +45,42 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Creates a StreamingContext. + * @param master Name of the Spark Master + * @param appName Name to be used when registering with the scheduler + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + */ + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String, + jars: Array[String]) = + this(new StreamingContext(master, appName, batchDuration, sparkHome, jars)) + + /** + * Creates a StreamingContext. + * @param master Name of the Spark Master + * @param appName Name to be used when registering with the scheduler + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param environment Environment variables to set on worker nodes + */ + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String, + jars: Array[String], + environment: JMap[String, String]) = + this(new StreamingContext(master, appName, batchDuration, sparkHome, jars, environment)) + + /** + * Creates a StreamingContext using an existing SparkContext. * @param sparkContext The underlying JavaSparkContext to use * @param batchDuration The time interval at which streaming data will be divided into batches */ diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index dc7139cc273cf..ddd9becf325e0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -13,6 +13,7 @@ import kafka.serializer.StringDecoder import kafka.utils.{Utils, ZKGroupTopicDirs} import kafka.utils.ZkUtils._ +import scala.collection.Map import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ From 351ac5233e20073e3161a5817992ac61317ce2f6 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 15:19:05 -0800 Subject: [PATCH 312/696] Some tweaks to docs --- docs/java-programming-guide.md | 2 +- docs/python-programming-guide.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 37a906ea1c780..ae8257b53938e 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -189,7 +189,7 @@ We hope to generate documentation with Java-style syntax in the future. # Where to Go from Here Spark includes several sample programs using the Java API in -`examples/src/main/java`. You can run them by passing the class name to the +[`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/examples). You can run them by passing the class name to the `run` script included in Spark -- for example, `./run spark.examples.JavaWordCount`. Each example program prints usage help when run without any arguments. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 2012241a6a77b..3a7a8db4a6ee4 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -109,9 +109,9 @@ Code dependencies can be added to an existing SparkContext using its `addPyFile( # Where to Go from Here -PySpark includes several sample programs using the Python API in `python/examples`. +PySpark includes several sample programs in the [`python/examples` folder](https://github.com/mesos/spark/tree/master/python/examples). You can run them by passing the files to the `pyspark` script -- for example `./pyspark python/examples/wordcount.py`. -Each example program prints usage help when run without any arguments. +Each program prints usage help when run without arguments. We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc. Many of the RDD method descriptions contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. From 918ee258673b2364730fe38fe781cb5167870e7a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Feb 2013 15:24:17 -0800 Subject: [PATCH 313/696] One more change done with TD --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 4b351d07e51f7..e1407a49b070b 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -288,7 +288,7 @@ Note that, unlike RDDs, the default persistence level of DStreams keeps the data # RDD Checkpointing within DStreams A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. -Because stateful operations have an infinitely growing lineage, their state must be periodically checkpointed to HDFS. Checkpointing truncates the lineage graph and stores the of the DStream in HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. +Because stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. To clear this meta-data, streaming supports periodic _checkpointing_ by saving intermediate data to HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using From 8316534eefe7e6b2d109ead53cf5f87c8cd50388 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Feb 2013 15:27:04 -0800 Subject: [PATCH 314/696] meta-data --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index e1407a49b070b..de2fa569d0482 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -288,7 +288,7 @@ Note that, unlike RDDs, the default persistence level of DStreams keeps the data # RDD Checkpointing within DStreams A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. -Because stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. To clear this meta-data, streaming supports periodic _checkpointing_ by saving intermediate data to HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. +Because stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. To clear this metadata, streaming supports periodic _checkpointing_ by saving intermediate data to HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try. To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using From 6494cab19dda6cb44af506bd86cc75990ce87f70 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 15:38:21 -0800 Subject: [PATCH 315/696] Update Hadoop dependency to 1.0.4 --- pom.xml | 2 +- project/SparkBuild.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 99eb17856a813..ce8fa7734a18f 100644 --- a/pom.xml +++ b/pom.xml @@ -514,7 +514,7 @@ org.apache.hadoop hadoop-core - 1.0.3 + 1.0.4 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 25c232837358d..b19bfd5f9a73a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -9,8 +9,8 @@ import twirl.sbt.TwirlPlugin._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or - // "1.0.3" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - val HADOOP_VERSION = "1.0.3" + // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. + val HADOOP_VERSION = "1.0.4" val HADOOP_MAJOR_VERSION = "1" // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" From 4d480ec59e8cf268054ed805abcd1e84eca17b41 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 15:53:43 -0800 Subject: [PATCH 316/696] Fixed something that was reported as a compile error in ScalaDoc. For some reason, ScalaDoc complained about no such constructor for StreamingContext; it doesn't seem like an actual Scala error but it prevented sbt publish and from working because docs weren't built. --- .../scala/spark/streaming/api/java/JavaStreamingContext.scala | 4 ++-- .../main/scala/spark/streaming/util/MasterFailureTest.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index b528ebbc1992a..755407aecc53a 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -41,7 +41,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param batchDuration The time interval at which streaming data will be divided into batches */ def this(master: String, appName: String, batchDuration: Duration) = - this(new StreamingContext(master, appName, batchDuration)) + this(new StreamingContext(master, appName, batchDuration, null, Nil, Map())) /** * Creates a StreamingContext. @@ -58,7 +58,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { batchDuration: Duration, sparkHome: String, jars: Array[String]) = - this(new StreamingContext(master, appName, batchDuration, sparkHome, jars)) + this(new StreamingContext(master, appName, batchDuration, sparkHome, jars, Map())) /** * Creates a StreamingContext. diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index bdd9f4d7535ea..f673e5be15485 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -159,7 +159,7 @@ object MasterFailureTest extends Logging { // Setup the streaming computation with the given operation System.clearProperty("spark.driver.port") - var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration) + var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map()) ssc.checkpoint(checkpointDir.toString) val inputStream = ssc.textFileStream(testDir.toString) val operatedStream = operation(inputStream) From c02e064938059133570547b686d655f2a543964e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 25 Feb 2013 17:27:46 -0800 Subject: [PATCH 317/696] Fixed replication bug in BlockManager --- .../main/scala/spark/storage/MemoryStore.scala | 4 ++-- core/src/test/scala/spark/DistributedSuite.scala | 16 +++++++++++++++- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index ae88ff0bb1694..949588476c201 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -32,8 +32,8 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + bytes.rewind() if (level.deserialized) { - bytes.rewind() val values = blockManager.dataDeserialize(blockId, bytes) val elements = new ArrayBuffer[Any] elements ++= values @@ -58,7 +58,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { val bytes = blockManager.dataSerialize(blockId, values.iterator) tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes)) + PutResult(bytes.limit(), Right(bytes.duplicate())) } } diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index caa4ba3a3705a..4104b33c8b681 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -1,5 +1,6 @@ package spark +import network.ConnectionManagerId import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers @@ -13,7 +14,7 @@ import com.google.common.io.Files import scala.collection.mutable.ArrayBuffer import SparkContext._ -import storage.StorageLevel +import storage.{GetBlock, BlockManagerWorker, StorageLevel} class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { @@ -140,9 +141,22 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter test("caching in memory and disk, serialized, replicated") { sc = new SparkContext(clusterUrl, "test") val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2) + assert(data.count() === 1000) assert(data.count() === 1000) assert(data.count() === 1000) + + // Get all the locations of the first partition and try to fetch the partitions + // from those locations. + val blockIds = data.partitions.indices.map(index => "rdd_%d_%d".format(data.id, index)).toArray + val blockId = blockIds(0) + val blockManager = SparkEnv.get.blockManager + blockManager.master.getLocations(blockId).foreach(id => { + val bytes = BlockManagerWorker.syncGetBlock( + GetBlock(blockId), ConnectionManagerId(id.ip, id.port)) + val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList + assert(deserialized === (1 to 100).toList) + }) } test("compute without caching when no partitions fit in memory") { From bc4a6eb850c84793aa65c0dd2e27f379876b1969 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 25 Feb 2013 18:04:21 -0800 Subject: [PATCH 318/696] Changed Flume test to use the same port as other tests, so that can be controlled centrally. --- .../src/test/scala/spark/streaming/InputStreamsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 1024d3ac9790e..ebcb6d0092971 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -94,7 +94,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("flume input stream") { // Set up the streaming context and input streams val ssc = new StreamingContext(master, framework, batchDuration) - val flumeStream = ssc.flumeStream("localhost", 33333, StorageLevel.MEMORY_AND_DISK) + val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) @@ -104,7 +104,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val input = Seq(1, 2, 3, 4, 5) Thread.sleep(1000) - val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", 33333)); + val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)); val client = SpecificRequestor.getClient( classOf[AvroSourceProtocol], transceiver); From 5d7b591cfe14177f083814fe3e81745c5d279810 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 19:34:32 -0800 Subject: [PATCH 319/696] Pass a code JAR to SparkContext in our examples. Fixes SPARK-594. --- .../main/java/spark/examples/JavaHdfsLR.java | 6 +- .../src/main/java/spark/examples/JavaTC.java | 5 +- .../java/spark/examples/JavaWordCount.java | 5 +- .../examples/JavaFlumeEventCount.java | 3 +- .../examples/JavaNetworkWordCount.java | 4 +- .../streaming/examples/JavaQueueStream.java | 3 +- .../scala/spark/examples/BroadcastTest.scala | 10 +-- .../examples/ExceptionHandlingTest.scala | 5 +- .../scala/spark/examples/GroupByTest.scala | 5 +- .../main/scala/spark/examples/HdfsTest.scala | 3 +- .../main/scala/spark/examples/LocalALS.scala | 4 +- .../scala/spark/examples/LocalKMeans.scala | 3 + .../main/scala/spark/examples/LocalLR.scala | 3 + .../main/scala/spark/examples/LogQuery.scala | 4 +- .../spark/examples/MultiBroadcastTest.scala | 16 +++-- .../examples/SimpleSkewedGroupByTest.scala | 3 +- .../spark/examples/SkewedGroupByTest.scala | 11 ++-- .../main/scala/spark/examples/SparkALS.scala | 62 ++++++++++--------- .../scala/spark/examples/SparkHdfsLR.scala | 6 +- .../scala/spark/examples/SparkKMeans.scala | 6 +- .../main/scala/spark/examples/SparkLR.scala | 6 +- .../main/scala/spark/examples/SparkPi.scala | 3 +- .../main/scala/spark/examples/SparkTC.scala | 4 +- .../streaming/examples/ActorWordCount.scala | 3 +- .../streaming/examples/FlumeEventCount.scala | 3 +- .../streaming/examples/HdfsWordCount.scala | 3 +- .../streaming/examples/KafkaWordCount.scala | 4 +- .../streaming/examples/NetworkWordCount.scala | 3 +- .../streaming/examples/QueueStream.scala | 3 +- .../streaming/examples/RawNetworkGrep.scala | 3 +- .../examples/TwitterAlgebirdCMS.scala | 3 +- .../examples/TwitterAlgebirdHLL.scala | 3 +- .../examples/TwitterPopularTags.scala | 3 +- .../streaming/examples/ZeroMQWordCount.scala | 5 +- .../examples/clickstream/PageViewStream.scala | 3 +- run | 10 +++ run2.cmd | 10 +++ .../api/java/JavaStreamingContext.scala | 17 +++++ 38 files changed, 174 insertions(+), 82 deletions(-) diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java index 29839d5668cb7..8b0a9b6808e8a 100644 --- a/examples/src/main/java/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java @@ -10,6 +10,9 @@ import java.util.StringTokenizer; import java.util.Random; +/** + * Logistic regression based classification. + */ public class JavaHdfsLR { static int D = 10; // Number of dimensions @@ -85,7 +88,8 @@ public static void main(String[] args) { System.exit(1); } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR"); + JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[2]); diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java index e3bd881b8f99f..b319bdab44e77 100644 --- a/examples/src/main/java/spark/examples/JavaTC.java +++ b/examples/src/main/java/spark/examples/JavaTC.java @@ -28,7 +28,7 @@ static List> generateGraph() { Tuple2 e = new Tuple2(from, to); if (from != to) edges.add(e); } - return new ArrayList(edges); + return new ArrayList>(edges); } static class ProjectFn extends PairFunction>, @@ -46,7 +46,8 @@ public static void main(String[] args) { System.exit(1); } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC"); + JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2; JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/spark/examples/JavaWordCount.java index a44cf8a1206b8..9d4c7a252df1c 100644 --- a/examples/src/main/java/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/spark/examples/JavaWordCount.java @@ -18,7 +18,8 @@ public static void main(String[] args) throws Exception { System.exit(1); } - JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount"); + JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); JavaRDD lines = ctx.textFile(args[1], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { @@ -29,7 +30,7 @@ public Iterable call(String s) { JavaPairRDD ones = words.map(new PairFunction() { public Tuple2 call(String s) { - return new Tuple2(s, 1); + return new Tuple2(s, 1); } }); diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java index cddce16e396cd..e24c6ddaa7929 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java @@ -32,7 +32,8 @@ public static void main(String[] args) { Duration batchInterval = new Duration(2000); - JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval); + JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); JavaDStream flumeStream = sc.flumeStream("localhost", port); diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java index 0e9eadd01b711..3e57580fd4e6c 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java @@ -30,8 +30,8 @@ public static void main(String[] args) { } // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext( - args[0], "NetworkWordCount", new Duration(1000)); + JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount", + new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java index 43c3cd4dfac4b..15b82c8da1594 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java @@ -22,7 +22,8 @@ public static void main(String[] args) throws InterruptedException { } // Create the context - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000)); + JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000), + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala index 230097c7db2a7..ba59be1687235 100644 --- a/examples/src/main/scala/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala @@ -9,19 +9,21 @@ object BroadcastTest { System.exit(1) } - val spark = new SparkContext(args(0), "Broadcast Test") + val sc = new SparkContext(args(0), "Broadcast Test", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 var arr1 = new Array[Int](num) - for (i <- 0 until arr1.length) + for (i <- 0 until arr1.length) { arr1(i) = i + } for (i <- 0 until 2) { println("Iteration " + i) println("===========") - val barr1 = spark.broadcast(arr1) - spark.parallelize(1 to 10, slices).foreach { + val barr1 = sc.broadcast(arr1) + sc.parallelize(1 to 10, slices).foreach { i => println(barr1.value.size) } } diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala index c89f3dac0c5a3..21a90f2e5ad5b 100644 --- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala @@ -9,9 +9,10 @@ object ExceptionHandlingTest { System.exit(1) } - val sc = new SparkContext(args(0), "ExceptionHandlingTest") + val sc = new SparkContext(args(0), "ExceptionHandlingTest", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) sc.parallelize(0 until sc.defaultParallelism).foreach { i => - if (Math.random > 0.75) + if (math.random > 0.75) throw new Exception("Testing exception handling") } diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala index 86dfba3a40498..a6603653f1503 100644 --- a/examples/src/main/scala/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/spark/examples/GroupByTest.scala @@ -9,14 +9,15 @@ object GroupByTest { if (args.length == 0) { System.err.println("Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) - } + } var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000 var numReducers = if (args.length > 4) args(4).toInt else numMappers - val sc = new SparkContext(args(0), "GroupBy Test") + val sc = new SparkContext(args(0), "GroupBy Test", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/spark/examples/HdfsTest.scala b/examples/src/main/scala/spark/examples/HdfsTest.scala index 7a4530609d737..dd61c467f7fc7 100644 --- a/examples/src/main/scala/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/spark/examples/HdfsTest.scala @@ -4,7 +4,8 @@ import spark._ object HdfsTest { def main(args: Array[String]) { - val sc = new SparkContext(args(0), "HdfsTest") + val sc = new SparkContext(args(0), "HdfsTest", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/spark/examples/LocalALS.scala index 10e03359c9596..2de810e062b78 100644 --- a/examples/src/main/scala/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/spark/examples/LocalALS.scala @@ -1,11 +1,13 @@ package spark.examples -import java.util.Random import scala.math.sqrt import cern.jet.math._ import cern.colt.matrix._ import cern.colt.matrix.linalg._ +/** + * Alternating least squares matrix factorization. + */ object LocalALS { // Parameters set through command line arguments var M = 0 // Number of movies diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala index b442c604cdd2c..b07e799cef5af 100644 --- a/examples/src/main/scala/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala @@ -6,6 +6,9 @@ import spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +/** + * K-means clustering. + */ object LocalKMeans { val N = 1000 val R = 1000 // Scaling factor diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala index 9553162004985..cd73f553d6e23 100644 --- a/examples/src/main/scala/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/spark/examples/LocalLR.scala @@ -3,6 +3,9 @@ package spark.examples import java.util.Random import spark.util.Vector +/** + * Logistic regression based classification. + */ object LocalLR { val N = 10000 // Number of data points val D = 10 // Number of dimensions diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/spark/examples/LogQuery.scala index 5330b8da9444f..6497596d35f34 100644 --- a/examples/src/main/scala/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/spark/examples/LogQuery.scala @@ -26,7 +26,9 @@ object LogQuery { System.err.println("Usage: LogQuery [logFile]") System.exit(1) } - val sc = new SparkContext(args(0), "Log Query") + + val sc = new SparkContext(args(0), "Log Query", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val dataSet = if (args.length == 2) sc.textFile(args(1)) diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala index 83ae014e94580..92cd81c48742f 100644 --- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala @@ -9,21 +9,25 @@ object MultiBroadcastTest { System.exit(1) } - val spark = new SparkContext(args(0), "Broadcast Test") + val sc = new SparkContext(args(0), "Broadcast Test", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 var arr1 = new Array[Int](num) - for (i <- 0 until arr1.length) + for (i <- 0 until arr1.length) { arr1(i) = i + } var arr2 = new Array[Int](num) - for (i <- 0 until arr2.length) + for (i <- 0 until arr2.length) { arr2(i) = i + } - val barr1 = spark.broadcast(arr1) - val barr2 = spark.broadcast(arr2) - spark.parallelize(1 to 10, slices).foreach { + val barr1 = sc.broadcast(arr1) + val barr2 = sc.broadcast(arr2) + sc.parallelize(1 to 10, slices).foreach { i => println(barr1.value.size + barr2.value.size) } diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala index 50b3a263b4feb..0d17bda004b6e 100644 --- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala @@ -18,7 +18,8 @@ object SimpleSkewedGroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers var ratio = if (args.length > 5) args(5).toInt else 5.0 - val sc = new SparkContext(args(0), "GroupBy Test") + val sc = new SparkContext(args(0), "GroupBy Test", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala index d2117a263e6d8..83be3fc27b5b0 100644 --- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala @@ -16,13 +16,14 @@ object SkewedGroupByTest { var valSize = if (args.length > 3) args(3).toInt else 1000 var numReducers = if (args.length > 4) args(4).toInt else numMappers - val sc = new SparkContext(args(0), "GroupBy Test") + val sc = new SparkContext(args(0), "GroupBy Test", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random // map output sizes lineraly increase from the 1st to the last - numKVPairs = (1. * (p + 1) / numMappers * numKVPairs).toInt + numKVPairs = (1.0 * (p + 1) / numMappers * numKVPairs).toInt var arr1 = new Array[(Int, Array[Byte])](numKVPairs) for (i <- 0 until numKVPairs) { @@ -31,11 +32,11 @@ object SkewedGroupByTest { arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr) } arr1 - }.cache + }.cache() // Enforce that everything has been calculated and in cache - pairs1.count + pairs1.count() - println(pairs1.groupByKey(numReducers).count) + println(pairs1.groupByKey(numReducers).count()) System.exit(0) } diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala index 5e01885dbb464..8fb3b0fb2ad78 100644 --- a/examples/src/main/scala/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/spark/examples/SparkALS.scala @@ -1,14 +1,14 @@ package spark.examples -import java.io.Serializable -import java.util.Random import scala.math.sqrt import cern.jet.math._ import cern.colt.matrix._ import cern.colt.matrix.linalg._ import spark._ -import scala.Option +/** + * Alternating least squares matrix factorization. + */ object SparkALS { // Parameters set through command line arguments var M = 0 // Number of movies @@ -70,30 +70,32 @@ object SparkALS { } def main(args: Array[String]) { + if (args.length == 0) { + System.err.println("Usage: SparkALS [ ]") + System.exit(1) + } + var host = "" var slices = 0 - (0 to 5).map(i => { - i match { - case a if a < args.length => Some(args(a)) - case _ => None - } - }).toArray match { - case Array(host_, m, u, f, iters, slices_) => { - host = host_ getOrElse "local" - M = (m getOrElse "100").toInt - U = (u getOrElse "500").toInt - F = (f getOrElse "10").toInt - ITERATIONS = (iters getOrElse "5").toInt - slices = (slices_ getOrElse "2").toInt - } - case _ => { - System.err.println("Usage: SparkALS [ ]") + val options = (0 to 5).map(i => if (i < args.length) Some(args(i)) else None) + + options.toArray match { + case Array(host_, m, u, f, iters, slices_) => + host = host_.get + M = m.getOrElse("100").toInt + U = u.getOrElse("500").toInt + F = f.getOrElse("10").toInt + ITERATIONS = iters.getOrElse("5").toInt + slices = slices_.getOrElse("2").toInt + case _ => + System.err.println("Usage: SparkALS [ ]") System.exit(1) - } } printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) - val spark = new SparkContext(host, "SparkALS") + + val sc = new SparkContext(host, "SparkALS", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val R = generateR() @@ -102,19 +104,19 @@ object SparkALS { var us = Array.fill(U)(factory1D.random(F)) // Iteratively update movies then users - val Rc = spark.broadcast(R) - var msc = spark.broadcast(ms) - var usc = spark.broadcast(us) + val Rc = sc.broadcast(R) + var msb = sc.broadcast(ms) + var usb = sc.broadcast(us) for (iter <- 1 to ITERATIONS) { println("Iteration " + iter + ":") - ms = spark.parallelize(0 until M, slices) - .map(i => update(i, msc.value(i), usc.value, Rc.value)) + ms = sc.parallelize(0 until M, slices) + .map(i => update(i, msb.value(i), usb.value, Rc.value)) .toArray - msc = spark.broadcast(ms) // Re-broadcast ms because it was updated - us = spark.parallelize(0 until U, slices) - .map(i => update(i, usc.value(i), msc.value, algebra.transpose(Rc.value))) + msb = sc.broadcast(ms) // Re-broadcast ms because it was updated + us = sc.parallelize(0 until U, slices) + .map(i => update(i, usb.value(i), msb.value, algebra.transpose(Rc.value))) .toArray - usc = spark.broadcast(us) // Re-broadcast us because it was updated + usb = sc.broadcast(us) // Re-broadcast us because it was updated println("RMSE = " + rmse(R, ms, us)) println() } diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala index 5b2bc84d6900f..0f42f405a058c 100644 --- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala @@ -5,6 +5,9 @@ import scala.math.exp import spark.util.Vector import spark._ +/** + * Logistic regression based classification. + */ object SparkHdfsLR { val D = 10 // Numer of dimensions val rand = new Random(42) @@ -29,7 +32,8 @@ object SparkHdfsLR { System.err.println("Usage: SparkHdfsLR ") System.exit(1) } - val sc = new SparkContext(args(0), "SparkHdfsLR") + val sc = new SparkContext(args(0), "SparkHdfsLR", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val lines = sc.textFile(args(1)) val points = lines.map(parsePoint _).cache() val ITERATIONS = args(2).toInt diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/spark/examples/SparkKMeans.scala index 63759613906b4..7c21ea12fb724 100644 --- a/examples/src/main/scala/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala @@ -7,6 +7,9 @@ import spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +/** + * K-means clustering. + */ object SparkKMeans { val R = 1000 // Scaling factor val rand = new Random(42) @@ -36,7 +39,8 @@ object SparkKMeans { System.err.println("Usage: SparkLocalKMeans ") System.exit(1) } - val sc = new SparkContext(args(0), "SparkLocalKMeans") + val sc = new SparkContext(args(0), "SparkLocalKMeans", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala index aaaf062c8f9ea..2f41aeb376fda 100644 --- a/examples/src/main/scala/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/spark/examples/SparkLR.scala @@ -5,6 +5,9 @@ import scala.math.exp import spark.util.Vector import spark._ +/** + * Logistic regression based classification. + */ object SparkLR { val N = 10000 // Number of data points val D = 10 // Numer of dimensions @@ -28,7 +31,8 @@ object SparkLR { System.err.println("Usage: SparkLR []") System.exit(1) } - val sc = new SparkContext(args(0), "SparkLR") + val sc = new SparkContext(args(0), "SparkLR", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val numSlices = if (args.length > 1) args(1).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala index 2f226f13805c0..5a31d74444f1c 100644 --- a/examples/src/main/scala/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/spark/examples/SparkPi.scala @@ -10,7 +10,8 @@ object SparkPi { System.err.println("Usage: SparkPi []") System.exit(1) } - val spark = new SparkContext(args(0), "SparkPi") + val spark = new SparkContext(args(0), "SparkPi", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/spark/examples/SparkTC.scala index 90bae011adfb6..911ae8f168d03 100644 --- a/examples/src/main/scala/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/spark/examples/SparkTC.scala @@ -9,7 +9,6 @@ import scala.collection.mutable * Transitive closure on a graph. */ object SparkTC { - val numEdges = 200 val numVertices = 100 val rand = new Random(42) @@ -29,7 +28,8 @@ object SparkTC { System.err.println("Usage: SparkTC []") System.exit(1) } - val spark = new SparkContext(args(0), "SparkTC") + val spark = new SparkContext(args(0), "SparkTC", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val slices = if (args.length > 1) args(1).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 76293fbb96bf7..3b847fe603409 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -131,7 +131,8 @@ object ActorWordCount { val Seq(master, host, port) = args.toSeq // Create the context and set the batch size - val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2)) + val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala index 461929fba22ef..39c76fd98a11e 100644 --- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala @@ -30,7 +30,8 @@ object FlumeEventCount { val batchInterval = Milliseconds(2000) // Create the context and set the batch size - val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval) + val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) // Create a flume stream val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY) diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala index 8530f5c175493..9389f8a38dcca 100644 --- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala @@ -22,7 +22,8 @@ object HdfsWordCount { } // Create the context - val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2)) + val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) // Create the FileInputDStream on the directory and use the // stream to count words in new files created diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 9b135a5c54cf3..c3a9e491ba854 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -32,8 +32,8 @@ object KafkaWordCount { val Array(master, zkQuorum, group, topics, numThreads) = args - val sc = new SparkContext(master, "KafkaWordCount") - val ssc = new StreamingContext(sc, Seconds(2)) + val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 5ac6d19b349da..704540c2bf51b 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -23,7 +23,8 @@ object NetworkWordCount { } // Create the context with a 1 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1)) + val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala index e9cb7b55ea132..f450e2104018b 100644 --- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala @@ -15,7 +15,8 @@ object QueueStream { } // Create the context - val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1)) + val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala index 49b3223eecd19..175281e0956d4 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala @@ -31,7 +31,8 @@ object RawNetworkGrep { val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args // Create the context - val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis)) + val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) // Warm up the JVMs on master and slave for JIT compilation to kick in RawTextHelper.warmUp(ssc.sparkContext) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 39a1a702eeae9..483aae452b05e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -43,7 +43,8 @@ object TwitterAlgebirdCMS { val Array(master, username, password) = args.slice(0, 3) val filters = args.slice(3, args.length) - val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10)) + val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 914fba4ca22c5..f3288bfb8547d 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -32,7 +32,8 @@ object TwitterAlgebirdHLL { val Array(master, username, password) = args.slice(0, 3) val filters = args.slice(3, args.length) - val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5)) + val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index fdb3a4c73c6cc..9d4494c6f2759 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -21,7 +21,8 @@ object TwitterPopularTags { val Array(master, username, password) = args.slice(0, 3) val filters = args.slice(3, args.length) - val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2)) + val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val stream = ssc.twitterStream(username, password, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala index 5ed9b7cb76875..74d0d338b754e 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -58,7 +58,8 @@ object ZeroMQWordCount { val Seq(master, url, topic) = args.toSeq // Create the context and set the batch size - val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2)) + val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator @@ -70,4 +71,4 @@ object ZeroMQWordCount { ssc.start() } -} \ No newline at end of file +} diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index 9a2ba30ee4baf..e226a4a73a16b 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -24,7 +24,8 @@ object PageViewStream { val port = args(2).toInt // Create the context - val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1)) + val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) diff --git a/run b/run index fd06fbe7c7e8b..2c780623c8b67 100755 --- a/run +++ b/run @@ -134,6 +134,16 @@ for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do done export CLASSPATH # Needed for spark-shell +# Figure out the JAR file that our examples were packaged into. +if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*".jar" ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*".jar"` +fi +if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop*".jar" ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop*".jar"` +fi + # Figure out whether to run our class with java or with the scala launcher. # In most cases, we'd prefer to execute our process with java because scala # creates a shell script as the parent of its Java process, which makes it diff --git a/run2.cmd b/run2.cmd index 705a4d1ff6830..f34869f1b18e0 100644 --- a/run2.cmd +++ b/run2.cmd @@ -62,6 +62,16 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +rem Figure out the JAR file that our examples were packaged into. +rem First search in the build path from SBT: +for /D %%d in ("%EXAMPLES_DIR%/target/scala-%SCALA_VERSION%/spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d +) +rem Then search in the build path from Maven: +for /D %%d in ("%EXAMPLES_DIR%/target/spark-examples*hadoop*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d +) + rem Figure out whether to run our class with java or with the scala launcher. rem In most cases, we'd prefer to execute our process with java because scala rem creates a shell script as the parent of its Java process, which makes it diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 755407aecc53a..3d149a742cb3c 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -43,6 +43,23 @@ class JavaStreamingContext(val ssc: StreamingContext) { def this(master: String, appName: String, batchDuration: Duration) = this(new StreamingContext(master, appName, batchDuration, null, Nil, Map())) + /** + * Creates a StreamingContext. + * @param master Name of the Spark Master + * @param appName Name to be used when registering with the scheduler + * @param batchDuration The time interval at which streaming data will be divided into batches + * @param sparkHome The SPARK_HOME directory on the slave nodes + * @param jarFile JAR file containing job code, to ship to cluster. This can be a path on the local + * file system or an HDFS, HTTP, HTTPS, or FTP URL. + */ + def this( + master: String, + appName: String, + batchDuration: Duration, + sparkHome: String, + jarFile: String) = + this(new StreamingContext(master, appName, batchDuration, sparkHome, Seq(jarFile), Map())) + /** * Creates a StreamingContext. * @param master Name of the Spark Master From 284ba90958df2d6efc08e3f8381bb9ef09f8b322 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Feb 2013 19:40:52 -0800 Subject: [PATCH 320/696] createNewSparkContext should use sparkHome/jars/environment. This fixes a bug introduced by Matei's recent change. --- streaming/src/main/scala/spark/streaming/StreamingContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 31b5d2c8bcff1..b8b60aab43303 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -505,7 +505,7 @@ object StreamingContext { if (MetadataCleaner.getDelaySeconds < 0) { MetadataCleaner.setDelaySeconds(3600) } - new SparkContext(master, appName) + new SparkContext(master, appName, sparkHome, jars, environment) } protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = { From 108cfaa5b4905f390c56cd5187c6aa1d6af63300 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 20:23:36 -0800 Subject: [PATCH 321/696] Fix Windows script for finding examples JAR --- run2.cmd | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/run2.cmd b/run2.cmd index f34869f1b18e0..cb20a4b7a2f9a 100644 --- a/run2.cmd +++ b/run2.cmd @@ -64,12 +64,12 @@ set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes rem Figure out the JAR file that our examples were packaged into. rem First search in the build path from SBT: -for /D %%d in ("%EXAMPLES_DIR%/target/scala-%SCALA_VERSION%/spark-examples*.jar") do ( - set SPARK_EXAMPLES_JAR=%%d +for %%d in ("examples/target/scala-%SCALA_VERSION%/spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=examples/target/scala-%SCALA_VERSION%/%%d ) rem Then search in the build path from Maven: -for /D %%d in ("%EXAMPLES_DIR%/target/spark-examples*hadoop*.jar") do ( - set SPARK_EXAMPLES_JAR=%%d +for %%d in ("examples/target/spark-examples*hadoop*.jar") do ( + set SPARK_EXAMPLES_JAR=examples/target/%%d ) rem Figure out whether to run our class with java or with the scala launcher. From 7e67c626eee0d8448746ddcec5f01f4ffe8bdcb2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 20:30:47 -0800 Subject: [PATCH 322/696] Change version number to 0.7.0 --- docs/_config.yml | 2 +- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index 09617e4a1efb6..391aa29047c36 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -3,7 +3,7 @@ markdown: kramdown # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 0.7.0-SNAPSHOT +SPARK_VERSION: 0.7.0 SPARK_VERSION_SHORT: 0.7.0 SCALA_VERSION: 2.9.2 MESOS_VERSION: 0.9.0-incubating diff --git a/pom.xml b/pom.xml index ce8fa7734a18f..a1359dac83acb 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project parent - 0.7.0-SNAPSHOT + 0.7.0 pom Spark Project Parent POM http://spark-project.org/ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b19bfd5f9a73a..5edac112b31ed 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -35,7 +35,7 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", - version := "0.7.0-SNAPSHOT", + version := "0.7.0", scalaVersion := "2.9.2", scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, From a65aa549ff9fdf205dd2c1cc04a832f4972c6c3c Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Mon, 25 Feb 2013 23:49:32 -0600 Subject: [PATCH 323/696] Override DAGScheduler.runLocally so we can remove the Thread.sleep. --- .../scala/spark/scheduler/DAGScheduler.scala | 37 +++++++++++-------- .../spark/scheduler/DAGSchedulerSuite.scala | 9 +++-- 2 files changed, 27 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bf0837c0660cb..9e7791fbb46b0 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -379,29 +379,34 @@ class DAGScheduler( * We run the operation in a separate thread just in case it takes a bunch of time, so that we * don't block the DAGScheduler event loop or other concurrent jobs. */ - private def runLocally(job: ActiveJob) { + protected def runLocally(job: ActiveJob) { logInfo("Computing the requested partition locally") new Thread("Local computation of job " + job.runId) { override def run() { - try { - SparkEnv.set(env) - val rdd = job.finalStage.rdd - val split = rdd.partitions(job.partitions(0)) - val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) - try { - val result = job.func(taskContext, rdd.iterator(split, taskContext)) - job.listener.taskSucceeded(0, result) - } finally { - taskContext.executeOnCompleteCallbacks() - } - } catch { - case e: Exception => - job.listener.jobFailed(e) - } + runLocallyWithinThread(job) } }.start() } + // Broken out for easier testing in DAGSchedulerSuite. + protected def runLocallyWithinThread(job: ActiveJob) { + try { + SparkEnv.set(env) + val rdd = job.finalStage.rdd + val split = rdd.partitions(job.partitions(0)) + val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) + try { + val result = job.func(taskContext, rdd.iterator(split, taskContext)) + job.listener.taskSucceeded(0, result) + } finally { + taskContext.executeOnCompleteCallbacks() + } + } catch { + case e: Exception => + job.listener.jobFailed(e) + } + } + /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 07cccc7ce0874..29b565ecad4af 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -90,7 +90,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter { cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTracker() - scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) + scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) { + override def runLocally(job: ActiveJob) { + // don't bother with the thread while unit testing + runLocallyWithinThread(job) + } + } } after { @@ -203,8 +208,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter { override def toString = "DAGSchedulerSuite Local RDD" } runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener)) - // this shouldn't be needed, but i haven't stubbed out runLocally yet - Thread.sleep(500) assert(results === Map(0 -> 42)) } From 9a046e30ac6013fc74f60892f73086e17e14fd3e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Feb 2013 22:18:47 -0800 Subject: [PATCH 324/696] Switch docs to use Akka repo instead of Typesafe --- docs/quick-start.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index a4c4c9a8fb0e3..36d024f13ac18 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -136,7 +136,7 @@ scalaVersion := "{{site.SCALA_VERSION}}" libraryDependencies += "org.spark-project" %% "spark-core" % "{{site.SPARK_VERSION}}" resolvers ++= Seq( - "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", + "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") {% endhighlight %} @@ -207,8 +207,8 @@ To build the job, we also write a Maven `pom.xml` file that lists Spark as a dep http://repo.spray.cc - Typesafe repository - http://repo.typesafe.com/typesafe/releases + Akka repository + http://repo.akka.io/releases From db957e5bd74437b3ecbf20406796e133257ae668 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 26 Feb 2013 01:38:50 -0600 Subject: [PATCH 325/696] Fix MapOutputTrackerSuite. --- core/src/test/scala/spark/MapOutputTrackerSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index b49cd81e235de..3abc584b6a177 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -82,11 +82,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) val masterTracker = new MapOutputTracker() - masterTracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(masterTracker))) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0) val slaveTracker = new MapOutputTracker() - slaveTracker.trackerActor = slaveSystem.actorFor("akka://spark@localhost:" + boundPort) + slaveTracker.trackerActor = slaveSystem.actorFor( + "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") masterTracker.registerShuffle(10, 1) masterTracker.incrementGeneration() From 73697e289174583fe7b0e829eb8387d1a1aa8721 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 12:07:59 -0800 Subject: [PATCH 326/696] Fix overly large thread names in PySpark --- core/src/main/scala/spark/api/python/PythonRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 8c734773847b5..9b4d54ab4e046 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -51,7 +51,7 @@ private[spark] class PythonRDD[T: ClassManifest]( val env = SparkEnv.get // Start a thread to print the process's stderr to ours - new Thread("stderr reader for " + command) { + new Thread("stderr reader for " + pythonExec) { override def run() { for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { System.err.println(line) @@ -60,7 +60,7 @@ private[spark] class PythonRDD[T: ClassManifest]( }.start() // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for " + command) { + new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) From ece3edfffa02f90a71569961b91bf44041f21afe Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 12:11:03 -0800 Subject: [PATCH 327/696] Fix a problem with no hosts being counted as alive in the first job --- .../scala/spark/scheduler/cluster/ClusterScheduler.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index d9c2f9517be50..26fdef101bb27 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -140,6 +140,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Mark each slave as alive and remember its hostname for (o <- offers) { executorIdToHost(o.executorId) = o.hostname + if (!executorsByHost.contains(o.hostname)) { + executorsByHost(o.hostname) = new HashSet() + } } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) @@ -159,9 +162,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) taskSetTaskIds(manager.taskSet.id) += tid taskIdToExecutorId(tid) = execId activeExecutorIds += execId - if (!executorsByHost.contains(host)) { - executorsByHost(host) = new HashSet() - } executorsByHost(host) += execId availableCpus(i) -= 1 launchedTask = true From 434a1ce7739b4f1abe93408edaff9388e480d806 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 12:24:18 -0800 Subject: [PATCH 328/696] Small hack to work around multiple JARs being built by sbt package --- run | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/run b/run index 2c780623c8b67..2c29cc4a6641c 100755 --- a/run +++ b/run @@ -134,14 +134,15 @@ for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do done export CLASSPATH # Needed for spark-shell -# Figure out the JAR file that our examples were packaged into. -if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*".jar" ]; then +# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack +# to avoid the -sources and -doc packages that are built by publish-local. +if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then # Use the JAR from the SBT build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*".jar"` + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop*".jar" ]; then +if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop*".jar"` + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` fi # Figure out whether to run our class with java or with the scala launcher. From 22334eafd96d0cb2b2206c9ad5b458bd8d91eb97 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 22:52:38 -0800 Subject: [PATCH 329/696] Some tweaks to docs --- docs/configuration.md | 7 +++++++ docs/ec2-scripts.md | 17 ++++++++++------- docs/index.md | 8 +------- 3 files changed, 18 insertions(+), 14 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 04eb6daaa5d01..17fdbf04d1ca0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -133,6 +133,13 @@ Apart from these, the following properties are also available, and may be useful it if you configure your own old generation size. + + + + + diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 931b7a66bdcc1..dc57035ebaf6c 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -45,9 +45,9 @@ identify machines belonging to each cluster in the Amazon EC2 Console. key pair, `` is the number of slave nodes to launch (try 1 at first), and `` is the name to give to your cluster. -- After everything launches, check that Mesos is up and sees all the - slaves by going to the Mesos Web UI link printed at the end of the - script (`http://:8080`). +- After everything launches, check that the cluster scheduler is up and sees + all the slaves by going to its web UI, which will be printed at the end of + the script (typically `http://:8080`). You can also run `./spark-ec2 --help` to see more usage options. The following options are worth pointing out: @@ -68,6 +68,9 @@ available. - `--ebs-vol-size=GB` will attach an EBS volume with a given amount of space to each node so that you can have a persistent HDFS cluster on your nodes across cluster restarts (see below). +- `--spot-price=PRICE` will launch the worker nodes as + [Spot Instances](http://aws.amazon.com/ec2/spot-instances/), + bidding for the given maximum price (in dollars). - If one of your launches fails due to e.g. not having the right permissions on your private key file, you can run `launch` with the `--resume` option to restart the setup process on an existing cluster. @@ -80,7 +83,7 @@ permissions on your private key file, you can run `launch` with the above. (This is just for convenience; you could also use the EC2 console.) - To deploy code or data within your cluster, you can log in and use the - provided script `~/mesos-ec2/copy-dir`, which, + provided script `~/spark-ec2/copy-dir`, which, given a directory path, RSYNCs it to the same location on all the slaves. - If your job needs to access large datasets, the fastest way to do that is to load them from Amazon S3 or an Amazon EBS device into an @@ -106,7 +109,7 @@ You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark config as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`). This file needs to be copied to **every machine** to reflect the change. The easiest way to do this is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, -then run `~/mesos-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. +then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. The [configuration guide](configuration.html) describes the available configuration options. @@ -152,10 +155,10 @@ If you have a patch or suggestion for one of these limitations, feel free to # Using a Newer Spark Version -The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/mesos-ec2/copy-dir /root/spark`. +The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/spark-ec2/copy-dir /root/spark`. # Accessing Data in S3 -Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n://:@/path`, where `` is your Amazon access key ID and `` is your Amazon secret access key. Note that you should escape any `/` characters in the secret key as `%2F`. Full instructions can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3). +Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n:///path`. You will also need to set your Amazon security credentials, either by setting the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` before your program or through `SparkContext.hadoopConfiguration`. Full instructions on S3 access using the Hadoop input libraries can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3). In addition to using a single input file, you can also use a directory of files as input by simply giving the path to the directory. diff --git a/docs/index.md b/docs/index.md index fd74a051e0d52..3a775decdb036 100644 --- a/docs/index.md +++ b/docs/index.md @@ -3,15 +3,9 @@ layout: global title: Spark Overview --- -{% comment %} -TODO(andyk): Rewrite to make the Java API a first class part of the story. -{% endcomment %} - Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter. It provides clean, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), with a rich array of parallel operators. -Spark can run on top of the [Apache Mesos](http://incubator.apache.org/mesos/) cluster manager, -[Hadoop YARN](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html), -Amazon EC2, or without an independent resource manager ("standalone mode"). +Spark can run on top of the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode"). # Downloading From fadeb1ddeaae734057a8dd518ecdb9422b47b482 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 23:20:49 -0800 Subject: [PATCH 330/696] More doc tweaks --- docs/index.md | 5 +++-- docs/streaming-programming-guide.md | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/index.md b/docs/index.md index 3a775decdb036..45facd8e63f32 100644 --- a/docs/index.md +++ b/docs/index.md @@ -5,7 +5,7 @@ title: Spark Overview Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter. It provides clean, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), with a rich array of parallel operators. -Spark can run on top of the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode"). +Spark can run on the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode"). # Downloading @@ -86,7 +86,8 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are available online for free. * [Code Examples](http://spark-project.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark -* [Paper Describing the Spark System](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) +* [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) +* [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) # Community diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 18ad3ed255e8d..b30699cf3df86 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -514,3 +514,4 @@ JavaPairDStream wordCounts = words.map( # Where to Go from Here * API docs - [Scala](api/streaming/index.html#spark.streaming.package) and [Java](api/streaming/index.html#spark.streaming.api.java.package) * More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) +* [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) From baa30fcd99aec83b1b704d7918be6bb78b45fbb5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Feb 2013 23:38:50 -0800 Subject: [PATCH 331/696] Use new Spark EC2 scripts by default --- ec2/README | 2 +- ec2/spark_ec2.py | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/ec2/README b/ec2/README index 58dc087a81185..0add81312c428 100644 --- a/ec2/README +++ b/ec2/README @@ -1,4 +1,4 @@ This folder contains a script, spark-ec2, for launching Spark clusters on Amazon EC2. Usage instructions are available online at: -https://github.com/mesos/spark/wiki/Running-Spark-on-Amazon-EC2 +http://spark-project.org/docs/latest/ec2-scripts.html diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 66b1faf2cd831..571d27fde66ed 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -35,7 +35,7 @@ from boto import ec2 # A static URL from which to figure out the latest Mesos EC2 AMI -LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.6" +LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.7" # Configure and parse our command-line arguments @@ -83,16 +83,16 @@ def parse_args(): help="If specified, launch slaves as spot instances with the given " + "maximum price (in dollars)") parser.add_option("--cluster-type", type="choice", metavar="TYPE", - choices=["mesos", "standalone"], default="mesos", + choices=["mesos", "standalone"], default="standalone", help="'mesos' for a Mesos cluster, 'standalone' for a standalone " + - "Spark cluster (default: mesos)") + "Spark cluster (default: standalone)") parser.add_option("--ganglia", action="store_true", default=True, help="Setup Ganglia monitoring on cluster (default: on). NOTE: " + "the Ganglia page will be publicly accessible") parser.add_option("--no-ganglia", action="store_false", dest="ganglia", help="Disable Ganglia monitoring for the cluster") - parser.add_option("--new-scripts", action="store_true", default=False, - help="Use new spark-ec2 scripts, for Spark >= 0.7 AMIs") + parser.add_option("--old-scripts", action="store_true", default=False, + help="Use old mesos-ec2 scripts, for Spark <= 0.6 AMIs") parser.add_option("-u", "--user", default="root", help="The SSH user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, @@ -383,7 +383,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k if opts.ganglia: modules.append('ganglia') - if opts.new_scripts: + if not opts.old_scripts: # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git") @@ -393,7 +393,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k zoo_nodes, modules) print "Running setup on master..." - if not opts.new_scripts: + if opts.old_scripts: if opts.cluster_type == "mesos": setup_mesos_cluster(master, opts) elif opts.cluster_type == "standalone": From 4f840f4e541cb61d508879be05a05b64b528da22 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 27 Feb 2013 09:14:11 -0800 Subject: [PATCH 332/696] Added commented-out Google analytics code for website docs --- docs/_layouts/global.html | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 4b3da7ef573de..280ead03232ad 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -23,6 +23,22 @@ + + + + 4 spaces --- .../main/java/spark/examples/JavaKMeans.java | 164 ++++++++--------- .../java/spark/examples/JavaLogQuery.java | 171 +++++++++--------- .../main/java/spark/examples/JavaSparkPi.java | 65 +++---- 3 files changed, 200 insertions(+), 200 deletions(-) diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java index 2d7ba6aecee02..626034eb0d4d3 100644 --- a/examples/src/main/java/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/spark/examples/JavaKMeans.java @@ -16,99 +16,99 @@ */ public class JavaKMeans { - /** Parses numbers split by whitespace to a vector */ - static Vector parseVector(String line) { - String[] splits = line.split(" "); - double[] data = new double[splits.length]; - int i = 0; - for (String s : splits) - data[i] = Double.parseDouble(splits[i++]); - return new Vector(data); + /** Parses numbers split by whitespace to a vector */ + static Vector parseVector(String line) { + String[] splits = line.split(" "); + double[] data = new double[splits.length]; + int i = 0; + for (String s : splits) + data[i] = Double.parseDouble(splits[i++]); + return new Vector(data); + } + + /** Computes the vector to which the input vector is closest using squared distance */ + static int closestPoint(Vector p, List centers) { + int bestIndex = 0; + double closest = Double.POSITIVE_INFINITY; + for (int i = 0; i < centers.size(); i++) { + double tempDist = p.squaredDist(centers.get(i)); + if (tempDist < closest) { + closest = tempDist; + bestIndex = i; + } } + return bestIndex; + } - /** Computes the vector to which the input vector is closest using squared distance */ - static int closestPoint(Vector p, List centers) { - int bestIndex = 0; - double closest = Double.POSITIVE_INFINITY; - for (int i = 0; i < centers.size(); i++) { - double tempDist = p.squaredDist(centers.get(i)); - if (tempDist < closest) { - closest = tempDist; - bestIndex = i; - } - } - return bestIndex; + /** Computes the mean across all vectors in the input set of vectors */ + static Vector average(List ps) { + int numVectors = ps.size(); + Vector out = new Vector(ps.get(0).elements()); + // start from i = 1 since we already copied index 0 above + for (int i = 1; i < numVectors; i++) { + out.addInPlace(ps.get(i)); } + return out.divide(numVectors); + } - /** Computes the mean across all vectors in the input set of vectors */ - static Vector average(List ps) { - int numVectors = ps.size(); - Vector out = new Vector(ps.get(0).elements()); - // start from i = 1 since we already copied index 0 above - for (int i = 1; i < numVectors; i++) { - out.addInPlace(ps.get(i)); - } - return out.divide(numVectors); + public static void main(String[] args) throws Exception { + if (args.length < 4) { + System.err.println("Usage: JavaKMeans "); + System.exit(1); } + JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + String path = args[1]; + int K = Integer.parseInt(args[2]); + double convergeDist = Double.parseDouble(args[3]); - public static void main(String[] args) throws Exception { - if (args.length < 4) { - System.err.println("Usage: JavaKMeans "); - System.exit(1); + JavaRDD data = sc.textFile(path).map( + new Function() { + @Override + public Vector call(String line) throws Exception { + return parseVector(line); } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - String path = args[1]; - int K = Integer.parseInt(args[2]); - double convergeDist = Double.parseDouble(args[3]); + } + ).cache(); - JavaRDD data = sc.textFile(path).map( - new Function() { - @Override - public Vector call(String line) throws Exception { - return parseVector(line); - } - } - ).cache(); + final List centroids = data.takeSample(false, K, 42); - final List centroids = data.takeSample(false, K, 42); - - double tempDist; - do { - // allocate each vector to closest centroid - JavaPairRDD closest = data.map( - new PairFunction() { - @Override - public Tuple2 call(Vector vector) throws Exception { - return new Tuple2( - closestPoint(vector, centroids), vector); - } - } - ); + double tempDist; + do { + // allocate each vector to closest centroid + JavaPairRDD closest = data.map( + new PairFunction() { + @Override + public Tuple2 call(Vector vector) throws Exception { + return new Tuple2( + closestPoint(vector, centroids), vector); + } + } + ); - // group by cluster id and average the vectors within each cluster to compute centroids - JavaPairRDD> pointsGroup = closest.groupByKey(); - Map newCentroids = pointsGroup.mapValues( - new Function, Vector>() { - public Vector call(List ps) throws Exception { - return average(ps); - } - }).collectAsMap(); - tempDist = 0.0; - for (int i = 0; i < K; i++) { - tempDist += centroids.get(i).squaredDist(newCentroids.get(i)); - } - for (Map.Entry t: newCentroids.entrySet()) { - centroids.set(t.getKey(), t.getValue()); - } - System.out.println("Finished iteration (delta = " + tempDist + ")"); - } while (tempDist > convergeDist); + // group by cluster id and average the vectors within each cluster to compute centroids + JavaPairRDD> pointsGroup = closest.groupByKey(); + Map newCentroids = pointsGroup.mapValues( + new Function, Vector>() { + public Vector call(List ps) throws Exception { + return average(ps); + } + }).collectAsMap(); + tempDist = 0.0; + for (int i = 0; i < K; i++) { + tempDist += centroids.get(i).squaredDist(newCentroids.get(i)); + } + for (Map.Entry t: newCentroids.entrySet()) { + centroids.set(t.getKey(), t.getValue()); + } + System.out.println("Finished iteration (delta = " + tempDist + ")"); + } while (tempDist > convergeDist); - System.out.println("Final centers:"); - for (Vector c : centroids) - System.out.println(c); + System.out.println("Final centers:"); + for (Vector c : centroids) + System.out.println(c); - System.exit(0); + System.exit(0); -} + } } diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java index 40f33aaa735d1..6b22e7120c917 100644 --- a/examples/src/main/java/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/spark/examples/JavaLogQuery.java @@ -20,96 +20,95 @@ */ public class JavaLogQuery { - public static List exampleApacheLogs = Lists.newArrayList( - "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + - "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + - "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + - ".NET CLR 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + - "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.350 \"-\" - \"\" 265 923 934 \"\" " + - "62.24.11.25 images.com 1358492167 - Whatup", - "10.10.10.10 - \"FRED\" [18/Jan/2013:18:02:37 +1100] \"GET http://images.com/2013/Generic.jpg " + - "HTTP/1.1\" 304 306 \"http:/referall.com\" \"Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; " + - "GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR " + - "3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + - "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + - "0 73.23.2.15 images.com 1358492557 - Whatup"); - - - public static Pattern apacheLogRegex = Pattern.compile( - "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); - - /** Tracks the total query count and number of aggregate bytes for a particular group. */ - public static class Stats implements Serializable { - - private int count; - private int numBytes; - - public Stats(int count, int numBytes) { - this.count = count; - this.numBytes = numBytes; - } - public Stats merge(Stats other) { - return new Stats(count + other.count, numBytes + other.numBytes); - } - - public String toString() { - return String.format("bytes=%s\tn=%s", numBytes, count); - } + public static List exampleApacheLogs = Lists.newArrayList( + "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + + "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + + "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + + ".NET CLR 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + + "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.350 \"-\" - \"\" 265 923 934 \"\" " + + "62.24.11.25 images.com 1358492167 - Whatup", + "10.10.10.10 - \"FRED\" [18/Jan/2013:18:02:37 +1100] \"GET http://images.com/2013/Generic.jpg " + + "HTTP/1.1\" 304 306 \"http:/referall.com\" \"Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; " + + "GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR " + + "3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + + "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + + "0 73.23.2.15 images.com 1358492557 - Whatup"); + + public static Pattern apacheLogRegex = Pattern.compile( + "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); + + /** Tracks the total query count and number of aggregate bytes for a particular group. */ + public static class Stats implements Serializable { + + private int count; + private int numBytes; + + public Stats(int count, int numBytes) { + this.count = count; + this.numBytes = numBytes; } - - public static Tuple3 extractKey(String line) { - Matcher m = apacheLogRegex.matcher(line); - List key = Collections.emptyList(); - if (m.find()) { - String ip = m.group(1); - String user = m.group(3); - String query = m.group(5); - if (!user.equalsIgnoreCase("-")) { - return new Tuple3(ip, user, query); - } - } - return new Tuple3(null, null, null); + public Stats merge(Stats other) { + return new Stats(count + other.count, numBytes + other.numBytes); } - public static Stats extractStats(String line) { - Matcher m = apacheLogRegex.matcher(line); - if (m.find()) { - int bytes = Integer.parseInt(m.group(7)); - return new Stats(1, bytes); - } - else - return new Stats(1, 0); + public String toString() { + return String.format("bytes=%s\tn=%s", numBytes, count); + } + } + + public static Tuple3 extractKey(String line) { + Matcher m = apacheLogRegex.matcher(line); + List key = Collections.emptyList(); + if (m.find()) { + String ip = m.group(1); + String user = m.group(3); + String query = m.group(5); + if (!user.equalsIgnoreCase("-")) { + return new Tuple3(ip, user, query); + } + } + return new Tuple3(null, null, null); + } + + public static Stats extractStats(String line) { + Matcher m = apacheLogRegex.matcher(line); + if (m.find()) { + int bytes = Integer.parseInt(m.group(7)); + return new Stats(1, bytes); + } + else + return new Stats(1, 0); + } + + public static void main(String[] args) throws Exception { + if (args.length == 0) { + System.err.println("Usage: JavaLogQuery [logFile]"); + System.exit(1); } - public static void main(String[] args) throws Exception { - if (args.length == 0) { - System.err.println("Usage: JavaLogQuery [logFile]"); - System.exit(1); - } - - JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - - JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); - - JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { - @Override - public Tuple2, Stats> call(String s) throws Exception { - return new Tuple2, Stats>(extractKey(s), extractStats(s)); - } - }); - - JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { - @Override - public Stats call(Stats stats, Stats stats2) throws Exception { - return stats.merge(stats2); - } - }); - - List, Stats>> output = counts.collect(); - for (Tuple2 t : output) { - System.out.println(t._1 + "\t" + t._2); - } - System.exit(0); + JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + + JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); + + JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { + @Override + public Tuple2, Stats> call(String s) throws Exception { + return new Tuple2, Stats>(extractKey(s), extractStats(s)); + } + }); + + JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { + @Override + public Stats call(Stats stats, Stats stats2) throws Exception { + return stats.merge(stats2); + } + }); + + List, Stats>> output = counts.collect(); + for (Tuple2 t : output) { + System.out.println(t._1 + "\t" + t._2); } + System.exit(0); + } } diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java index e4cee97a42e0f..a15a967de85df 100644 --- a/examples/src/main/java/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/spark/examples/JavaSparkPi.java @@ -11,37 +11,38 @@ /** Computes an approximation to pi */ public class JavaSparkPi { - public static void main(String[] args) throws Exception { - if (args.length == 0) { - System.err.println("Usage: JavaLogQuery [slices]"); - System.exit(1); - } - - JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - - int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; - int n = 100000 * slices; - List l = new ArrayList(n); - for (int i = 0; i < n; i++) - l.add(i); - - JavaRDD dataSet = jsc.parallelize(l, slices); - - int count = dataSet.map(new Function() { - @Override - public Integer call(Integer integer) throws Exception { - double x = Math.random() * 2 - 1; - double y = Math.random() * 2 - 1; - return (x * x + y * y < 1) ? 1 : 0; - } - }).reduce(new Function2() { - @Override - public Integer call(Integer integer, Integer integer2) throws Exception { - return integer + integer2; - } - }); - - System.out.println("Pi is roughly " + 4.0 * count / n); + + public static void main(String[] args) throws Exception { + if (args.length == 0) { + System.err.println("Usage: JavaLogQuery [slices]"); + System.exit(1); } + + JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + + int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; + int n = 100000 * slices; + List l = new ArrayList(n); + for (int i = 0; i < n; i++) + l.add(i); + + JavaRDD dataSet = jsc.parallelize(l, slices); + + int count = dataSet.map(new Function() { + @Override + public Integer call(Integer integer) throws Exception { + double x = Math.random() * 2 - 1; + double y = Math.random() * 2 - 1; + return (x * x + y * y < 1) ? 1 : 0; + } + }).reduce(new Function2() { + @Override + public Integer call(Integer integer, Integer integer2) throws Exception { + return integer + integer2; + } + }); + + System.out.println("Pi is roughly " + 4.0 * count / n); + } } From 52398cc1a3ac0a3cd534224986f585a03e0c6259 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 20 Mar 2013 09:49:57 +0200 Subject: [PATCH 399/696] Java indentation 4 --> 2 spaces --- .../main/java/spark/examples/JavaKMeans.java | 164 ++++++++--------- .../java/spark/examples/JavaLogQuery.java | 171 +++++++++--------- .../main/java/spark/examples/JavaSparkPi.java | 65 +++---- 3 files changed, 200 insertions(+), 200 deletions(-) diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java index 2d7ba6aecee02..626034eb0d4d3 100644 --- a/examples/src/main/java/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/spark/examples/JavaKMeans.java @@ -16,99 +16,99 @@ */ public class JavaKMeans { - /** Parses numbers split by whitespace to a vector */ - static Vector parseVector(String line) { - String[] splits = line.split(" "); - double[] data = new double[splits.length]; - int i = 0; - for (String s : splits) - data[i] = Double.parseDouble(splits[i++]); - return new Vector(data); + /** Parses numbers split by whitespace to a vector */ + static Vector parseVector(String line) { + String[] splits = line.split(" "); + double[] data = new double[splits.length]; + int i = 0; + for (String s : splits) + data[i] = Double.parseDouble(splits[i++]); + return new Vector(data); + } + + /** Computes the vector to which the input vector is closest using squared distance */ + static int closestPoint(Vector p, List centers) { + int bestIndex = 0; + double closest = Double.POSITIVE_INFINITY; + for (int i = 0; i < centers.size(); i++) { + double tempDist = p.squaredDist(centers.get(i)); + if (tempDist < closest) { + closest = tempDist; + bestIndex = i; + } } + return bestIndex; + } - /** Computes the vector to which the input vector is closest using squared distance */ - static int closestPoint(Vector p, List centers) { - int bestIndex = 0; - double closest = Double.POSITIVE_INFINITY; - for (int i = 0; i < centers.size(); i++) { - double tempDist = p.squaredDist(centers.get(i)); - if (tempDist < closest) { - closest = tempDist; - bestIndex = i; - } - } - return bestIndex; + /** Computes the mean across all vectors in the input set of vectors */ + static Vector average(List ps) { + int numVectors = ps.size(); + Vector out = new Vector(ps.get(0).elements()); + // start from i = 1 since we already copied index 0 above + for (int i = 1; i < numVectors; i++) { + out.addInPlace(ps.get(i)); } + return out.divide(numVectors); + } - /** Computes the mean across all vectors in the input set of vectors */ - static Vector average(List ps) { - int numVectors = ps.size(); - Vector out = new Vector(ps.get(0).elements()); - // start from i = 1 since we already copied index 0 above - for (int i = 1; i < numVectors; i++) { - out.addInPlace(ps.get(i)); - } - return out.divide(numVectors); + public static void main(String[] args) throws Exception { + if (args.length < 4) { + System.err.println("Usage: JavaKMeans "); + System.exit(1); } + JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + String path = args[1]; + int K = Integer.parseInt(args[2]); + double convergeDist = Double.parseDouble(args[3]); - public static void main(String[] args) throws Exception { - if (args.length < 4) { - System.err.println("Usage: JavaKMeans "); - System.exit(1); + JavaRDD data = sc.textFile(path).map( + new Function() { + @Override + public Vector call(String line) throws Exception { + return parseVector(line); } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - String path = args[1]; - int K = Integer.parseInt(args[2]); - double convergeDist = Double.parseDouble(args[3]); + } + ).cache(); - JavaRDD data = sc.textFile(path).map( - new Function() { - @Override - public Vector call(String line) throws Exception { - return parseVector(line); - } - } - ).cache(); + final List centroids = data.takeSample(false, K, 42); - final List centroids = data.takeSample(false, K, 42); - - double tempDist; - do { - // allocate each vector to closest centroid - JavaPairRDD closest = data.map( - new PairFunction() { - @Override - public Tuple2 call(Vector vector) throws Exception { - return new Tuple2( - closestPoint(vector, centroids), vector); - } - } - ); + double tempDist; + do { + // allocate each vector to closest centroid + JavaPairRDD closest = data.map( + new PairFunction() { + @Override + public Tuple2 call(Vector vector) throws Exception { + return new Tuple2( + closestPoint(vector, centroids), vector); + } + } + ); - // group by cluster id and average the vectors within each cluster to compute centroids - JavaPairRDD> pointsGroup = closest.groupByKey(); - Map newCentroids = pointsGroup.mapValues( - new Function, Vector>() { - public Vector call(List ps) throws Exception { - return average(ps); - } - }).collectAsMap(); - tempDist = 0.0; - for (int i = 0; i < K; i++) { - tempDist += centroids.get(i).squaredDist(newCentroids.get(i)); - } - for (Map.Entry t: newCentroids.entrySet()) { - centroids.set(t.getKey(), t.getValue()); - } - System.out.println("Finished iteration (delta = " + tempDist + ")"); - } while (tempDist > convergeDist); + // group by cluster id and average the vectors within each cluster to compute centroids + JavaPairRDD> pointsGroup = closest.groupByKey(); + Map newCentroids = pointsGroup.mapValues( + new Function, Vector>() { + public Vector call(List ps) throws Exception { + return average(ps); + } + }).collectAsMap(); + tempDist = 0.0; + for (int i = 0; i < K; i++) { + tempDist += centroids.get(i).squaredDist(newCentroids.get(i)); + } + for (Map.Entry t: newCentroids.entrySet()) { + centroids.set(t.getKey(), t.getValue()); + } + System.out.println("Finished iteration (delta = " + tempDist + ")"); + } while (tempDist > convergeDist); - System.out.println("Final centers:"); - for (Vector c : centroids) - System.out.println(c); + System.out.println("Final centers:"); + for (Vector c : centroids) + System.out.println(c); - System.exit(0); + System.exit(0); -} + } } diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java index 40f33aaa735d1..6b22e7120c917 100644 --- a/examples/src/main/java/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/spark/examples/JavaLogQuery.java @@ -20,96 +20,95 @@ */ public class JavaLogQuery { - public static List exampleApacheLogs = Lists.newArrayList( - "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + - "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + - "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + - ".NET CLR 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + - "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.350 \"-\" - \"\" 265 923 934 \"\" " + - "62.24.11.25 images.com 1358492167 - Whatup", - "10.10.10.10 - \"FRED\" [18/Jan/2013:18:02:37 +1100] \"GET http://images.com/2013/Generic.jpg " + - "HTTP/1.1\" 304 306 \"http:/referall.com\" \"Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; " + - "GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR " + - "3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + - "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + - "0 73.23.2.15 images.com 1358492557 - Whatup"); - - - public static Pattern apacheLogRegex = Pattern.compile( - "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); - - /** Tracks the total query count and number of aggregate bytes for a particular group. */ - public static class Stats implements Serializable { - - private int count; - private int numBytes; - - public Stats(int count, int numBytes) { - this.count = count; - this.numBytes = numBytes; - } - public Stats merge(Stats other) { - return new Stats(count + other.count, numBytes + other.numBytes); - } - - public String toString() { - return String.format("bytes=%s\tn=%s", numBytes, count); - } + public static List exampleApacheLogs = Lists.newArrayList( + "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + + "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + + "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + + ".NET CLR 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + + "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.350 \"-\" - \"\" 265 923 934 \"\" " + + "62.24.11.25 images.com 1358492167 - Whatup", + "10.10.10.10 - \"FRED\" [18/Jan/2013:18:02:37 +1100] \"GET http://images.com/2013/Generic.jpg " + + "HTTP/1.1\" 304 306 \"http:/referall.com\" \"Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; " + + "GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR " + + "3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + + "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + + "0 73.23.2.15 images.com 1358492557 - Whatup"); + + public static Pattern apacheLogRegex = Pattern.compile( + "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); + + /** Tracks the total query count and number of aggregate bytes for a particular group. */ + public static class Stats implements Serializable { + + private int count; + private int numBytes; + + public Stats(int count, int numBytes) { + this.count = count; + this.numBytes = numBytes; } - - public static Tuple3 extractKey(String line) { - Matcher m = apacheLogRegex.matcher(line); - List key = Collections.emptyList(); - if (m.find()) { - String ip = m.group(1); - String user = m.group(3); - String query = m.group(5); - if (!user.equalsIgnoreCase("-")) { - return new Tuple3(ip, user, query); - } - } - return new Tuple3(null, null, null); + public Stats merge(Stats other) { + return new Stats(count + other.count, numBytes + other.numBytes); } - public static Stats extractStats(String line) { - Matcher m = apacheLogRegex.matcher(line); - if (m.find()) { - int bytes = Integer.parseInt(m.group(7)); - return new Stats(1, bytes); - } - else - return new Stats(1, 0); + public String toString() { + return String.format("bytes=%s\tn=%s", numBytes, count); + } + } + + public static Tuple3 extractKey(String line) { + Matcher m = apacheLogRegex.matcher(line); + List key = Collections.emptyList(); + if (m.find()) { + String ip = m.group(1); + String user = m.group(3); + String query = m.group(5); + if (!user.equalsIgnoreCase("-")) { + return new Tuple3(ip, user, query); + } + } + return new Tuple3(null, null, null); + } + + public static Stats extractStats(String line) { + Matcher m = apacheLogRegex.matcher(line); + if (m.find()) { + int bytes = Integer.parseInt(m.group(7)); + return new Stats(1, bytes); + } + else + return new Stats(1, 0); + } + + public static void main(String[] args) throws Exception { + if (args.length == 0) { + System.err.println("Usage: JavaLogQuery [logFile]"); + System.exit(1); } - public static void main(String[] args) throws Exception { - if (args.length == 0) { - System.err.println("Usage: JavaLogQuery [logFile]"); - System.exit(1); - } - - JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - - JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); - - JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { - @Override - public Tuple2, Stats> call(String s) throws Exception { - return new Tuple2, Stats>(extractKey(s), extractStats(s)); - } - }); - - JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { - @Override - public Stats call(Stats stats, Stats stats2) throws Exception { - return stats.merge(stats2); - } - }); - - List, Stats>> output = counts.collect(); - for (Tuple2 t : output) { - System.out.println(t._1 + "\t" + t._2); - } - System.exit(0); + JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + + JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); + + JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { + @Override + public Tuple2, Stats> call(String s) throws Exception { + return new Tuple2, Stats>(extractKey(s), extractStats(s)); + } + }); + + JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { + @Override + public Stats call(Stats stats, Stats stats2) throws Exception { + return stats.merge(stats2); + } + }); + + List, Stats>> output = counts.collect(); + for (Tuple2 t : output) { + System.out.println(t._1 + "\t" + t._2); } + System.exit(0); + } } diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java index e4cee97a42e0f..a15a967de85df 100644 --- a/examples/src/main/java/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/spark/examples/JavaSparkPi.java @@ -11,37 +11,38 @@ /** Computes an approximation to pi */ public class JavaSparkPi { - public static void main(String[] args) throws Exception { - if (args.length == 0) { - System.err.println("Usage: JavaLogQuery [slices]"); - System.exit(1); - } - - JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - - int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; - int n = 100000 * slices; - List l = new ArrayList(n); - for (int i = 0; i < n; i++) - l.add(i); - - JavaRDD dataSet = jsc.parallelize(l, slices); - - int count = dataSet.map(new Function() { - @Override - public Integer call(Integer integer) throws Exception { - double x = Math.random() * 2 - 1; - double y = Math.random() * 2 - 1; - return (x * x + y * y < 1) ? 1 : 0; - } - }).reduce(new Function2() { - @Override - public Integer call(Integer integer, Integer integer2) throws Exception { - return integer + integer2; - } - }); - - System.out.println("Pi is roughly " + 4.0 * count / n); + + public static void main(String[] args) throws Exception { + if (args.length == 0) { + System.err.println("Usage: JavaLogQuery [slices]"); + System.exit(1); } + + JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + + int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; + int n = 100000 * slices; + List l = new ArrayList(n); + for (int i = 0; i < n; i++) + l.add(i); + + JavaRDD dataSet = jsc.parallelize(l, slices); + + int count = dataSet.map(new Function() { + @Override + public Integer call(Integer integer) throws Exception { + double x = Math.random() * 2 - 1; + double y = Math.random() * 2 - 1; + return (x * x + y * y < 1) ? 1 : 0; + } + }).reduce(new Function2() { + @Override + public Integer call(Integer integer, Integer integer2) throws Exception { + return integer + integer2; + } + }); + + System.out.println("Pi is roughly " + 4.0 * count / n); + } } From 446b801b3b888b335e8be3a9ab7a29aba3285d78 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Wed, 20 Mar 2013 17:30:31 -0700 Subject: [PATCH 400/696] Fixing typos pointed out by Matei --- docs/building-with-maven.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index bbf89cf17a2cf..c2eeafd07af75 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -29,11 +29,11 @@ It uses the scala-maven-plugin which supports incremental and continuous compila Tests are run by default via the scalatest-maven-plugin. With this you can do things like: -skip test execution (but not compilation): +Skip test execution (but not compilation): $ mvn -DskipTests -Phadoop2 clean install -To runn a specific test suite: +To run a specific test suite: $ mvn -Phadoop2 -Dsuites=spark.repl.ReplSuite test From 445f387ef4213b5b63f28cbc011236c2aba1d1c9 Mon Sep 17 00:00:00 2001 From: Christoph Grothaus Date: Mon, 18 Feb 2013 13:54:01 +0100 Subject: [PATCH 401/696] Bugfix: WorkerWebUI must respect workDirPath from Worker --- core/src/main/scala/spark/deploy/worker/Worker.scala | 2 +- core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index da3f4f636ce84..8919d1261cc86 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -81,7 +81,7 @@ private[spark] class Worker( } def startWebUi() { - val webUi = new WorkerWebUI(context.system, self) + val webUi = new WorkerWebUI(context.system, self, workDir) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 135cc2e86cc92..c834f87d50111 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -12,12 +12,13 @@ import cc.spray.typeconversion.SprayJsonSupport._ import spark.deploy.{WorkerState, RequestWorkerState} import spark.deploy.JsonProtocol._ +import java.io.File /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { +class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) extends Directives { val RESOURCE_DIR = "spark/deploy/worker/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" @@ -43,7 +44,7 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct path("log") { parameters("appId", "executorId", "logType") { (appId, executorId, logType) => respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) { - getFromFileName("work/" + appId + "/" + executorId + "/" + logType) + getFromFileName(workDir.getPath() + "/" + appId + "/" + executorId + "/" + logType) } } } ~ From 1c67c7dfd1660dd39226742eedcb2948ab1445d0 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 22 Mar 2013 08:54:44 -0500 Subject: [PATCH 402/696] Add a shuffle parameter to coalesce. This is useful for when you want just 1 output file (part-00000) but still up the upstream RDD to be computed in parallel. --- core/src/main/scala/spark/RDD.scala | 10 +++++++++- core/src/main/scala/spark/api/java/JavaDoubleRDD.scala | 6 ++++++ core/src/main/scala/spark/api/java/JavaPairRDD.scala | 8 +++++++- core/src/main/scala/spark/api/java/JavaRDD.scala | 6 ++++++ core/src/test/scala/spark/RDDSuite.scala | 6 +++++- 5 files changed, 33 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 9bd8a0f98daa8..0cd904f89d8a2 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -31,6 +31,7 @@ import spark.rdd.MapPartitionsRDD import spark.rdd.MapPartitionsWithIndexRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD +import spark.rdd.ShuffledRDD import spark.rdd.SubtractedRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD @@ -237,7 +238,14 @@ abstract class RDD[T: ClassManifest]( /** * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numPartitions: Int): RDD[T] = new CoalescedRDD(this, numPartitions) + def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { + if (shuffle) { + // include a shuffle step so that are upstream tasks are still distributed + new CoalescedRDD(new ShuffledRDD(map(x => (x, null)), new HashPartitioner(numPartitions)), numPartitions).keys + } else { + new CoalescedRDD(this, numPartitions) + } + } /** * Return a sampled subset of this RDD. diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index ba00b6a8448f1..26cbb1a641d6a 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -57,6 +57,12 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav */ def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions)) + /** + * Return a new RDD that is reduced into `numPartitions` partitions. + */ + def coalesce(numPartitions: Int, shuffle: java.lang.Boolean): JavaDoubleRDD = + fromRDD(srdd.coalesce(numPartitions, shuffle)) + /** * Return an RDD with the elements from `this` that are not in `other`. * diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 49aaabf835648..9ce77e8e84f72 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -66,7 +66,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.coalesce(numPartitions)) + def coalesce(numPartitions: Int): JavaPairRDD[K, V] = fromRDD(rdd.coalesce(numPartitions)) + + /** + * Return a new RDD that is reduced into `numPartitions` partitions. + */ + def coalesce(numPartitions: Int, shuffle: java.lang.Boolean): JavaPairRDD[K, V] = + fromRDD(rdd.coalesce(numPartitions, shuffle)) /** * Return a sampled subset of this RDD. diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 301688889898e..7223dbbe645c2 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -43,6 +43,12 @@ JavaRDDLike[T, JavaRDD[T]] { */ def coalesce(numPartitions: Int): JavaRDD[T] = rdd.coalesce(numPartitions) + /** + * Return a new RDD that is reduced into `numPartitions` partitions. + */ + def coalesce(numPartitions: Int, shuffle: java.lang.Boolean): JavaRDD[T] = + rdd.coalesce(numPartitions, shuffle) + /** * Return a sampled subset of this RDD. */ diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 9739ba869b318..bcbb472f6c219 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -3,7 +3,7 @@ package spark import scala.collection.mutable.HashMap import org.scalatest.FunSuite import spark.SparkContext._ -import spark.rdd.{CoalescedRDD, PartitionPruningRDD} +import spark.rdd.{CoalescedRDD, PartitionPruningRDD, ShuffledRDD} class RDDSuite extends FunSuite with LocalSparkContext { @@ -154,6 +154,10 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(coalesced4.collect().toList === (1 to 10).toList) assert(coalesced4.glom().collect().map(_.toList).toList === (1 to 10).map(x => List(x)).toList) + + // we can optionally shuffle to keep the upstream parallel + val coalesced5 = data.coalesce(1, shuffle = true) + assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _]] ne null) } test("zipped RDDs") { From 00170eb0b9c30c1ab4f470c694b434385b8cbb7a Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 22 Mar 2013 12:59:08 -0500 Subject: [PATCH 403/696] Fix are/our typo. --- core/src/main/scala/spark/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 0cd904f89d8a2..b88d711dcfe2b 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -240,7 +240,7 @@ abstract class RDD[T: ClassManifest]( */ def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { if (shuffle) { - // include a shuffle step so that are upstream tasks are still distributed + // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD(new ShuffledRDD(map(x => (x, null)), new HashPartitioner(numPartitions)), numPartitions).keys } else { new CoalescedRDD(this, numPartitions) From d1d9bdaabe24cc60097f843e0bef92e57b404941 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 23 Mar 2013 07:25:30 +0800 Subject: [PATCH 404/696] Just update typo and comments --- core/src/main/scala/spark/scheduler/cluster/Pool.scala | 3 +-- .../main/scala/spark/scheduler/cluster/TaskSetManager.scala | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 7b58a99582113..68e1d2a75afed 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -5,8 +5,7 @@ import scala.collection.mutable.ArrayBuffer import spark.Logging import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** - * An interface for - * + * An Schedulable entity that represent collection of TaskSetManager */ private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,val minShare:Int, val weight:Int) extends Schedulable with Logging { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 723c3b46bdd3c..064593f48641b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -28,7 +28,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Maximum times a task is allowed to fail before failing the job val MAX_TASK_FAILURES = 4 - val TASKSET_MINIMIUM_SHARES = 1 + val TASKSET_MINIMUM_SHARES = 1 val TASKSET_WEIGHT = 1 // Quantile of tasks at which to start speculation @@ -102,7 +102,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe override def getMinShare(): Int = { - return TASKSET_MINIMIUM_SHARES + return TASKSET_MINIMUM_SHARES } override def getRunningTasks(): Int = From dd854d5b9fdc1fe60af4dc649af4202a8ddac0d8 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 23 Mar 2013 11:49:45 -0500 Subject: [PATCH 405/696] Use Boolean in the Java API, and != for assert. --- core/src/main/scala/spark/api/java/JavaDoubleRDD.scala | 2 +- core/src/main/scala/spark/api/java/JavaPairRDD.scala | 2 +- core/src/main/scala/spark/api/java/JavaRDD.scala | 2 +- core/src/test/scala/spark/RDDSuite.scala | 3 ++- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 26cbb1a641d6a..16692c044037c 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -60,7 +60,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numPartitions: Int, shuffle: java.lang.Boolean): JavaDoubleRDD = + def coalesce(numPartitions: Int, shuffle: Boolean): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions, shuffle)) /** diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 9ce77e8e84f72..30084df4e20fc 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -71,7 +71,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numPartitions: Int, shuffle: java.lang.Boolean): JavaPairRDD[K, V] = + def coalesce(numPartitions: Int, shuffle: Boolean): JavaPairRDD[K, V] = fromRDD(rdd.coalesce(numPartitions, shuffle)) /** diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 7223dbbe645c2..e29f1e58990b8 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -46,7 +46,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** * Return a new RDD that is reduced into `numPartitions` partitions. */ - def coalesce(numPartitions: Int, shuffle: java.lang.Boolean): JavaRDD[T] = + def coalesce(numPartitions: Int, shuffle: Boolean): JavaRDD[T] = rdd.coalesce(numPartitions, shuffle) /** diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index a6178867bc6e0..7fbdd44340abe 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -187,7 +187,8 @@ class RDDSuite extends FunSuite with LocalSparkContext { // we can optionally shuffle to keep the upstream parallel val coalesced5 = data.coalesce(1, shuffle = true) - assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _]] ne null) + assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _]] != + null) } test("zipped RDDs") { From f5df729b12c80a847de0e788956b9020266ad91f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 24 Mar 2013 16:15:32 -0700 Subject: [PATCH 406/696] Explicitly catch all throwables (warning in 2.10) --- core/src/main/scala/spark/CacheManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala index c7b379a3fbe2d..f7a2b7e8027ef 100644 --- a/core/src/main/scala/spark/CacheManager.scala +++ b/core/src/main/scala/spark/CacheManager.scala @@ -27,7 +27,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { if (loading.contains(key)) { logInfo("Loading contains " + key + ", waiting...") while (loading.contains(key)) { - try {loading.wait()} catch {case _ =>} + try {loading.wait()} catch {case _ : Throwable =>} } logInfo("Loading no longer contains " + key + ", so returning cached result") // See whether someone else has successfully loaded it. The main way this would fail From e104a760164137cee2ad547178a99ca1863561e5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 24 Mar 2013 16:16:05 -0700 Subject: [PATCH 407/696] Makes the syntax highlighting on the build file not broken in emacs. --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d44bf3b5e3937..554cf6c539b72 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -37,7 +37,7 @@ object SparkBuild extends Build { organization := "org.spark-project", version := "0.7.1-SNAPSHOT", scalaVersion := "2.9.2", - scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + scalacOptions := Seq(/*"-deprecation", */ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", From 8456d673e27ee557b2e6fc1ad93b46058e039cea Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 24 Mar 2013 16:20:34 -0700 Subject: [PATCH 408/696] Re-enable deprecation warnings since there are only two --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 554cf6c539b72..5f378b2398c5b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -37,7 +37,7 @@ object SparkBuild extends Build { organization := "org.spark-project", version := "0.7.1-SNAPSHOT", scalaVersion := "2.9.2", - scalacOptions := Seq(/*"-deprecation", */ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", From 1f5381119f8c8afd0ba69bc7773c10972dd43bc1 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 24 Mar 2013 16:21:51 -0700 Subject: [PATCH 409/696] method first in trait IterableLike is deprecated: use `head' instead --- .../main/scala/spark/streaming/dstream/QueueInputDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala index 6b310bc0b611c..da224ad6f718c 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala @@ -28,7 +28,7 @@ class QueueInputDStream[T: ClassManifest]( } if (buffer.size > 0) { if (oneAtATime) { - Some(buffer.first) + Some(buffer.head) } else { Some(new UnionRDD(ssc.sc, buffer.toSeq)) } From 6c18131487e8537da19e2daf45b385c5af494ea3 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Tue, 26 Mar 2013 14:28:08 -0700 Subject: [PATCH 410/696] Add comment to README that 2.10 not yet supported --- README.md | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 1f8f7b6876c4c..ba24ab43b1a10 100644 --- a/README.md +++ b/README.md @@ -12,12 +12,15 @@ This README file only contains basic setup instructions. ## Building -Spark requires Scala 2.9.2. The project is built using Simple Build Tool (SBT), -which is packaged with it. To build Spark and its example programs, run: +Spark requires Scala 2.9.2 (Scala 2.10 is not yet supported). The project is +built using Simple Build Tool (SBT), which is packaged with it. To build +Spark and its example programs, run: sbt/sbt package -Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) in the spark documentation.. +Spark also supports building using Maven. If you would like to build using Maven, +see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) +in the spark documentation.. To run Spark, you will need to have Scala's bin directory in your `PATH`, or you will need to set the `SCALA_HOME` environment variable to point to where From f0164e5047f3a023768cdf0515a7a7d1c75ef7c3 Mon Sep 17 00:00:00 2001 From: kalpit Date: Wed, 6 Mar 2013 17:03:15 -0800 Subject: [PATCH 411/696] upgraded sbt version, sbt plugins and some library dependencies to latest stable version --- project/SparkBuild.scala | 8 ++++---- project/build.properties | 2 +- project/plugins.sbt | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f378b2398c5b..ea146b7b0b1d1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -92,10 +92,10 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", - "org.scalatest" %% "scalatest" % "1.8" % "test", - "org.scalacheck" %% "scalacheck" % "1.9" % "test", - "com.novocode" % "junit-interface" % "0.8" % "test", + "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", + "com.novocode" % "junit-interface" % "0.9" % "test", "org.easymock" % "easymock" % "3.1" % "test" ), parallelExecution := false, diff --git a/project/build.properties b/project/build.properties index d4287112c6afb..66ad72ce2eba1 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.11.3 +sbt.version=0.12.2 diff --git a/project/plugins.sbt b/project/plugins.sbt index 4d0e696a11358..d4f2442872363 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -4,13 +4,13 @@ resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/release resolvers += "Spray Repository" at "http://repo.spray.cc/" -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.5") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1") +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0") +addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") -addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2") +addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") // For Sonatype publishing //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) From f08db010d3ba4eff132f5c06a087b0c7c2e58576 Mon Sep 17 00:00:00 2001 From: kalpit Date: Wed, 6 Mar 2013 18:06:32 -0800 Subject: [PATCH 412/696] added SPARK_WORKER_INSTANCES : allows spawning multiple worker instances/processes on every slave machine --- bin/spark-daemon.sh | 6 ++++-- bin/spark-daemons.sh | 2 +- bin/start-master.sh | 2 +- bin/start-slave.sh | 2 +- bin/start-slaves.sh | 11 ++++++++++- bin/stop-master.sh | 2 +- bin/stop-slaves.sh | 12 +++++++++++- conf/spark-env.sh.template | 1 + 8 files changed, 30 insertions(+), 8 deletions(-) diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index 0c584055c7946..d7dc62ab08162 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -30,7 +30,7 @@ # SPARK_NICENESS The scheduling priority for daemons. Defaults to 0. ## -usage="Usage: spark-daemon.sh [--config ] [--hosts hostlistfile] (start|stop) " +usage="Usage: spark-daemon.sh [--config ] [--hosts hostlistfile] (start|stop) " # if no args specified, show usage if [ $# -le 1 ]; then @@ -48,6 +48,8 @@ startStop=$1 shift command=$1 shift +instance=$1 +shift spark_rotate_log () { @@ -95,7 +97,7 @@ fi export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$HOSTNAME.log export SPARK_ROOT_LOGGER="INFO,DRFA" log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$HOSTNAME.out -pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command.pid +pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid # Set default scheduling priority if [ "$SPARK_NICENESS" = "" ]; then diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh index 4f9719ee809e8..0619097e4dc3a 100755 --- a/bin/spark-daemons.sh +++ b/bin/spark-daemons.sh @@ -2,7 +2,7 @@ # Run a Spark command on all slave hosts. -usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command args..." +usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..." # if no args specified, show usage if [ $# -le 1 ]; then diff --git a/bin/start-master.sh b/bin/start-master.sh index 87feb261fe86b..83a3e1f3dc1a3 100755 --- a/bin/start-master.sh +++ b/bin/start-master.sh @@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.master.Master --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT +"$bin"/spark-daemon.sh start spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 45a0cf7a6b7ac..616c76e4ee6e0 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -11,4 +11,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.worker.Worker $1 +"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@" diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh index 390247ca4aa49..4e05224190e3b 100755 --- a/bin/start-slaves.sh +++ b/bin/start-slaves.sh @@ -21,4 +21,13 @@ fi echo "Master IP: $SPARK_MASTER_IP" # Launch the slaves -exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT +else + if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then + SPARK_WORKER_WEBUI_PORT=8081 + fi + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" $(( $i + 1 )) spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) + done +fi diff --git a/bin/stop-master.sh b/bin/stop-master.sh index f75167dd2c72d..172ee5891d17f 100755 --- a/bin/stop-master.sh +++ b/bin/stop-master.sh @@ -7,4 +7,4 @@ bin=`cd "$bin"; pwd` . "$bin/spark-config.sh" -"$bin"/spark-daemon.sh stop spark.deploy.master.Master \ No newline at end of file +"$bin"/spark-daemon.sh stop spark.deploy.master.Master 1 diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh index 21c9ebf324fdc..fbfc594472fe7 100755 --- a/bin/stop-slaves.sh +++ b/bin/stop-slaves.sh @@ -7,4 +7,14 @@ bin=`cd "$bin"; pwd` . "$bin/spark-config.sh" -"$bin"/spark-daemons.sh stop spark.deploy.worker.Worker \ No newline at end of file +if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then + . "${SPARK_CONF_DIR}/spark-env.sh" +fi + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker $(( $i + 1 )) + done +fi diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 6d71ec56914c3..37565ca827980 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -12,6 +12,7 @@ # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT +# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes to be spawned on every slave machine # # Finally, Spark also relies on the following variables, but these can be set # on just the *master* (i.e. in your driver program), and will automatically From aa9134f72a13cdc1fd1a662ec9389724e8e0e35f Mon Sep 17 00:00:00 2001 From: kalpit Date: Fri, 8 Mar 2013 18:28:43 -0800 Subject: [PATCH 413/696] spark instance number must be present in log filename to prevent multiple workers from overriding each other's logs --- bin/spark-daemon.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index d7dc62ab08162..8ee3ec481fe0b 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -94,9 +94,9 @@ if [ "$SPARK_PID_DIR" = "" ]; then fi # some variables -export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$HOSTNAME.log +export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.log export SPARK_ROOT_LOGGER="INFO,DRFA" -log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$HOSTNAME.out +log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid # Set default scheduling priority From 4e2c96538339e627013c8c0c58bd30494cc8d13b Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 28 Mar 2013 17:47:37 -0300 Subject: [PATCH 414/696] Don't use deprecated Application in example As of 2.9.0 extending from Application is not recommended http://www.scala-lang.org/api/2.9.3/index.html#scala.Application --- docs/quick-start.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index 216f7c9cc5800..5c80d2ed3a52e 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -111,14 +111,16 @@ We'll create a very simple Spark job in Scala. So simple, in fact, that it's nam import spark.SparkContext import SparkContext._ -object SimpleJob extends Application { - val logFile = "/var/log/syslog" // Should be some file on your system - val sc = new SparkContext("local", "Simple Job", "$YOUR_SPARK_HOME", - List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar")) - val logData = sc.textFile(logFile, 2).cache() - val numAs = logData.filter(line => line.contains("a")).count() - val numBs = logData.filter(line => line.contains("b")).count() - println("Lines with a: %s, Lines with b: %s".format(numAs, numBs)) +object SimpleJob { + def main(args: Array[String]) { + val logFile = "/var/log/syslog" // Should be some file on your system + val sc = new SparkContext("local", "Simple Job", "$YOUR_SPARK_HOME", + List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar")) + val logData = sc.textFile(logFile, 2).cache() + val numAs = logData.filter(line => line.contains("a")).count() + val numBs = logData.filter(line => line.contains("b")).count() + println("Lines with a: %s, Lines with b: %s".format(numAs, numBs)) + } } {% endhighlight %} From b569b3f20032cb581024f17dc0f45fbfb5f75f43 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Tue, 26 Mar 2013 18:14:04 -0700 Subject: [PATCH 415/696] Move streaming test initialization into 'before' blocks --- .../test/scala/spark/streaming/BasicOperationsSuite.scala | 6 ++++-- .../src/test/scala/spark/streaming/InputStreamsSuite.scala | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 8fce91853c77e..cf2ed8b1d4bef 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -6,10 +6,12 @@ import util.ManualClock class BasicOperationsSuite extends TestSuiteBase { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") - override def framework() = "BasicOperationsSuite" + before { + System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + } + after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 4d33857b25ed5..67dca2ac31215 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -30,12 +30,14 @@ import com.google.common.io.Files class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") - val testPort = 9999 override def checkpointDir = "checkpoint" + before { + System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + } + after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") From e8f3669c639ed51ee7c87864634f55b84fcc228f Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 28 Mar 2013 19:17:39 -0300 Subject: [PATCH 416/696] Update tuning.md Make the example more compilable --- docs/tuning.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tuning.md b/docs/tuning.md index 843380b9a2882..32c7ab86e9991 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -49,7 +49,7 @@ Finally, to register your classes with Kryo, create a public class that extends {% highlight scala %} import com.esotericsoftware.kryo.Kryo -class MyRegistrator extends KryoRegistrator { +class MyRegistrator extends spark.KryoRegistrator { override def registerClasses(kryo: Kryo) { kryo.register(classOf[MyClass1]) kryo.register(classOf[MyClass2]) From bc8ba222ff29d46cd2e3331753c9c4ce681eccb4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 28 Mar 2013 15:42:01 -0700 Subject: [PATCH 417/696] Bump development version to 0.8.0 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- pom.xml | 2 +- project/SparkBuild.scala | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- repl/src/main/scala/spark/repl/SparkILoop.scala | 2 +- streaming/pom.xml | 2 +- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 510cff46695a8..be2e3580917d8 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.1-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index fe9c803728654..08717860a7735 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.1-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index f99d5bb376027..a6aa38a46d757 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -3,8 +3,8 @@ markdown: kramdown # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 0.7.1-SNAPSHOT -SPARK_VERSION_SHORT: 0.7.1 +SPARK_VERSION: 0.8.0-SNAPSHOT +SPARK_VERSION_SHORT: 0.8.0 SCALA_VERSION: 2.9.2 MESOS_VERSION: 0.9.0-incubating SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net diff --git a/examples/pom.xml b/examples/pom.xml index 39cc47c709385..d014089fe4987 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.1-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 08d1fc12e031f..1174b475d3252 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.7.1-SNAPSHOT + 0.8.0-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f378b2398c5b..250211fb0c0d1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -35,7 +35,7 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", - version := "0.7.1-SNAPSHOT", + version := "0.8.0-SNAPSHOT", scalaVersion := "2.9.2", scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dd720e22915eb..fe526a7616c7c 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.1-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index a3e4606edc4d3..0b5e400c3d146 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.1-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index cd7b5128b24df..39b213851fd97 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -200,7 +200,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 0.7.1 + /___/ .__/\_,_/_/ /_/\_\ version 0.8.0-SNAPSHOT /_/ """) import Properties._ diff --git a/streaming/pom.xml b/streaming/pom.xml index ec077e8089116..b0d0cd0ff35ee 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.1-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml From def3d1c84a3e0d1371239e9358294a4b4ad46b9f Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 29 Mar 2013 08:20:35 +0800 Subject: [PATCH 418/696] 1.remove redundant spacing in source code 2.replace get/set functions with val and var defination --- .../scala/spark/scheduler/ActiveJob.scala | 2 +- .../scala/spark/scheduler/DAGScheduler.scala | 4 +- .../scheduler/cluster/ClusterScheduler.scala | 13 ++-- .../cluster/FIFOTaskSetQueuesManager.scala | 13 ++-- .../cluster/FairTaskSetQueuesManager.scala | 65 +++++++++---------- .../scala/spark/scheduler/cluster/Pool.scala | 43 +++++------- .../spark/scheduler/cluster/Schedulable.scala | 21 ++---- .../cluster/SchedulingAlgorithm.scala | 34 +++++----- .../scheduler/cluster/SchedulingMode.scala | 2 +- .../scheduler/cluster/TaskSetManager.scala | 37 ++--------- .../cluster/TaskSetQueuesManager.scala | 1 - 11 files changed, 91 insertions(+), 144 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index b6d3c2c08902b..105eaecb22e5d 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -13,7 +13,7 @@ private[spark] class ActiveJob( val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], val callSite: String, - val listener: JobListener, + val listener: JobListener, val properties: Properties) { val numPartitions = partitions.length diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 717cc27739027..0a64a4f041155 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -241,7 +241,7 @@ class DAGScheduler( partitions: Seq[Int], callSite: String, allowLocal: Boolean, - resultHandler: (Int, U) => Unit, + resultHandler: (Int, U) => Unit, properties: Properties = null) { if (partitions.size == 0) { @@ -263,7 +263,7 @@ class DAGScheduler( func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], callSite: String, - timeout: Long, + timeout: Long, properties: Properties = null) : PartialResult[R] = { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 092b0a0cfc603..be0d480aa079b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -60,7 +60,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var backend: SchedulerBackend = null val mapOutputTracker = SparkEnv.get.mapOutputTracker - + var taskSetQueuesManager: TaskSetQueuesManager = null override def setListener(listener: TaskSchedulerListener) { @@ -131,11 +131,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - def taskFinished(manager: TaskSetManager) { + def taskFinished(manager: TaskSetManager) { this.synchronized { - taskSetQueuesManager.taskFinished(manager) + taskSetQueuesManager.taskFinished(manager) } - } + } /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task @@ -144,7 +144,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) */ 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) { @@ -228,7 +227,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) taskSetToUpdate.get.statusUpdate(tid, state, serializedData) } if (failedExecutor != None) { - listener.executorLost(failedExecutor.get) + listener.executorLost(failedExecutor.get) backend.reviveOffers() } if (taskFailed) { @@ -299,7 +298,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Call listener.executorLost without holding the lock on this to prevent deadlock if (failedExecutor != None) { - listener.executorLost(failedExecutor.get) + listener.executorLost(failedExecutor.get) backend.reviveOffers() } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala index 868b11c8d6c4e..5949ee773ffa3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala @@ -8,26 +8,26 @@ import spark.Logging * A FIFO Implementation of the TaskSetQueuesManager */ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] val tasksetSchedulingAlgorithm = new FIFOSchedulingAlgorithm() - + override def addTaskSetManager(manager: TaskSetManager) { activeTaskSetsQueue += manager } - + override def removeTaskSetManager(manager: TaskSetManager) { activeTaskSetsQueue -= manager } - + override def taskFinished(manager: TaskSetManager) { //do nothing } - + override def removeExecutor(executorId: String, host: String) { activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) } - + override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = { for(manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) @@ -48,5 +48,4 @@ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with } return shouldRevive } - } diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala index 4e26cedfda27b..0609600f35156 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala @@ -14,15 +14,14 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * A Fair Implementation of the TaskSetQueuesManager - * + * * Currently we support minShare,weight for fair scheduler between pools * Within a pool, it supports FIFO or FS * Also, currently we could allocate pools dynamically - * */ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - - val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") + + val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") val poolNameToPool= new HashMap[String, Pool] var pools = new ArrayBuffer[Pool] val poolScheduleAlgorithm = new FairSchedulingAlgorithm() @@ -36,9 +35,9 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with val POOL_DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO val POOL_DEFAULT_MINIMUM_SHARES = 2 val POOL_DEFAULT_WEIGHT = 1 - + loadPoolProperties() - + def loadPoolProperties() { //first check if the file exists val file = new File(schedulerAllocFile) @@ -51,26 +50,25 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE var minShares = POOL_DEFAULT_MINIMUM_SHARES var weight = POOL_DEFAULT_WEIGHT - - + val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text if( xmlSchedulingMode != "") { - try + try { schedulingMode = SchedulingMode.withName(xmlSchedulingMode) } catch{ - case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") } } - + val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text if(xmlMinShares != "") { minShares = xmlMinShares.toInt } - + val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text if(xmlWeight != "") { @@ -84,15 +82,15 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with } } - if(!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) + if(!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) { val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) pools += pool poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } + } } - + override def addTaskSetManager(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME if(manager.taskSet.properties != null) @@ -100,19 +98,19 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) if(!poolNameToPool.contains(poolName)) { - //we will create a new pool that user has configured in app,but not contained in xml file + //we will create a new pool that user has configured in app instead of being defined in xml file val pool = new Pool(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) pools += pool poolNameToPool(poolName) = pool - logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) } } poolNameToPool(poolName).addTaskSetManager(manager) - logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) + logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) } - + override def removeTaskSetManager(manager: TaskSetManager) { - + var poolName = POOL_DEFAULT_POOL_NAME if(manager.taskSet.properties != null) { @@ -121,10 +119,9 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id,poolName)) val pool = poolNameToPool(poolName) pool.removeTaskSetManager(manager) - pool.setRunningTasks(pool.getRunningTasks() - manager.getRunningTasks()) - + pool.runningTasks -= manager.runningTasks } - + override def taskFinished(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME if(manager.taskSet.properties != null) @@ -132,40 +129,40 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) } val pool = poolNameToPool(poolName) - pool.setRunningTasks(pool.getRunningTasks() - 1) - manager.setRunningTasks(manager.getRunningTasks() - 1) + pool.runningTasks -= 1 + manager.runningTasks -=1 } - + override def removeExecutor(executorId: String, host: String) { for (pool <- pools) { - pool.removeExecutor(executorId,host) - } + pool.removeExecutor(executorId,host) + } } - + override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = { val sortedPools = pools.sortWith(poolScheduleAlgorithm.comparator) for(pool <- sortedPools) { - logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format(pool.poolName,pool.activeTaskSetsQueue.length,pool.getMinShare(),pool.getRunningTasks())) + logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format(pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) } for (pool <- sortedPools) { val task = pool.receiveOffer(execId,host,avaiableCpus) if(task != None) { - pool.setRunningTasks(pool.getRunningTasks() + 1) + pool.runningTasks += 1 return task } } return None } - - override def checkSpeculatableTasks(): Boolean = + + override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for (pool <- pools) + for (pool <- pools) { shouldRevive |= pool.checkSpeculatableTasks() } diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 68e1d2a75afed..8fdca5d2b450e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -7,13 +7,21 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * An Schedulable entity that represent collection of TaskSetManager */ -private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,val minShare:Int, val weight:Int) extends Schedulable with Logging { - +private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMode, initMinShare:Int, initWeight:Int) extends Schedulable with Logging +{ + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] - var numRunningTasks: Int = 0 - var taskSetSchedulingAlgorithm: SchedulingAlgorithm = + + var weight = initWeight + var minShare = initMinShare + var runningTasks = 0 + + val priority = 0 + val stageId = 0 + + var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { - schedulingMode match + schedulingMode match { case SchedulingMode.FAIR => val schedule = new FairSchedulingAlgorithm() @@ -23,26 +31,6 @@ private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,v schedule } } - - override def getMinShare():Int = - { - return minShare - } - - override def getRunningTasks():Int = - { - return numRunningTasks - } - - def setRunningTasks(taskNum : Int) - { - numRunningTasks = taskNum - } - - override def getWeight(): Int = - { - return weight - } def addTaskSetManager(manager:TaskSetManager) { @@ -74,15 +62,14 @@ private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,v val sortedActiveTasksSetQueue = activeTaskSetsQueue.sortWith(taskSetSchedulingAlgorithm.comparator) for(manager <- sortedActiveTasksSetQueue) { - - logDebug("taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format(manager.taskSet.id,manager.numTasks,manager.getMinShare(),manager.getWeight(),manager.getRunningTasks())) + logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format(poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) } for(manager <- sortedActiveTasksSetQueue) { val task = manager.slaveOffer(execId,host,availableCpus) if (task != None) { - manager.setRunningTasks(manager.getRunningTasks() + 1) + manager.runningTasks += 1 return task } } diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 837f9c49838bf..6f4f104f42cfe 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,21 +1,12 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer - /** - * An interface for schedulable entities, there are two type Schedulable entities(Pools and TaskSetManagers) + * An interface for schedulable entities, there are two type Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { - - def getMinShare(): Int - def getRunningTasks(): Int - def getPriority(): Int = - { - return 0 - } - def getWeight(): Int - def getStageId(): Int = - { - return 0 - } + def weight:Int + def minShare:Int + def runningTasks:Int + def priority:Int + def stageId:Int } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index f8919e7374ade..2f8123587f69c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -1,7 +1,7 @@ package spark.scheduler.cluster /** - * An interface for sort algorithm + * An interface for sort algorithm * FIFO: FIFO algorithm for TaskSetManagers * FS: FS algorithm for Pools, and FIFO or FS for TaskSetManagers */ @@ -13,13 +13,13 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { - val priority1 = s1.getPriority() - val priority2 = s2.getPriority() + val priority1 = s1.priority + val priority2 = s2.priority var res = Math.signum(priority1 - priority2) if (res == 0) { - val stageId1 = s1.getStageId() - val stageId2 = s2.getStageId() + val stageId1 = s1.stageId + val stageId2 = s2.stageId res = Math.signum(stageId1 - stageId2) } if (res < 0) @@ -29,7 +29,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm else { return false - } + } } } @@ -37,16 +37,18 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { def comparator(s1: Schedulable, s2:Schedulable): Boolean = { - val minShare1 = s1.getMinShare() - val minShare2 = s2.getMinShare() - val s1Needy = s1.getRunningTasks() < minShare1 - val s2Needy = s2.getRunningTasks() < minShare2 - val minShareRatio1 = s1.getRunningTasks().toDouble / Math.max(minShare1,1.0).toDouble - val minShareRatio2 = s2.getRunningTasks().toDouble / Math.max(minShare2,1.0).toDouble - val taskToWeightRatio1 = s1.getRunningTasks().toDouble / s1.getWeight().toDouble - val taskToWeightRatio2 = s2.getRunningTasks().toDouble / s2.getWeight().toDouble + val minShare1 = s1.minShare + val minShare2 = s2.minShare + val runningTasks1 = s1.runningTasks + val runningTasks2 = s2.runningTasks + val s1Needy = runningTasks1 < minShare1 + val s2Needy = runningTasks2 < minShare2 + val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1,1.0).toDouble + val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2,1.0).toDouble + val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble + val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true - + if(s1Needy && !s2Needy) { res = true @@ -57,7 +59,7 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm } else if (s1Needy && s2Needy) { - res = minShareRatio1 <= minShareRatio2 + res = minShareRatio1 <= minShareRatio2 } else { diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6be4f3cd849b2..480af2c1a30e3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,4 +1,4 @@ -package spark.scheduler.cluster +package spark.scheduler.cluster object SchedulingMode extends Enumeration("FAIR","FIFO") { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 064593f48641b..ddc4fa6642842 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -29,7 +29,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val MAX_TASK_FAILURES = 4 val TASKSET_MINIMUM_SHARES = 1 - val TASKSET_WEIGHT = 1 // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble @@ -38,7 +37,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Serializer for closures and tasks. val ser = SparkEnv.get.closureSerializer.newInstance() + var weight = TASKSET_WEIGHT + var minShare = TASKSET_MINIMUM_SHARES + var runningTasks = 0 val priority = taskSet.priority + val stageId = taskSet.stageId + val tasks = taskSet.tasks val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) @@ -46,7 +50,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val numFailures = new Array[Int](numTasks) val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 - var numRunningTasks =0; // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -100,36 +103,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } - override def getMinShare(): Int = - { - return TASKSET_MINIMUM_SHARES - } - - override def getRunningTasks(): Int = - { - return numRunningTasks - } - - def setRunningTasks(taskNum :Int) - { - numRunningTasks = taskNum - } - - override def getPriority(): Int = - { - return priority - } - - override def getWeight(): Int = - { - return TASKSET_WEIGHT - } - - override def getStageId(): Int = - { - return taskSet.stageId - } - // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala index c117ee7a85164..86971d47e6147 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala @@ -5,7 +5,6 @@ import scala.collection.mutable.ArrayBuffer /** * An interface for managing TaskSet queue/s that allows plugging different policy for * offering tasks to resources - * */ private[spark] trait TaskSetQueuesManager { def addTaskSetManager(manager: TaskSetManager): Unit From 1a28f92711cb59ad99bc9e3dd84a5990181e572b Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 29 Mar 2013 08:34:28 +0800 Subject: [PATCH 419/696] change some typo and some spacing --- core/src/main/scala/spark/SparkContext.scala | 12 ++++++------ .../main/scala/spark/scheduler/DAGScheduler.scala | 4 ++-- core/src/main/scala/spark/scheduler/Stage.scala | 5 ++--- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 6eccb501c7e94..ed5f68637982d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -74,7 +74,7 @@ class SparkContext( if (System.getProperty("spark.driver.port") == null) { System.setProperty("spark.driver.port", "0") } - + //Set the default task scheduler if (System.getProperty("spark.cluster.taskscheduler") == null) { System.setProperty("spark.cluster.taskscheduler", "spark.scheduler.cluster.FIFOTaskSetQueuesManager") @@ -119,7 +119,7 @@ class SparkContext( } } executorEnvs ++= environment - + // Create and start the scheduler private var taskScheduler: TaskScheduler = { // Regular expression used for local[N] master format @@ -216,14 +216,14 @@ class SparkContext( } private[spark] var checkpointDir: Option[String] = None - + // Thread Local variable that can be used by users to pass information down the stack private val localProperties = new DynamicVariable[Properties](null) - + def initLocalProperties() { localProperties.value = new Properties() } - + def addLocalProperties(key: String, value: String) { if(localProperties.value == null) { localProperties.value = new Properties() @@ -673,7 +673,7 @@ class SparkContext( val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter) runJob[T, U](rdd, processFunc, 0 until rdd.partitions.size, false, resultHandler) } - + /** * Run a job that can return approximate results. */ diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 0a64a4f041155..abc24c027090e 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -333,7 +333,7 @@ class DAGScheduler( submitStage(stage) } } - + /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. @@ -720,7 +720,7 @@ class DAGScheduler( sizeBefore = shuffleToMapStage.size shuffleToMapStage.clearOldValues(cleanupTime) logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size) - + sizeBefore = pendingTasks.size pendingTasks.clearOldValues(cleanupTime) logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index bc54cd601d788..7fc9e13fd9dc4 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -5,7 +5,6 @@ import java.net.URI import spark._ import spark.storage.BlockManagerId - /** * A stage is a set of independent tasks all computing the same function that need to run as part * of a Spark job, where all the tasks have the same shuffle dependencies. Each DAG of tasks run @@ -27,7 +26,7 @@ private[spark] class Stage( val parents: List[Stage], val priority: Int) extends Logging { - + val isShuffleMap = shuffleDep != None val numPartitions = rdd.partitions.size val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) @@ -61,7 +60,7 @@ private[spark] class Stage( numAvailableOutputs -= 1 } } - + def removeOutputsOnExecutor(execId: String) { var becameUnavailable = false for (partition <- 0 until numPartitions) { From 8bcdc6400516d3fc7d221c63d5935fa6fe6af24b Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sat, 30 Mar 2013 22:09:52 -0700 Subject: [PATCH 420/696] Fixed broken filter in getWritableClass[T] --- core/src/main/scala/spark/SequenceFileRDDFunctions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 6b4a11d6d3f9e..b68609090f874 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -46,7 +46,7 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla // from T to Writable. Since we have two apply methods we filter out the one which // is of the form "java.lang.Object apply(java.lang.Object)" implicitly[T => Writable].getClass.getDeclaredMethods().filter( - m => m.getReturnType().toString != "java.lang.Object" && + m => m.getReturnType().toString != "class java.lang.Object" && m.getName() == "apply")(0).getReturnType } From e215f67923035dcf5515946a4236a6ef36c5b309 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sun, 31 Mar 2013 08:00:13 -0700 Subject: [PATCH 421/696] Correct sense of 'filter out' in comment. --- .../spark/SequenceFileRDDFunctions.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index b68609090f874..518034e07bf17 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -36,15 +36,15 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla self: RDD[(K, V)]) extends Logging with Serializable { - + private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = { val c = { - if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { + if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { classManifest[T].erasure } else { // We get the type of the Writable class by looking at the apply method which converts // from T to Writable. Since we have two apply methods we filter out the one which - // is of the form "java.lang.Object apply(java.lang.Object)" + // is not of the form "java.lang.Object apply(java.lang.Object)" implicitly[T => Writable].getClass.getDeclaredMethods().filter( m => m.getReturnType().toString != "class java.lang.Object" && m.getName() == "apply")(0).getReturnType @@ -69,17 +69,17 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla val valueClass = getWritableClass[V] val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) - - logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) + + logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] if (!convertKey && !convertValue) { - self.saveAsHadoopFile(path, keyClass, valueClass, format) + self.saveAsHadoopFile(path, keyClass, valueClass, format) } else if (!convertKey && convertValue) { - self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) } else if (convertKey && !convertValue) { - self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(path, keyClass, valueClass, format) } else if (convertKey && convertValue) { - self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) - } + self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) + } } } From 2b373dd07a7b3f2906607d910c869e3290ca9d05 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 2 Apr 2013 12:11:14 +0800 Subject: [PATCH 422/696] add properties default value null to fix sbt/sbt test errors --- core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 79588891e74a1..6f4e5cd83e397 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,7 @@ private[spark] case class JobSubmitted( partitions: Array[Int], allowLocal: Boolean, callSite: String, - listener: JobListener, properties: Properties) + listener: JobListener, properties: Properties = null) extends DAGSchedulerEvent private[spark] case class CompletionEvent( From b5e60c3253748cf87a76c713ae3388dce1b03107 Mon Sep 17 00:00:00 2001 From: Erik van oosten Date: Tue, 2 Apr 2013 15:25:22 +0300 Subject: [PATCH 423/696] Corrected order of CountMinSketchMonoid arguments --- .../scala/spark/streaming/examples/TwitterAlgebirdCMS.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 483aae452b05e..a9642100e3d48 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -49,7 +49,7 @@ object TwitterAlgebirdCMS { val users = stream.map(status => status.getUser.getId) - val cms = new CountMinSketchMonoid(DELTA, EPS, SEED, PERC) + val cms = new CountMinSketchMonoid(EPS, DELTA, SEED, PERC) var globalCMS = cms.zero val mm = new MapMonoid[Long, Int]() var globalExact = Map[Long, Int]() From 20604001e280a445d7c25bac6eb31b1f0512c20f Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sun, 10 Mar 2013 06:16:19 +0000 Subject: [PATCH 424/696] Fix argv handling in Python transitive closure example --- python/examples/transitive_closure.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py index 73f7f8fbafcc5..7f85a1008e9f4 100644 --- a/python/examples/transitive_closure.py +++ b/python/examples/transitive_closure.py @@ -24,7 +24,7 @@ def generateGraph(): "Usage: PythonTC []" exit(-1) sc = SparkContext(sys.argv[1], "PythonTC") - slices = sys.argv[2] if len(sys.argv) > 2 else 2 + slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 tc = sc.parallelize(generateGraph(), slices).cache() # Linear transitive closure: each round grows paths by one edge, From 9a731f5a6dd56f5bc9eda3bb3177f7bbbb251c31 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Tue, 2 Apr 2013 11:58:40 -0700 Subject: [PATCH 425/696] Fix Python saveAsTextFile doctest to not expect order to be preserved --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 172ed85fab926..a9fec17a9da69 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -399,7 +399,7 @@ def saveAsTextFile(self, path): >>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name) >>> from fileinput import input >>> from glob import glob - >>> ''.join(input(glob(tempFile.name + "/part-0000*"))) + >>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*")))) '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' """ def func(split, iterator): From 0f54344fd858acb327174cc976a07038a3aadf24 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 3 Apr 2013 13:15:34 +0200 Subject: [PATCH 426/696] Bumping Algebird version in examples now that it supports JDK 1.6 --- examples/pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 39cc47c709385..0537404040b4d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -22,7 +22,7 @@ com.twitter algebird-core_2.9.2 - 0.1.8 + 0.1.11 org.scalatest diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d44bf3b5e3937..082e2b985d2da 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -157,7 +157,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8") + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From 9b68ceaa26ffe0cf0692781dd010e69f1d9633e2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 3 Apr 2013 14:29:46 -0700 Subject: [PATCH 427/696] SPARK-724 - Have Akka logging enabled by default for standalone daemons See the JIRA for more details. I was only able to test the bash version (don't have Windows) so maybe check over that the syntax is correct there. --- run | 1 + run2.cmd | 1 + 2 files changed, 2 insertions(+) diff --git a/run b/run index 2c29cc4a6641c..ceb6e5817329f 100755 --- a/run +++ b/run @@ -22,6 +22,7 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} + SPARK_DAEMON_JAVA_OPTS+="-Dspark.akka.logLifecycleEvents=true" SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default fi diff --git a/run2.cmd b/run2.cmd index cb20a4b7a2f9a..cfbb35ddca991 100644 --- a/run2.cmd +++ b/run2.cmd @@ -21,6 +21,7 @@ set RUNNING_DAEMON=0 if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1 if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1 if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m +set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% if "%RUNNING_DAEMON%"=="1" set SPARK_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% From 5555811bd54ddb84bce11d4ab04b1f818c221a14 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Thu, 4 Apr 2013 13:26:45 -0700 Subject: [PATCH 428/696] Update build to Scala 2.9.3 --- project/SparkBuild.scala | 8 ++++---- project/plugins.sbt | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f378b2398c5b..b54fa1ebf845c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -36,7 +36,7 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", version := "0.7.1-SNAPSHOT", - scalaVersion := "2.9.2", + scalaVersion := "2.9.3", scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, @@ -93,8 +93,8 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", - "org.scalatest" %% "scalatest" % "1.8" % "test", - "org.scalacheck" %% "scalacheck" % "1.9" % "test", + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", "com.novocode" % "junit-interface" % "0.8" % "test", "org.easymock" % "easymock" % "3.1" % "test" ), @@ -140,7 +140,7 @@ object SparkBuild extends Build { "colt" % "colt" % "1.2.0", "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", - "cc.spray" %% "spray-json" % "1.1.1", + "cc.spray" % "spray-json_2.9.2" % "1.1.1", "org.apache.mesos" % "mesos" % "0.9.0-incubating" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } diff --git a/project/plugins.sbt b/project/plugins.sbt index 4d0e696a11358..9cea0b5ee49fa 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -10,7 +10,7 @@ addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0") -addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2") +addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") // For Sonatype publishing //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) From ff2130a0ad17388036b66fcdf2b1848e208fa0f8 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 6 Apr 2013 00:35:50 -0700 Subject: [PATCH 429/696] Retry failed ssh commands. This is especially useful during system startup when the hosts may not have yet come on-line but can be useful at other points for people with flakey connections --- ec2/retry_decorator.py | 44 ++++++++++++++++++++++++++++++++++++++++++ ec2/spark_ec2.py | 2 ++ 2 files changed, 46 insertions(+) create mode 100644 ec2/retry_decorator.py diff --git a/ec2/retry_decorator.py b/ec2/retry_decorator.py new file mode 100644 index 0000000000000..1a2f79ae844f7 --- /dev/null +++ b/ec2/retry_decorator.py @@ -0,0 +1,44 @@ +import time +from functools import wraps + +def retry(ExceptionToCheck, tries=4, delay=3, backoff=2, logger=None): + """Retry calling the decorated function using an exponential backoff. + + http://www.saltycrane.com/blog/2009/11/trying-out-retry-decorator-python/ + original from: http://wiki.python.org/moin/PythonDecoratorLibrary#Retry + + :param ExceptionToCheck: the exception to check. may be a tuple of + exceptions to check + :type ExceptionToCheck: Exception or tuple + :param tries: number of times to try (not retry) before giving up + :type tries: int + :param delay: initial delay between retries in seconds + :type delay: int + :param backoff: backoff multiplier e.g. value of 2 will double the delay + each retry + :type backoff: int + :param logger: logger to use. If None, print + :type logger: logging.Logger instance + """ + def deco_retry(f): + + @wraps(f) + def f_retry(*args, **kwargs): + mtries, mdelay = tries, delay + while mtries > 1: + try: + return f(*args, **kwargs) + except ExceptionToCheck, e: + msg = "%s, Retrying in %d seconds..." % (str(e), mdelay) + if logger: + logger.warning(msg) + else: + print msg + time.sleep(mdelay) + mtries -= 1 + mdelay *= backoff + return f(*args, **kwargs) + + return f_retry # true decorator + + return deco_retry diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 571d27fde66ed..aa2d360fbbcaf 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -22,6 +22,7 @@ import logging import os import random +from retry_decorator import retry import shutil import subprocess import sys @@ -541,6 +542,7 @@ def scp(host, opts, local_file, dest_file): # Run a command on a host through ssh, throwing an exception if ssh fails +@retry(subprocess.CalledProcessError, tries=3, delay=30) def ssh(host, opts, command): subprocess.check_call( "ssh -t -o StrictHostKeyChecking=no -i %s %s@%s '%s'" % From df47b40b764e25cbd10ce49d7152e1d33f51a263 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Wed, 20 Feb 2013 11:51:13 +0800 Subject: [PATCH 430/696] Shuffle Performance fix: Use netty embeded OIO file server instead of ConnectionManager Shuffle Performance Optimization: do not send 0-byte block requests to reduce network messages change reference from io.Source to scala.io.Source to avoid looking into io.netty package Signed-off-by: shane-huang --- .../java/spark/network/netty/FileClient.java | 89 ++++++ .../netty/FileClientChannelInitializer.java | 29 ++ .../network/netty/FileClientHandler.java | 38 +++ .../java/spark/network/netty/FileServer.java | 59 ++++ .../netty/FileServerChannelInitializer.java | 33 +++ .../network/netty/FileServerHandler.java | 68 +++++ .../spark/network/netty/PathResolver.java | 12 + .../spark/network/netty/FileHeader.scala | 57 ++++ .../spark/network/netty/ShuffleCopier.scala | 88 ++++++ .../spark/network/netty/ShuffleSender.scala | 50 ++++ .../scala/spark/storage/BlockManager.scala | 272 ++++++++++++++---- .../main/scala/spark/storage/DiskStore.scala | 51 +++- project/SparkBuild.scala | 3 +- .../spark/streaming/util/RawTextSender.scala | 2 +- 14 files changed, 795 insertions(+), 56 deletions(-) create mode 100644 core/src/main/java/spark/network/netty/FileClient.java create mode 100644 core/src/main/java/spark/network/netty/FileClientChannelInitializer.java create mode 100644 core/src/main/java/spark/network/netty/FileClientHandler.java create mode 100644 core/src/main/java/spark/network/netty/FileServer.java create mode 100644 core/src/main/java/spark/network/netty/FileServerChannelInitializer.java create mode 100644 core/src/main/java/spark/network/netty/FileServerHandler.java create mode 100755 core/src/main/java/spark/network/netty/PathResolver.java create mode 100644 core/src/main/scala/spark/network/netty/FileHeader.scala create mode 100644 core/src/main/scala/spark/network/netty/ShuffleCopier.scala create mode 100644 core/src/main/scala/spark/network/netty/ShuffleSender.scala diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java new file mode 100644 index 0000000000000..d0c5081dd2a53 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -0,0 +1,89 @@ +package spark.network.netty; + +import io.netty.bootstrap.Bootstrap; +import io.netty.channel.AbstractChannel; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelOption; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.channel.oio.OioEventLoopGroup; +import io.netty.channel.socket.oio.OioSocketChannel; + +import java.util.Arrays; + +public class FileClient { + + private FileClientHandler handler = null; + private Channel channel = null; + private Bootstrap bootstrap = null; + + public FileClient(FileClientHandler handler){ + this.handler = handler; + } + + public void init(){ + bootstrap = new Bootstrap(); + bootstrap.group(new OioEventLoopGroup()) + .channel(OioSocketChannel.class) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.TCP_NODELAY, true) + .handler(new FileClientChannelInitializer(handler)); + } + + public static final class ChannelCloseListener implements ChannelFutureListener { + private FileClient fc = null; + public ChannelCloseListener(FileClient fc){ + this.fc = fc; + } + @Override + public void operationComplete(ChannelFuture future) { + if (fc.bootstrap!=null){ + fc.bootstrap.shutdown(); + fc.bootstrap = null; + } + } + } + + public void connect(String host, int port){ + try { + + // Start the connection attempt. + channel = bootstrap.connect(host, port).sync().channel(); + // ChannelFuture cf = channel.closeFuture(); + //cf.addListener(new ChannelCloseListener(this)); + } catch (InterruptedException e) { + close(); + } + } + + public void waitForClose(){ + try { + channel.closeFuture().sync(); + } catch (InterruptedException e){ + e.printStackTrace(); + } + } + + public void sendRequest(String file){ + //assert(file == null); + //assert(channel == null); + channel.write(file+"\r\n"); + } + + public void close(){ + if(channel != null) { + channel.close(); + channel = null; + } + if ( bootstrap!=null) { + bootstrap.shutdown(); + bootstrap = null; + } + } + + +} + + diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java new file mode 100644 index 0000000000000..50e5704619a60 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java @@ -0,0 +1,29 @@ +package spark.network.netty; + +import io.netty.buffer.BufType; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.handler.codec.string.StringEncoder; +import io.netty.util.CharsetUtil; + +import io.netty.handler.logging.LoggingHandler; +import io.netty.handler.logging.LogLevel; + +public class FileClientChannelInitializer extends + ChannelInitializer { + + private FileClientHandler fhandler; + + public FileClientChannelInitializer(FileClientHandler handler) { + fhandler = handler; + } + + @Override + public void initChannel(SocketChannel channel) { + // file no more than 2G + channel.pipeline() + .addLast("encoder", new StringEncoder(BufType.BYTE)) + .addLast("handler", fhandler); + } +} diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java new file mode 100644 index 0000000000000..911c8b32b550f --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -0,0 +1,38 @@ +package spark.network.netty; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundByteHandlerAdapter; +import io.netty.util.CharsetUtil; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.logging.Logger; + +public abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { + + private FileHeader currentHeader = null; + + public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header); + + @Override + public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) { + // Use direct buffer if possible. + return ctx.alloc().ioBuffer(); + } + + @Override + public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) { + // get header + if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) { + currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE())); + } + // get file + if(in.readableBytes() >= currentHeader.fileLen()){ + handle(ctx,in,currentHeader); + currentHeader = null; + ctx.close(); + } + } + +} + diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java new file mode 100644 index 0000000000000..729e45f0a13a8 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -0,0 +1,59 @@ +package spark.network.netty; + +import java.io.File; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelOption; +import io.netty.channel.Channel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.oio.OioEventLoopGroup; +import io.netty.channel.socket.oio.OioServerSocketChannel; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; + +/** + * Server that accept the path of a file an echo back its content. + */ +public class FileServer { + + private ServerBootstrap bootstrap = null; + private Channel channel = null; + private PathResolver pResolver; + + public FileServer(PathResolver pResolver){ + this.pResolver = pResolver; + } + + public void run(int port) { + // Configure the server. + bootstrap = new ServerBootstrap(); + try { + bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) + .channel(OioServerSocketChannel.class) + .option(ChannelOption.SO_BACKLOG, 100) + .option(ChannelOption.SO_RCVBUF, 1500) + .childHandler(new FileServerChannelInitializer(pResolver)); + // Start the server. + channel = bootstrap.bind(port).sync().channel(); + channel.closeFuture().sync(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } finally{ + bootstrap.shutdown(); + } + } + + public void stop(){ + if (channel!=null){ + channel.close(); + } + if (bootstrap != null){ + bootstrap.shutdown(); + bootstrap = null; + } + } +} + + diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java new file mode 100644 index 0000000000000..9d0618ff1c492 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java @@ -0,0 +1,33 @@ +package spark.network.netty; + +import java.io.File; +import io.netty.buffer.BufType; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.string.StringDecoder; +import io.netty.handler.codec.string.StringEncoder; +import io.netty.handler.codec.DelimiterBasedFrameDecoder; +import io.netty.handler.codec.Delimiters; +import io.netty.util.CharsetUtil; +import io.netty.handler.logging.LoggingHandler; +import io.netty.handler.logging.LogLevel; + +public class FileServerChannelInitializer extends + ChannelInitializer { + + PathResolver pResolver; + + public FileServerChannelInitializer(PathResolver pResolver) { + this.pResolver = pResolver; + } + + @Override + public void initChannel(SocketChannel channel) { + channel.pipeline() + .addLast("framer", new DelimiterBasedFrameDecoder( + 8192, Delimiters.lineDelimiter())) + .addLast("strDecoder", new StringDecoder()) + .addLast("handler", new FileServerHandler(pResolver)); + + } +} diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java new file mode 100644 index 0000000000000..e1083e87a26de --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileServerHandler.java @@ -0,0 +1,68 @@ +package spark.network.netty; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundMessageHandlerAdapter; +import io.netty.channel.DefaultFileRegion; +import io.netty.handler.stream.ChunkedFile; +import java.io.File; +import java.io.FileInputStream; + +public class FileServerHandler extends + ChannelInboundMessageHandlerAdapter { + + PathResolver pResolver; + + public FileServerHandler(PathResolver pResolver){ + this.pResolver = pResolver; + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, String blockId) { + String path = pResolver.getAbsolutePath(blockId); + // if getFilePath returns null, close the channel + if (path == null) { + //ctx.close(); + return; + } + File file = new File(path); + if (file.exists()) { + if (!file.isFile()) { + //logger.info("Not a file : " + file.getAbsolutePath()); + ctx.write(new FileHeader(0, blockId).buffer()); + ctx.flush(); + return; + } + long length = file.length(); + if (length > Integer.MAX_VALUE || length <= 0 ) { + //logger.info("too large file : " + file.getAbsolutePath() + " of size "+ length); + ctx.write(new FileHeader(0, blockId).buffer()); + ctx.flush(); + return; + } + int len = new Long(length).intValue(); + //logger.info("Sending block "+blockId+" filelen = "+len); + //logger.info("header = "+ (new FileHeader(len, blockId)).buffer()); + ctx.write((new FileHeader(len, blockId)).buffer()); + try { + ctx.sendFile(new DefaultFileRegion(new FileInputStream(file) + .getChannel(), 0, file.length())); + } catch (Exception e) { + // TODO Auto-generated catch block + //logger.warning("Exception when sending file : " + //+ file.getAbsolutePath()); + e.printStackTrace(); + } + } else { + //logger.warning("File not found: " + file.getAbsolutePath()); + ctx.write(new FileHeader(0, blockId).buffer()); + } + ctx.flush(); + } + + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + cause.printStackTrace(); + ctx.close(); + } +} diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java new file mode 100755 index 0000000000000..5d5eda006eb19 --- /dev/null +++ b/core/src/main/java/spark/network/netty/PathResolver.java @@ -0,0 +1,12 @@ +package spark.network.netty; + +public interface PathResolver { + /** + * Get the absolute path of the file + * + * @param fileId + * @return the absolute path of file + */ + public String getAbsolutePath(String fileId); + +} diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/spark/network/netty/FileHeader.scala new file mode 100644 index 0000000000000..aed4254234e39 --- /dev/null +++ b/core/src/main/scala/spark/network/netty/FileHeader.scala @@ -0,0 +1,57 @@ +package spark.network.netty + +import io.netty.buffer._ + +import spark.Logging + +private[spark] class FileHeader ( + val fileLen: Int, + val blockId: String) extends Logging { + + lazy val buffer = { + val buf = Unpooled.buffer() + buf.capacity(FileHeader.HEADER_SIZE) + buf.writeInt(fileLen) + buf.writeInt(blockId.length) + blockId.foreach((x: Char) => buf.writeByte(x)) + //padding the rest of header + if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { + buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) + } else { + throw new Exception("too long header " + buf.readableBytes) + logInfo("too long header") + } + buf + } + +} + +private[spark] object FileHeader { + + val HEADER_SIZE = 40 + + def getFileLenOffset = 0 + def getFileLenSize = Integer.SIZE/8 + + def create(buf: ByteBuf): FileHeader = { + val length = buf.readInt + val idLength = buf.readInt + val idBuilder = new StringBuilder(idLength) + for (i <- 1 to idLength) { + idBuilder += buf.readByte().asInstanceOf[Char] + } + val blockId = idBuilder.toString() + new FileHeader(length, blockId) + } + + + def main (args:Array[String]){ + + val header = new FileHeader(25,"block_0"); + val buf = header.buffer; + val newheader = FileHeader.create(buf); + System.out.println("id="+newheader.blockId+",size="+newheader.fileLen) + + } +} + diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala new file mode 100644 index 0000000000000..d8d35bfeecc0e --- /dev/null +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -0,0 +1,88 @@ +package spark.network.netty + +import io.netty.buffer.ByteBuf +import io.netty.channel.ChannelHandlerContext +import io.netty.channel.ChannelInboundByteHandlerAdapter +import io.netty.util.CharsetUtil + +import java.util.concurrent.atomic.AtomicInteger +import java.util.logging.Logger +import spark.Logging +import spark.network.ConnectionManagerId +import java.util.concurrent.Executors + +private[spark] class ShuffleCopier extends Logging { + + def getBlock(cmId: ConnectionManagerId, + blockId: String, + resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + + val handler = new ShuffleClientHandler(resultCollectCallback) + val fc = new FileClient(handler) + fc.init() + fc.connect(cmId.host, cmId.port) + fc.sendRequest(blockId) + fc.waitForClose() + fc.close() + } + + def getBlocks(cmId: ConnectionManagerId, + blocks: Seq[(String, Long)], + resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + + blocks.map { + case(blockId,size) => { + getBlock(cmId,blockId,resultCollectCallback) + } + } + } +} + +private[spark] class ShuffleClientHandler(val resultCollectCallBack: (String, Long, ByteBuf) => Unit ) extends FileClientHandler with Logging { + + def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { + logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); + resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) + } +} + +private[spark] object ShuffleCopier extends Logging { + + def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) = { + logInfo("File: " + blockId + " content is : \" " + + content.toString(CharsetUtil.UTF_8) + "\"") + } + + def runGetBlock(host:String, port:Int, file:String){ + val handler = new ShuffleClientHandler(echoResultCollectCallBack) + val fc = new FileClient(handler) + fc.init(); + fc.connect(host, port) + fc.sendRequest(file) + fc.waitForClose(); + fc.close() + } + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: ShuffleCopier ") + System.exit(1) + } + val host = args(0) + val port = args(1).toInt + val file = args(2) + val threads = if (args.length>3) args(3).toInt else 10 + + val copiers = Executors.newFixedThreadPool(80) + for (i <- Range(0,threads)){ + val runnable = new Runnable() { + def run() { + runGetBlock(host,port,file) + } + } + copiers.execute(runnable) + } + copiers.shutdown + } + +} diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala new file mode 100644 index 0000000000000..c1986812e93e8 --- /dev/null +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -0,0 +1,50 @@ +package spark.network.netty + +import spark.Logging +import java.io.File + + +private[spark] class ShuffleSender(val port: Int, val pResolver:PathResolver) extends Logging { + val server = new FileServer(pResolver) + + Runtime.getRuntime().addShutdownHook( + new Thread() { + override def run() { + server.stop() + } + } + ) + + def start() { + server.run(port) + } +} + +private[spark] object ShuffleSender { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: ShuffleSender ") + System.exit(1) + } + val port = args(0).toInt + val subDirsPerLocalDir = args(1).toInt + val localDirs = args.drop(2) map {new File(_)} + val pResovler = new PathResolver { + def getAbsolutePath(blockId:String):String = { + if (!blockId.startsWith("shuffle_")) { + throw new Exception("Block " + blockId + " is not a shuffle block") + } + // Figure out which local directory it hashes to, and which subdirectory in that + val hash = math.abs(blockId.hashCode) + val dirId = hash % localDirs.length + val subDirId = (hash / localDirs.length) % subDirsPerLocalDir + val subDir = new File(localDirs(dirId), "%02x".format(subDirId)) + val file = new File(subDir, blockId) + return file.getAbsolutePath + } + } + val sender = new ShuffleSender(port, pResovler) + + sender.start() + } +} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 210061e9726b4..b8b68d4283a61 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -23,6 +23,8 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer +import spark.network.netty.ShuffleCopier +import io.netty.buffer.ByteBuf private[spark] case class BlockException(blockId: String, message: String, ex: Exception = null) @@ -467,6 +469,21 @@ class BlockManager( getLocal(blockId).orElse(getRemote(blockId)) } + /** + * A request to fetch one or more blocks, complete with their sizes + */ + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + /** + * A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + * the block (since we want all deserializaton to happen in the calling thread); can also + * represent a fetch failure if size == -1. + */ + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } /** * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined @@ -475,7 +492,12 @@ class BlockManager( */ def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]) : BlockFetcherIterator = { - return new BlockFetcherIterator(this, blocksByAddress) + + if(System.getProperty("spark.shuffle.use.netty", "false").toBoolean){ + return new NettyBlockFetcherIterator(this, blocksByAddress) + } else { + return new BlockFetcherIterator(this, blocksByAddress) + } } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -908,7 +930,7 @@ class BlockFetcherIterator( if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") } - val totalBlocks = blocksByAddress.map(_._2.size).sum + var totalBlocks = blocksByAddress.map(_._2.size).sum logDebug("Getting " + totalBlocks + " blocks") var startTime = System.currentTimeMillis val localBlockIds = new ArrayBuffer[String]() @@ -974,68 +996,83 @@ class BlockFetcherIterator( } } - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest + def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + curBlocks += ((blockId, size)) + curRequestSize += size + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] } } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } } + remoteRequests } - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) + def getLocalBlocks(){ + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + for (id <- localBlockIds) { + getLocal(id) match { + case Some(iter) => { + results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + logDebug("Got local block " + id) + } + case None => { + throw new BlockException(id, "Could not get block " + id + " from local machine") + } + } + } } - val numGets = remoteBlockIds.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - startTime = System.currentTimeMillis - for (id <- localBlockIds) { - getLocal(id) match { - case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") - } + def initialize(){ + // Split local and remote blocks. + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) } + + val numGets = remoteBlockIds.size - fetchRequests.size + logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + initialize() //an iterator that will read fetched blocks off the queue as they arrive. var resultsGotten = 0 @@ -1066,3 +1103,132 @@ class BlockFetcherIterator( def remoteBytesRead = _remoteBytesRead } + +class NettyBlockFetcherIterator( + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] +) extends BlockFetcherIterator(blockManager,blocksByAddress) { + + import blockManager._ + + val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] + + def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, + results : LinkedBlockingQueue[FetchResult]){ + results.put(new FetchResult( + blockId, blockSize, () => dataDeserialize(blockId, blockData) )) + } + + def startCopiers (numCopiers: Int): List [ _ <: Thread]= { + (for ( i <- Range(0,numCopiers) ) yield { + val copier = new Thread { + override def run(){ + try { + while(!isInterrupted && !fetchRequestsSync.isEmpty) { + sendRequest(fetchRequestsSync.take()) + } + } catch { + case x: InterruptedException => logInfo("Copier Interrupted") + case _ => throw new SparkException("Exception Throw in Shuffle Copier") + } + } + } + copier.start + copier + }).toList + } + + //keep this to interrupt the threads when necessary + def stopCopiers(copiers : List[_ <: Thread]) { + for (copier <- copiers) { + copier.interrupt() + } + } + + override def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) + val cmId = new ConnectionManagerId(req.address.ip, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cpier = new ShuffleCopier + cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) + logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.ip ) + } + + override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val originalTotalBlocks = totalBlocks; + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + if (size > 0) { + curBlocks += ((blockId, size)) + curRequestSize += size + } else if (size == 0){ + //here we changes the totalBlocks + totalBlocks -= 1 + } else { + throw new SparkException("Negative block size "+blockId) + } + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") + remoteRequests + } + + var copiers : List[_ <: Thread] = null + + override def initialize(){ + // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + for (request <- Utils.randomize(remoteRequests)) { + fetchRequestsSync.put(request) + } + + copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) + logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val result = results.take() + // if all the results has been retrieved + // shutdown the copiers + if (resultsGotten == totalBlocks) { + if( copiers != null ) + stopCopiers(copiers) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + } + diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index ddbf8821ad15a..d702bb23e0138 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -13,24 +13,35 @@ import scala.collection.mutable.ArrayBuffer import spark.executor.ExecutorExitCode import spark.Utils +import spark.Logging +import spark.network.netty.ShuffleSender +import spark.network.netty.PathResolver /** * Stores BlockManager blocks on disk. */ private class DiskStore(blockManager: BlockManager, rootDirs: String) - extends BlockStore(blockManager) { + extends BlockStore(blockManager) with Logging { val MAX_DIR_CREATION_ATTEMPTS: Int = 10 val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + var shuffleSender : Thread = null + val thisInstance = this // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid // having really large inodes at the top level. val localDirs = createLocalDirs() val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) + val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean + addShutdownHook() + if(useNetty){ + startShuffleBlockSender() + } + override def getSize(blockId: String): Long = { getFile(blockId).length() } @@ -180,10 +191,48 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) logDebug("Shutdown hook called") try { localDirs.foreach(localDir => Utils.deleteRecursively(localDir)) + if (useNetty && shuffleSender != null) + shuffleSender.stop } catch { case t: Throwable => logError("Exception while deleting local spark dirs", t) } } }) } + + private def startShuffleBlockSender (){ + try { + val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt + + val pResolver = new PathResolver { + def getAbsolutePath(blockId:String):String = { + if (!blockId.startsWith("shuffle_")) { + return null + } + thisInstance.getFile(blockId).getAbsolutePath() + } + } + shuffleSender = new Thread { + override def run() = { + val sender = new ShuffleSender(port,pResolver) + logInfo("created ShuffleSender binding to port : "+ port) + sender.start + } + } + shuffleSender.setDaemon(true) + shuffleSender.start + + } catch { + case interrupted: InterruptedException => + logInfo("Runner thread for ShuffleBlockSender interrupted") + + case e: Exception => { + logError("Error running ShuffleBlockSender ", e) + if (shuffleSender != null) { + shuffleSender.stop + shuffleSender = null + } + } + } + } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f378b2398c5b..e3645653ee46a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -141,7 +141,8 @@ object SparkBuild extends Build { "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", "cc.spray" %% "spray-json" % "1.1.1", - "org.apache.mesos" % "mesos" % "0.9.0-incubating" + "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "io.netty" % "netty-all" % "4.0.0.Beta2" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala index d8b987ec860bb..bd0b0e74c1d9d 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala @@ -5,7 +5,7 @@ import spark.util.{RateLimitedOutputStream, IntParam} import java.net.ServerSocket import spark.{Logging, KryoSerializer} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import io.Source +import scala.io.Source import java.io.IOException /** From 6798a09df84fb97e196c84d55cf3e21ad676871f Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sun, 7 Apr 2013 17:47:38 +0530 Subject: [PATCH 431/696] Add support for building against hadoop2-yarn : adding new maven profile for it --- bagel/pom.xml | 37 ++++++++++ core/pom.xml | 62 ++++++++++++++++ .../hadoop/mapred/HadoopMapRedUtil.scala | 3 + .../mapreduce/HadoopMapReduceUtil.scala | 3 + .../hadoop/mapred/HadoopMapRedUtil.scala | 13 ++++ .../mapreduce/HadoopMapReduceUtil.scala | 13 ++++ .../hadoop/mapred/HadoopMapRedUtil.scala | 3 + .../mapreduce/HadoopMapReduceUtil.scala | 3 + .../main/scala/spark/PairRDDFunctions.scala | 5 +- .../main/scala/spark/rdd/NewHadoopRDD.scala | 2 +- examples/pom.xml | 43 +++++++++++ pom.xml | 54 ++++++++++++++ project/SparkBuild.scala | 34 +++++++-- repl-bin/pom.xml | 50 +++++++++++++ repl/pom.xml | 71 +++++++++++++++++++ streaming/pom.xml | 37 ++++++++++ 16 files changed, 424 insertions(+), 9 deletions(-) create mode 100644 core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala create mode 100644 core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala diff --git a/bagel/pom.xml b/bagel/pom.xml index 510cff46695a8..89282161eaaba 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -102,5 +102,42 @@ + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + diff --git a/core/pom.xml b/core/pom.xml index fe9c803728654..9baa4476621b3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -279,5 +279,67 @@ + + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-source + generate-sources + + add-source + + + + src/main/scala + src/hadoop2-yarn/scala + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index ca9f7219de739..f286f2cf9c512 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -4,4 +4,7 @@ trait HadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContext(conf, jobId) def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId) + + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, + jobId, isMap, taskId, attemptId) } diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index de7b0f81e3801..264d421d14a18 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -6,4 +6,7 @@ trait HadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContext(conf, jobId) def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId) + + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, + jobId, isMap, taskId, attemptId) } diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala new file mode 100644 index 0000000000000..875c0a220bd36 --- /dev/null +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -0,0 +1,13 @@ + +package org.apache.hadoop.mapred + +import org.apache.hadoop.mapreduce.TaskType + +trait HadoopMapRedUtil { + def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) + + def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = + new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId) +} diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala new file mode 100644 index 0000000000000..8bc6fb6dea186 --- /dev/null +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -0,0 +1,13 @@ +package org.apache.hadoop.mapreduce + +import org.apache.hadoop.conf.Configuration +import task.{TaskAttemptContextImpl, JobContextImpl} + +trait HadoopMapReduceUtil { + def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) + + def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = + new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId) +} diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index 35300cea58835..a0652d7fc78a9 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -4,4 +4,7 @@ trait HadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, + jobId, isMap, taskId, attemptId) } diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 7afdbff3205c3..7fdbe322fdf0b 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -7,4 +7,7 @@ trait HadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, + jobId, isMap, taskId, attemptId) } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 07efba9e8d26e..39469fa3c87e8 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -545,8 +545,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = new TaskAttemptID(jobtrackerID, - stageId, false, context.splitId, attemptNumber) + val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.splitId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outputFormatClass.newInstance val committer = format.getOutputCommitter(hadoopContext) @@ -565,7 +564,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * however we're only going to use this local OutputCommitter for * setupJob/commitJob, so we just use a dummy "map" task. */ - val jobAttemptId = new TaskAttemptID(jobtrackerID, stageId, true, 0, 0) + val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, true, 0, 0) val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) jobCommitter.setupJob(jobTaskContext) diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index bdd974590af59..901d01ef3084f 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -57,7 +57,7 @@ class NewHadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopPartition] val conf = confBroadcast.value.value - val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0) + val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance if (format.isInstanceOf[Configurable]) { diff --git a/examples/pom.xml b/examples/pom.xml index 39cc47c709385..9594257ad4036 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -118,5 +118,48 @@ + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.spark-project + spark-streaming + ${project.version} + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + diff --git a/pom.xml b/pom.xml index 08d1fc12e031f..b3134a957d1a3 100644 --- a/pom.xml +++ b/pom.xml @@ -558,5 +558,59 @@ + + + hadoop2-yarn + + 2 + 2.0.3-alpha + + + + + maven-root + Maven root repository + http://repo1.maven.org/maven2/ + + true + + + false + + + + + + + + + org.apache.hadoop + hadoop-client + ${yarn.version} + + + org.apache.hadoop + hadoop-yarn-api + ${yarn.version} + + + org.apache.hadoop + hadoop-yarn-common + ${yarn.version} + + + + org.apache.avro + avro + 1.7.1.cloudera.2 + + + org.apache.avro + avro-ipc + 1.7.1.cloudera.2 + + + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f378b2398c5b..f041930b4ec5d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1,3 +1,4 @@ + import sbt._ import sbt.Classpaths.publishTask import Keys._ @@ -10,12 +11,18 @@ import twirl.sbt.TwirlPlugin._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - val HADOOP_VERSION = "1.0.4" - val HADOOP_MAJOR_VERSION = "1" + //val HADOOP_VERSION = "1.0.4" + //val HADOOP_MAJOR_VERSION = "1" // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" + //val HADOOP_YARN = false + + // For Hadoop 2 YARN support + val HADOOP_VERSION = "2.0.3-alpha" + val HADOOP_MAJOR_VERSION = "2" + val HADOOP_YARN = true lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming) @@ -129,7 +136,6 @@ object SparkBuild extends Build { "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "com.ning" % "compress-lzf" % "0.8.4", - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, "asm" % "asm-all" % "3.3.1", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", @@ -142,8 +148,26 @@ object SparkBuild extends Build { "cc.spray" % "spray-server" % "1.0-M2.1", "cc.spray" %% "spray-json" % "1.1.1", "org.apache.mesos" % "mesos" % "0.9.0-incubating" - ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, - unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } + ) ++ ( + if (HADOOP_MAJOR_VERSION == "2") { + if (HADOOP_YARN) { + Seq( + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION, + "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION, + "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION + ) + } else { + Seq( + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION + ) + } + } else { + Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION) + }), + unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / + ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") "src/hadoop2-yarn/scala" else "src/hadoop" + HADOOP_MAJOR_VERSION + "/scala" ) + } ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings def rootSettings = sharedSettings ++ Seq( diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dd720e22915eb..f9d84fd3c4ff2 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -153,6 +153,56 @@ + + hadoop2-yarn + + hadoop2-yarn + + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.spark-project + spark-bagel + ${project.version} + hadoop2-yarn + runtime + + + org.spark-project + spark-examples + ${project.version} + hadoop2-yarn + runtime + + + org.spark-project + spark-repl + ${project.version} + hadoop2-yarn + runtime + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + deb diff --git a/repl/pom.xml b/repl/pom.xml index a3e4606edc4d3..1f885673f463a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -187,5 +187,76 @@ + + hadoop2-yarn + + hadoop2-yarn + + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.spark-project + spark-bagel + ${project.version} + hadoop2-yarn + runtime + + + org.spark-project + spark-examples + ${project.version} + hadoop2-yarn + runtime + + + org.spark-project + spark-streaming + ${project.version} + hadoop2-yarn + runtime + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + diff --git a/streaming/pom.xml b/streaming/pom.xml index ec077e8089116..fc2e211a42eed 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -149,5 +149,42 @@ + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + From 34a47b8bc9c8d3357d42201ec41e5af2675bc766 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 7 Apr 2013 20:27:03 -0400 Subject: [PATCH 432/696] Update Scala version in docs --- docs/_config.yml | 2 +- docs/building-with-maven.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index f99d5bb376027..055ba77c5b25d 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -5,6 +5,6 @@ markdown: kramdown # of Spark, Scala, and Mesos. SPARK_VERSION: 0.7.1-SNAPSHOT SPARK_VERSION_SHORT: 0.7.1 -SCALA_VERSION: 2.9.2 +SCALA_VERSION: 2.9.3 MESOS_VERSION: 0.9.0-incubating SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index c2eeafd07af75..04cd79d039ec1 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -42,10 +42,10 @@ To run a specific test suite: You might run into the following errors if you're using a vanilla installation of Maven: - [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/andyk/Development/spark/core/target/scala-2.9.2/classes... + [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... [ERROR] PermGen space -> [Help 1] - [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/andyk/Development/spark/core/target/scala-2.9.2/classes... + [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... [ERROR] Java heap space -> [Help 1] To fix these, you can do the following: From 6962d40b44f11651ec7021c8a8d9e0cc64fff970 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 7 Apr 2013 20:27:33 -0400 Subject: [PATCH 433/696] Fix deprecated warning --- core/src/main/scala/spark/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 33dc7627a3dc3..ccd9d0364ad95 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -366,7 +366,7 @@ abstract class RDD[T: ClassManifest]( * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - @deprecated("use mapPartitionsWithIndex") + @deprecated("use mapPartitionsWithIndex", "0.7.0") def mapPartitionsWithSplit[U: ClassManifest]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = From a1586412d647da1a192259a58b10d50b02f02a9c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 7 Apr 2013 20:31:19 -0400 Subject: [PATCH 434/696] Updated link to SBT --- docs/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/index.md b/docs/index.md index 51d505e1fa8c9..0c4add45dcd2a 100644 --- a/docs/index.md +++ b/docs/index.md @@ -18,7 +18,7 @@ or you will need to set the `SCALA_HOME` environment variable to point to where you've installed Scala. Scala must also be accessible through one of these methods on slave nodes on your cluster. -Spark uses [Simple Build Tool](https://github.com/harrah/xsbt/wiki), which is bundled with it. To compile the code, go into the top-level Spark directory and run +Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run sbt/sbt package From b496decf0a486a31afe3cac706a6b80d075cd327 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 7 Apr 2013 17:44:48 -0700 Subject: [PATCH 435/696] Updating based on code review --- run | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/run b/run index ceb6e5817329f..93e4b0d18f9a2 100755 --- a/run +++ b/run @@ -22,7 +22,7 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - SPARK_DAEMON_JAVA_OPTS+="-Dspark.akka.logLifecycleEvents=true" + SPARK_DAEMON_JAVA_OPTS+=" -Dspark.akka.logLifecycleEvents=true" SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default fi From b5900d47b1386f5bc21df5db32b09b2d44e9dba7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 7 Apr 2013 20:55:42 -0400 Subject: [PATCH 436/696] Fix compile warning --- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 9e37bdf659201..5db77eb142453 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -122,6 +122,6 @@ private[spark] object CheckpointRDD extends Logging { val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same") - fs.delete(path) + fs.delete(path, true) } } From 054feb6448578de5542f9ef54d4cc88f706c22f5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 7 Apr 2013 21:15:21 -0400 Subject: [PATCH 437/696] Fixed a bug with zip --- core/src/main/scala/spark/rdd/ZippedRDD.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index e80ec17aa5e0a..35b0e06785cf3 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -10,17 +10,17 @@ private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest]( @transient rdd2: RDD[U] ) extends Partition { - var split1 = rdd1.partitions(idx) - var split2 = rdd1.partitions(idx) + var partition1 = rdd1.partitions(idx) + var partition2 = rdd2.partitions(idx) override val index: Int = idx - def splits = (split1, split2) + def partitions = (partition1, partition2) @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { - // Update the reference to parent split at the time of task serialization - split1 = rdd1.partitions(idx) - split2 = rdd2.partitions(idx) + // Update the reference to parent partition at the time of task serialization + partition1 = rdd1.partitions(idx) + partition2 = rdd2.partitions(idx) oos.defaultWriteObject() } } @@ -43,13 +43,13 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( } override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = { - val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits - rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context)) + val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions + rdd1.iterator(partition1, context).zip(rdd2.iterator(partition2, context)) } override def getPreferredLocations(s: Partition): Seq[String] = { - val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits - rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2)) + val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions + rdd1.preferredLocations(partition1).intersect(rdd2.preferredLocations(partition2)) } override def clearDependencies() { From adba773fab6294b5764d101d248815a7d3cb3558 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 8 Apr 2013 17:34:38 -0400 Subject: [PATCH 438/696] Fix passing of superstep in Bagel to avoid seeing new values of the superstep value upon recomputation, and set the default storage level in Bagel to MEMORY_AND_DISK --- bagel/src/main/scala/spark/bagel/Bagel.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala index e10c03f6bad1c..5ecdd7d0045fa 100644 --- a/bagel/src/main/scala/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/spark/bagel/Bagel.scala @@ -7,8 +7,7 @@ import scala.collection.mutable.ArrayBuffer import storage.StorageLevel object Bagel extends Logging { - - val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_ONLY + val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK /** * Runs a Bagel program. @@ -63,8 +62,9 @@ object Bagel extends Logging { val combinedMsgs = msgs.combineByKey( combiner.createCombiner _, combiner.mergeMsg _, combiner.mergeCombiners _, partitioner) val grouped = combinedMsgs.groupWith(verts) + val superstep_ = superstep // Create a read-only copy of superstep for capture in closure val (processed, numMsgs, numActiveVerts) = - comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep), storageLevel) + comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep_), storageLevel) val timeTaken = System.currentTimeMillis - startTime logInfo("Superstep %d took %d s".format(superstep, timeTaken / 1000)) From 2f883c515fe4577f0105e62dd9f395d7de42bd68 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 9 Apr 2013 13:02:50 +0800 Subject: [PATCH 439/696] Contiue to update codes for scala code style 1.refactor braces for "class" "if" "while" "for" "match" 2.make code lines less than 100 3.refactor class parameter and extends defination --- core/src/main/scala/spark/SparkContext.scala | 12 +- .../spark/scheduler/DAGSchedulerEvent.scala | 3 +- .../main/scala/spark/scheduler/TaskSet.scala | 9 +- .../scheduler/cluster/ClusterScheduler.scala | 6 +- .../cluster/FIFOTaskSetQueuesManager.scala | 10 +- .../cluster/FairTaskSetQueuesManager.scala | 139 ++++++++---------- .../scala/spark/scheduler/cluster/Pool.scala | 46 +++--- .../spark/scheduler/cluster/Schedulable.scala | 3 +- .../cluster/SchedulingAlgorithm.scala | 30 +--- .../scheduler/cluster/SchedulingMode.scala | 5 +- .../scheduler/cluster/TaskSetManager.scala | 6 +- 11 files changed, 122 insertions(+), 147 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index ed5f68637982d..7c96ae637b3ef 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -146,7 +146,8 @@ class SparkContext( case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). + newInstance().asInstanceOf[TaskSetQueuesManager] scheduler.initialize(backend, taskSetQueuesManager) scheduler @@ -166,7 +167,8 @@ class SparkContext( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). + newInstance().asInstanceOf[TaskSetQueuesManager] scheduler.initialize(backend, taskSetQueuesManager) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() @@ -186,7 +188,8 @@ class SparkContext( } else { new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName) } - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). + newInstance().asInstanceOf[TaskSetQueuesManager] scheduler.initialize(backend, taskSetQueuesManager) scheduler } @@ -602,7 +605,8 @@ class SparkContext( val callSite = Utils.getSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler,localProperties.value) + val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler + ,localProperties.value) logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() result diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 6f4e5cd83e397..11fec568c6077 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,8 @@ private[spark] case class JobSubmitted( partitions: Array[Int], allowLocal: Boolean, callSite: String, - listener: JobListener, properties: Properties = null) + listener: JobListener, + properties: Properties = null) extends DAGSchedulerEvent private[spark] case class CompletionEvent( diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala index 2498e8a5aa6b0..e4b5fcaedb67e 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/spark/scheduler/TaskSet.scala @@ -6,8 +6,13 @@ import java.util.Properties * A set of tasks submitted together to the low-level TaskScheduler, usually representing * missing partitions of a particular stage. */ -private[spark] class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int, val properties: Properties) { - val id: String = stageId + "." + attempt +private[spark] class TaskSet( + val tasks: Array[Task[_]], + val stageId: Int, + val attempt: Int, + val priority: Int, + val properties: Properties) { + val id: String = stageId + "." + attempt override def toString: String = "TaskSet " + id } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index be0d480aa079b..2ddac0ff30f75 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -155,13 +155,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray - for (i <- 0 until offers.size) - { + for (i <- 0 until offers.size){ var launchedTask = true val execId = offers(i).executorId val host = offers(i).hostname - while (availableCpus(i) > 0 && launchedTask) - { + while (availableCpus(i) > 0 && launchedTask){ launchedTask = false taskSetQueuesManager.receiveOffer(execId,host,availableCpus(i)) match { case Some(task) => diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala index 5949ee773ffa3..62d31303414de 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala @@ -28,13 +28,11 @@ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) } - override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = - { - for(manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) - { + override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = { + + for (manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) { val task = manager.slaveOffer(execId,host,avaiableCpus) - if (task != None) - { + if (task != None) { return task } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala index 0609600f35156..89b74fbb4731d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala @@ -38,71 +38,17 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with loadPoolProperties() - def loadPoolProperties() { - //first check if the file exists - val file = new File(schedulerAllocFile) - if(file.exists()) - { - val xml = XML.loadFile(file) - for (poolNode <- (xml \\ POOL_POOLS_PROPERTY)) { - - val poolName = (poolNode \ POOL_POOL_NAME_PROPERTY).text - var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE - var minShares = POOL_DEFAULT_MINIMUM_SHARES - var weight = POOL_DEFAULT_WEIGHT - - val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text - if( xmlSchedulingMode != "") - { - try - { - schedulingMode = SchedulingMode.withName(xmlSchedulingMode) - } - catch{ - case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") - } - } - - val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text - if(xmlMinShares != "") - { - minShares = xmlMinShares.toInt - } - - val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text - if(xmlWeight != "") - { - weight = xmlWeight.toInt - } - - val pool = new Pool(poolName,schedulingMode,minShares,weight) - pools += pool - poolNameToPool(poolName) = pool - logInfo("Create new pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,schedulingMode,minShares,weight)) - } - } - - if(!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) - { - val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) - pools += pool - poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool - logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } - } - override def addTaskSetManager(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME - if(manager.taskSet.properties != null) - { + if (manager.taskSet.properties != null) { poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - if(!poolNameToPool.contains(poolName)) - { + if (!poolNameToPool.contains(poolName)) { //we will create a new pool that user has configured in app instead of being defined in xml file val pool = new Pool(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) pools += pool poolNameToPool(poolName) = pool - logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( + poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) } } poolNameToPool(poolName).addTaskSetManager(manager) @@ -110,10 +56,8 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with } override def removeTaskSetManager(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if(manager.taskSet.properties != null) - { + if (manager.taskSet.properties != null) { poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) } logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id,poolName)) @@ -124,8 +68,7 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with override def taskFinished(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME - if(manager.taskSet.properties != null) - { + if (manager.taskSet.properties != null) { poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) } val pool = poolNameToPool(poolName) @@ -139,19 +82,15 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with } } - override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = - { - + override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = { val sortedPools = pools.sortWith(poolScheduleAlgorithm.comparator) - for(pool <- sortedPools) - { - logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format(pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) + for (pool <- sortedPools) { + logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format( + pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) } - for (pool <- sortedPools) - { + for (pool <- sortedPools) { val task = pool.receiveOffer(execId,host,avaiableCpus) - if(task != None) - { + if(task != None) { pool.runningTasks += 1 return task } @@ -159,14 +98,60 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with return None } - override def checkSpeculatableTasks(): Boolean = - { + override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for (pool <- pools) - { + for (pool <- pools) { shouldRevive |= pool.checkSpeculatableTasks() } return shouldRevive } + def loadPoolProperties() { + //first check if the file exists + val file = new File(schedulerAllocFile) + if (file.exists()) { + val xml = XML.loadFile(file) + for (poolNode <- (xml \\ POOL_POOLS_PROPERTY)) { + + val poolName = (poolNode \ POOL_POOL_NAME_PROPERTY).text + var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE + var minShares = POOL_DEFAULT_MINIMUM_SHARES + var weight = POOL_DEFAULT_WEIGHT + + val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text + if (xmlSchedulingMode != "") { + try{ + schedulingMode = SchedulingMode.withName(xmlSchedulingMode) + } + catch{ + case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + } + } + + val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text + if (xmlMinShares != "") { + minShares = xmlMinShares.toInt + } + + val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text + if (xmlWeight != "") { + weight = xmlWeight.toInt + } + + val pool = new Pool(poolName,schedulingMode,minShares,weight) + pools += pool + poolNameToPool(poolName) = pool + logInfo("Create new pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( + poolName,schedulingMode,minShares,weight)) + } + } + + if (!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) { + val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) + pools += pool + poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool + logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( + POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + } + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 8fdca5d2b450e..e0917ca1ca21b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -7,8 +7,13 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * An Schedulable entity that represent collection of TaskSetManager */ -private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMode, initMinShare:Int, initWeight:Int) extends Schedulable with Logging -{ +private[spark] class Pool( + val poolName: String, + val schedulingMode: SchedulingMode, + initMinShare:Int, + initWeight:Int) + extends Schedulable + with Logging { var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -19,10 +24,8 @@ private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMod val priority = 0 val stageId = 0 - var taskSetSchedulingAlgorithm: SchedulingAlgorithm = - { - schedulingMode match - { + var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { + schedulingMode match { case SchedulingMode.FAIR => val schedule = new FairSchedulingAlgorithm() schedule @@ -32,43 +35,36 @@ private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMod } } - def addTaskSetManager(manager:TaskSetManager) - { + def addTaskSetManager(manager:TaskSetManager) { activeTaskSetsQueue += manager } - def removeTaskSetManager(manager:TaskSetManager) - { + def removeTaskSetManager(manager:TaskSetManager) { activeTaskSetsQueue -= manager } - def removeExecutor(executorId: String, host: String) - { + def removeExecutor(executorId: String, host: String) { activeTaskSetsQueue.foreach(_.executorLost(executorId,host)) } - def checkSpeculatableTasks(): Boolean = - { + def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for(ts <- activeTaskSetsQueue) - { + for (ts <- activeTaskSetsQueue) { shouldRevive |= ts.checkSpeculatableTasks() } return shouldRevive } - def receiveOffer(execId:String,host:String,availableCpus:Double):Option[TaskDescription] = - { + def receiveOffer(execId:String,host:String,availableCpus:Double):Option[TaskDescription] = { val sortedActiveTasksSetQueue = activeTaskSetsQueue.sortWith(taskSetSchedulingAlgorithm.comparator) - for(manager <- sortedActiveTasksSetQueue) - { - logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format(poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) + for (manager <- sortedActiveTasksSetQueue) { + logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format( + poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) } - for(manager <- sortedActiveTasksSetQueue) - { + + for (manager <- sortedActiveTasksSetQueue) { val task = manager.slaveOffer(execId,host,availableCpus) - if (task != None) - { + if (task != None) { manager.runningTasks += 1 return task } diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 6f4f104f42cfe..8dfc369c035be 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,7 +1,8 @@ package spark.scheduler.cluster /** - * An interface for schedulable entities, there are two type Schedulable entities(Pools and TaskSetManagers) + * An interface for schedulable entities. + * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { def weight:Int diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index 2f8123587f69c..ac2237a7ef1ff 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -9,34 +9,25 @@ private[spark] trait SchedulingAlgorithm { def comparator(s1: Schedulable,s2: Schedulable): Boolean } -private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm -{ - override def comparator(s1: Schedulable, s2: Schedulable): Boolean = - { +private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { + override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { val priority1 = s1.priority val priority2 = s2.priority var res = Math.signum(priority1 - priority2) - if (res == 0) - { + if (res == 0) { val stageId1 = s1.stageId val stageId2 = s2.stageId res = Math.signum(stageId1 - stageId2) } if (res < 0) - { return true - } else - { return false - } } } -private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm -{ - def comparator(s1: Schedulable, s2:Schedulable): Boolean = - { +private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { + def comparator(s1: Schedulable, s2:Schedulable): Boolean = { val minShare1 = s1.minShare val minShare2 = s2.minShare val runningTasks1 = s1.runningTasks @@ -49,22 +40,15 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true - if(s1Needy && !s2Needy) - { + if (s1Needy && !s2Needy) res = true - } else if(!s1Needy && s2Needy) - { res = false - } else if (s1Needy && s2Needy) - { res = minShareRatio1 <= minShareRatio2 - } else - { res = taskToWeightRatio1 <= taskToWeightRatio2 - } + return res } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 480af2c1a30e3..6e0c6793e0153 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,8 +1,7 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR","FIFO") -{ - type SchedulingMode = Value +object SchedulingMode extends Enumeration("FAIR","FIFO"){ + type SchedulingMode = Value val FAIR,FIFO = Value } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index ddc4fa6642842..7ec2f69da559a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,7 +17,11 @@ import java.nio.ByteBuffer /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) extends Schedulable with Logging { +private[spark] class TaskSetManager( + sched: ClusterScheduler, + val taskSet: TaskSet) + extends Schedulable + with Logging { // Maximum time to wait to run a task in a preferred location (in ms) val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong From 18bd41d1a38bdb9d68a9e6f360a15f9d7b25ebf5 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Tue, 9 Apr 2013 14:35:29 -0400 Subject: [PATCH 440/696] Don't pull in old versions of Jackson via hadoop-core --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f03efd6c83527..029ade2e2eedc 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -129,7 +129,7 @@ object SparkBuild extends Build { "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "com.ning" % "compress-lzf" % "0.8.4", - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), "asm" % "asm-all" % "3.3.1", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", From f1d8871ca14567e58111c1794d2a1b4ad48ce7cd Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Tue, 9 Apr 2013 23:35:13 -0400 Subject: [PATCH 441/696] Uniform whitespace across scala examples --- .../scala/spark/examples/LocalKMeans.scala | 138 +++++++++--------- .../spark/examples/MultiBroadcastTest.scala | 6 +- .../examples/SimpleSkewedGroupByTest.scala | 4 +- .../spark/examples/SkewedGroupByTest.scala | 4 +- 4 files changed, 76 insertions(+), 76 deletions(-) diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala index b07e799cef5af..4849f216fb293 100644 --- a/examples/src/main/scala/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala @@ -10,73 +10,73 @@ import scala.collection.mutable.HashSet * K-means clustering. */ object LocalKMeans { - val N = 1000 - val R = 1000 // Scaling factor - val D = 10 - val K = 10 - val convergeDist = 0.001 - val rand = new Random(42) - - def generateData = { - def generatePoint(i: Int) = { - Vector(D, _ => rand.nextDouble * R) - } - Array.tabulate(N)(generatePoint) - } - - def closestPoint(p: Vector, centers: HashMap[Int, Vector]): Int = { - var index = 0 - var bestIndex = 0 - var closest = Double.PositiveInfinity - - for (i <- 1 to centers.size) { - val vCurr = centers.get(i).get - val tempDist = p.squaredDist(vCurr) - if (tempDist < closest) { - closest = tempDist - bestIndex = i - } - } - - return bestIndex - } - - def main(args: Array[String]) { - val data = generateData - var points = new HashSet[Vector] - var kPoints = new HashMap[Int, Vector] - var tempDist = 1.0 - - while (points.size < K) { - points.add(data(rand.nextInt(N))) - } - - val iter = points.iterator - for (i <- 1 to points.size) { - kPoints.put(i, iter.next()) - } - - println("Initial centers: " + kPoints) - - while(tempDist > convergeDist) { - var closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) - - var mappings = closest.groupBy[Int] (x => x._1) - - var pointStats = mappings.map(pair => pair._2.reduceLeft [(Int, (Vector, Int))] {case ((id1, (x1, y1)), (id2, (x2, y2))) => (id1, (x1 + x2, y1+y2))}) - - var newPoints = pointStats.map {mapping => (mapping._1, mapping._2._1/mapping._2._2)} - - tempDist = 0.0 - for (mapping <- newPoints) { - tempDist += kPoints.get(mapping._1).get.squaredDist(mapping._2) - } - - for (newP <- newPoints) { - kPoints.put(newP._1, newP._2) - } - } - - println("Final centers: " + kPoints) - } + val N = 1000 + val R = 1000 // Scaling factor + val D = 10 + val K = 10 + val convergeDist = 0.001 + val rand = new Random(42) + + def generateData = { + def generatePoint(i: Int) = { + Vector(D, _ => rand.nextDouble * R) + } + Array.tabulate(N)(generatePoint) + } + + def closestPoint(p: Vector, centers: HashMap[Int, Vector]): Int = { + var index = 0 + var bestIndex = 0 + var closest = Double.PositiveInfinity + + for (i <- 1 to centers.size) { + val vCurr = centers.get(i).get + val tempDist = p.squaredDist(vCurr) + if (tempDist < closest) { + closest = tempDist + bestIndex = i + } + } + + return bestIndex + } + + def main(args: Array[String]) { + val data = generateData + var points = new HashSet[Vector] + var kPoints = new HashMap[Int, Vector] + var tempDist = 1.0 + + while (points.size < K) { + points.add(data(rand.nextInt(N))) + } + + val iter = points.iterator + for (i <- 1 to points.size) { + kPoints.put(i, iter.next()) + } + + println("Initial centers: " + kPoints) + + while(tempDist > convergeDist) { + var closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) + + var mappings = closest.groupBy[Int] (x => x._1) + + var pointStats = mappings.map(pair => pair._2.reduceLeft [(Int, (Vector, Int))] {case ((id1, (x1, y1)), (id2, (x2, y2))) => (id1, (x1 + x2, y1+y2))}) + + var newPoints = pointStats.map {mapping => (mapping._1, mapping._2._1/mapping._2._2)} + + tempDist = 0.0 + for (mapping <- newPoints) { + tempDist += kPoints.get(mapping._1).get.squaredDist(mapping._2) + } + + for (newP <- newPoints) { + kPoints.put(newP._1, newP._2) + } + } + + println("Final centers: " + kPoints) + } } diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala index 92cd81c48742f..a0aaf609186b7 100644 --- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala @@ -8,7 +8,7 @@ object MultiBroadcastTest { System.err.println("Usage: BroadcastTest [] [numElem]") System.exit(1) } - + val sc = new SparkContext(args(0), "Broadcast Test", System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) @@ -19,7 +19,7 @@ object MultiBroadcastTest { for (i <- 0 until arr1.length) { arr1(i) = i } - + var arr2 = new Array[Int](num) for (i <- 0 until arr2.length) { arr2(i) = i @@ -30,7 +30,7 @@ object MultiBroadcastTest { sc.parallelize(1 to 10, slices).foreach { i => println(barr1.value.size + barr2.value.size) } - + System.exit(0) } } diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala index 0d17bda004b6e..461b84a2c6623 100644 --- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala @@ -11,7 +11,7 @@ object SimpleSkewedGroupByTest { "[numMappers] [numKVPairs] [valSize] [numReducers] [ratio]") System.exit(1) } - + var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000 @@ -20,7 +20,7 @@ object SimpleSkewedGroupByTest { val sc = new SparkContext(args(0), "GroupBy Test", System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - + val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random var result = new Array[(Int, Array[Byte])](numKVPairs) diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala index 83be3fc27b5b0..435675f9de489 100644 --- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala @@ -10,7 +10,7 @@ object SkewedGroupByTest { System.err.println("Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) } - + var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000 @@ -18,7 +18,7 @@ object SkewedGroupByTest { val sc = new SparkContext(args(0), "GroupBy Test", System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - + val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random From 9bf24e1d61a629b410183dfc4296bba994f0a79e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 9 Apr 2013 21:37:02 -0700 Subject: [PATCH 442/696] Just use a loop for retries --- ec2/retry_decorator.py | 44 ------------------------------------------ ec2/spark_ec2.py | 23 ++++++++++++++++------ 2 files changed, 17 insertions(+), 50 deletions(-) delete mode 100644 ec2/retry_decorator.py diff --git a/ec2/retry_decorator.py b/ec2/retry_decorator.py deleted file mode 100644 index 1a2f79ae844f7..0000000000000 --- a/ec2/retry_decorator.py +++ /dev/null @@ -1,44 +0,0 @@ -import time -from functools import wraps - -def retry(ExceptionToCheck, tries=4, delay=3, backoff=2, logger=None): - """Retry calling the decorated function using an exponential backoff. - - http://www.saltycrane.com/blog/2009/11/trying-out-retry-decorator-python/ - original from: http://wiki.python.org/moin/PythonDecoratorLibrary#Retry - - :param ExceptionToCheck: the exception to check. may be a tuple of - exceptions to check - :type ExceptionToCheck: Exception or tuple - :param tries: number of times to try (not retry) before giving up - :type tries: int - :param delay: initial delay between retries in seconds - :type delay: int - :param backoff: backoff multiplier e.g. value of 2 will double the delay - each retry - :type backoff: int - :param logger: logger to use. If None, print - :type logger: logging.Logger instance - """ - def deco_retry(f): - - @wraps(f) - def f_retry(*args, **kwargs): - mtries, mdelay = tries, delay - while mtries > 1: - try: - return f(*args, **kwargs) - except ExceptionToCheck, e: - msg = "%s, Retrying in %d seconds..." % (str(e), mdelay) - if logger: - logger.warning(msg) - else: - print msg - time.sleep(mdelay) - mtries -= 1 - mdelay *= backoff - return f(*args, **kwargs) - - return f_retry # true decorator - - return deco_retry diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index aa2d360fbbcaf..9f2daad2b6323 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -22,7 +22,6 @@ import logging import os import random -from retry_decorator import retry import shutil import subprocess import sys @@ -541,12 +540,24 @@ def scp(host, opts, local_file, dest_file): (opts.identity_file, local_file, opts.user, host, dest_file), shell=True) -# Run a command on a host through ssh, throwing an exception if ssh fails -@retry(subprocess.CalledProcessError, tries=3, delay=30) +# Run a command on a host through ssh, retrying up to two times +# and then throwing an exception if ssh continues to fail. def ssh(host, opts, command): - subprocess.check_call( - "ssh -t -o StrictHostKeyChecking=no -i %s %s@%s '%s'" % - (opts.identity_file, opts.user, host, command), shell=True) + tries = 0 + while True: + try: + return subprocess.check_call( + "ssh -t -o StrictHostKeyChecking=no -i %s %s@%s '%s'" % + (opts.identity_file, opts.user, host, command), shell=True) + except subprocess.CalledProcessError as e: + if (tries > 2): + raise e + print "Error connecting to host {0}, sleeping 30".format(e) + time.sleep(30) + tries = tries + 1 + + + # Gets a list of zones to launch instances in From 6efc8cae8f4497d431e2a861778e2e120e774990 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 10 Apr 2013 13:44:10 -0300 Subject: [PATCH 443/696] Typos: cluser -> cluster --- docs/streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index b30699cf3df86..f5788dc46776d 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -83,7 +83,7 @@ DStreams support many of the transformations available on normal Spark RDD's: @@ -132,7 +132,7 @@ Spark Streaming features windowed computations, which allow you to apply transfo +Note: By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluster) to do the grouping. You can pass an optional numTasks argument to set a different number of tasks. From 29d3440efbc6e59da388f8d9c7f638a74dd9e9c5 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 11 Apr 2013 01:54:02 -0300 Subject: [PATCH 444/696] Add details when BlockManager heartbeats time out MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Makes it more clear what the threshold was for tuning spark.storage.blockManagerSlaveTimeoutMs Before: WARN  "Removing BlockManager BlockManagerId(201304022120-1976232532-5050-27464-0, myhostname, 51337) with no recent heart beats After: WARN  "Removing BlockManager BlockManagerId(201304022120-1976232532-5050-27464-0, myhostname, 51337) with no recent heart beats: 19216ms exceeds 15000ms --- .../src/main/scala/spark/storage/BlockManagerMasterActor.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 2830bc62975ae..9ab451b46b95c 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -121,7 +121,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { val toRemove = new HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { if (info.lastSeenMs < minSeenTime) { - logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats") + logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId } } From ed336e0d44d27e9be66adb0962f82af7d1ac4d87 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 11 Apr 2013 22:29:37 -0400 Subject: [PATCH 445/696] Fix tests from different projects running in parallel in SBT 0.12 --- project/SparkBuild.scala | 6 ++++-- project/build.properties | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f03efd6c83527..0e55c5120444b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,7 +44,10 @@ object SparkBuild extends Build { transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), - // shared between both core and streaming. + // Only allow one test at a time, even across projects, since they run in the same JVM + concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), + + // Shared between both core and streaming. resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), // For Sonatype publishing @@ -98,7 +101,6 @@ object SparkBuild extends Build { "com.novocode" % "junit-interface" % "0.9" % "test", "org.easymock" % "easymock" % "3.1" % "test" ), - parallelExecution := false, /* Workaround for issue #206 (fixed after SBT 0.11.0) */ watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task, const(std.TaskExtra.constant(Nil)), aggregate = true, includeRoot = true) apply { _.join.map(_.flatten) }, diff --git a/project/build.properties b/project/build.properties index 66ad72ce2eba1..9b860e23c51a6 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.12.2 +sbt.version=0.12.3 From 6f688608915a82e6dcf4a27dc92e4b25a3570fa4 Mon Sep 17 00:00:00 2001 From: Mike Date: Thu, 11 Apr 2013 20:52:06 -0700 Subject: [PATCH 446/696] Reversed the order of tests to find a scala executable (in the case when SPARK_LAUNCH_WITH_SCALA is defined): instead of checking in the PATH first, and only then (if not found) for SCALA_HOME, now we check for SCALA_HOME first, and only then (if not defined) do we look in the PATH. The advantage is that now if the user has a more recent (non-compatible) version of scala in her PATH, she can use SCALA_HOME to point to the older (compatible) version for use with spark. Suggested by Josh Rosen in this thread: https://groups.google.com/forum/?fromgroups=#!topic/spark-users/NC9JKvP8808 --- run | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/run b/run index 73239097b9f1a..756f8703f2502 100755 --- a/run +++ b/run @@ -47,14 +47,15 @@ case "$1" in esac if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - if [ `command -v scala` ]; then - RUNNER="scala" + if [ "$SCALA_HOME" ]; then + RUNNER="${SCALA_HOME}/bin/scala" else - if [ -z "$SCALA_HOME" ]; then - echo "SCALA_HOME is not set" >&2 + if [ `command -v scala` ]; then + RUNNER="scala" + else + echo "SCALA_HOME is not set and scala is not in PATH" >&2 exit 1 fi - RUNNER="${SCALA_HOME}/bin/scala" fi else if [ `command -v java` ]; then From 60a91b3b59513b00a3afaa5a7621ee17eb6ee425 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 12 Apr 2013 12:32:58 -0700 Subject: [PATCH 447/696] Update quick-start.md heading on Operations (not just Transformations). --- docs/quick-start.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index 5c80d2ed3a52e..2d961b29cb197 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -53,8 +53,8 @@ scala> textFile.filter(line => line.contains("Spark")).count() // How many lines res3: Long = 15 {% endhighlight %} -## Transformations -RDD transformations can be used for more complex computations. Let's say we want to find the line with the most words: +## More On RDD Operations +RDD actions and transformations can be used for more complex computations. Let's say we want to find the line with the most words: {% highlight scala %} scala> textFile.map(line => line.split(" ").size).reduce((a, b) => if (a > b) a else b) From c35d530bcfea2e1764863eb9f47a794d8fa001af Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 13 Apr 2013 12:43:12 -0400 Subject: [PATCH 448/696] Fix compile error --- core/src/main/scala/spark/storage/BlockManagerMasterActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 9ab451b46b95c..1a6a6cfd3fb55 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -121,7 +121,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { val toRemove = new HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { if (info.lastSeenMs < minSeenTime) { - logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId } From d90d2af1036e909f81cf77c85bfe589993c4f9f3 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Mon, 15 Apr 2013 18:12:11 +0530 Subject: [PATCH 449/696] Checkpoint commit - compiles and passes a lot of tests - not all though, looking into FileSuite issues --- .../scala/spark/deploy/SparkHadoopUtil.scala | 18 + .../scala/spark/deploy/SparkHadoopUtil.scala | 59 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 342 +++++++++++ .../yarn/ApplicationMasterArguments.scala | 78 +++ .../scala/spark/deploy/yarn/Client.scala | 326 +++++++++++ .../spark/deploy/yarn/ClientArguments.scala | 104 ++++ .../spark/deploy/yarn/WorkerRunnable.scala | 171 ++++++ .../deploy/yarn/YarnAllocationHandler.scala | 547 ++++++++++++++++++ .../cluster/YarnClusterScheduler.scala | 42 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 18 + .../src/main/scala/spark/ClosureCleaner.scala | 12 +- .../scala/spark/FetchFailedException.scala | 25 +- core/src/main/scala/spark/Logging.scala | 4 + .../main/scala/spark/MapOutputTracker.scala | 100 ++-- core/src/main/scala/spark/SparkContext.scala | 46 +- core/src/main/scala/spark/SparkEnv.scala | 15 +- core/src/main/scala/spark/Utils.scala | 138 ++++- .../scala/spark/api/python/PythonRDD.scala | 2 + .../scala/spark/deploy/DeployMessage.scala | 19 +- .../scala/spark/deploy/JsonProtocol.scala | 1 + .../spark/deploy/LocalSparkCluster.scala | 8 +- .../scala/spark/deploy/client/Client.scala | 6 +- .../spark/deploy/client/ClientListener.scala | 2 +- .../spark/deploy/client/TestClient.scala | 2 +- .../scala/spark/deploy/master/Master.scala | 16 +- .../spark/deploy/master/MasterArguments.scala | 17 +- .../spark/deploy/master/WorkerInfo.scala | 9 + .../spark/deploy/worker/ExecutorRunner.scala | 6 +- .../scala/spark/deploy/worker/Worker.scala | 19 +- .../spark/deploy/worker/WorkerArguments.scala | 13 +- .../main/scala/spark/executor/Executor.scala | 7 +- .../executor/StandaloneExecutorBackend.scala | 31 +- .../main/scala/spark/network/Connection.scala | 192 ++++-- .../spark/network/ConnectionManager.scala | 390 +++++++++---- .../main/scala/spark/network/Message.scala | 3 +- .../scala/spark/scheduler/DAGScheduler.scala | 18 +- .../spark/scheduler/DAGSchedulerEvent.scala | 4 + .../spark/scheduler/InputFormatInfo.scala | 156 +++++ .../scala/spark/scheduler/ResultTask.scala | 10 +- .../spark/scheduler/ShuffleMapTask.scala | 12 +- .../scala/spark/scheduler/SplitInfo.scala | 61 ++ .../scala/spark/scheduler/TaskScheduler.scala | 4 + .../scheduler/TaskSchedulerListener.scala | 3 + .../scheduler/cluster/ClusterScheduler.scala | 276 ++++++++- .../cluster/SparkDeploySchedulerBackend.scala | 8 +- .../cluster/StandaloneClusterMessage.scala | 7 +- .../cluster/StandaloneSchedulerBackend.scala | 33 +- .../spark/scheduler/cluster/TaskInfo.scala | 9 +- .../scheduler/cluster/TaskSetManager.scala | 309 ++++++++-- .../spark/scheduler/cluster/WorkerOffer.scala | 2 +- .../scheduler/local/LocalScheduler.scala | 4 +- .../scala/spark/storage/BlockManager.scala | 123 ++-- .../scala/spark/storage/BlockManagerId.scala | 40 +- .../storage/BlockManagerMasterActor.scala | 20 +- .../spark/storage/BlockMessageArray.scala | 1 + .../main/scala/spark/storage/DiskStore.scala | 33 +- .../scala/spark/storage/MemoryStore.scala | 4 +- .../scala/spark/storage/StorageLevel.scala | 8 +- .../src/main/scala/spark/util/AkkaUtils.scala | 13 +- .../scala/spark/util/TimeStampedHashMap.scala | 8 + .../spark/deploy/master/index.scala.html | 2 +- .../spark/deploy/worker/index.scala.html | 2 +- .../spark/storage/worker_table.scala.html | 2 +- .../test/scala/spark/DistributedSuite.scala | 2 +- core/src/test/scala/spark/FileSuite.scala | 1 + .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- 66 files changed, 3488 insertions(+), 477 deletions(-) create mode 100644 core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala create mode 100644 core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala create mode 100644 core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala create mode 100644 core/src/main/scala/spark/scheduler/InputFormatInfo.scala create mode 100644 core/src/main/scala/spark/scheduler/SplitInfo.scala diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala new file mode 100644 index 0000000000000..d4badbc5c4254 --- /dev/null +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -0,0 +1,18 @@ +package spark.deploy + +/** + * Contains util methods to interact with Hadoop from spark. + */ +object SparkHadoopUtil { + + def getUserNameFromEnvironment(): String = { + // defaulting to -D ... + System.getProperty("user.name") + } + + def runAsUser(func: (Product) => Unit, args: Product) { + + // Add support, if exists - for now, simply run func ! + func(args) + } +} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala new file mode 100644 index 0000000000000..66e5ad8491064 --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -0,0 +1,59 @@ +package spark.deploy + +import collection.mutable.HashMap +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import java.security.PrivilegedExceptionAction + +/** + * Contains util methods to interact with Hadoop from spark. + */ +object SparkHadoopUtil { + + val yarnConf = new YarnConfiguration(new Configuration()) + + def getUserNameFromEnvironment(): String = { + // defaulting to env if -D is not present ... + val retval = System.getProperty(Environment.USER.name, System.getenv(Environment.USER.name)) + + // If nothing found, default to user we are running as + if (retval == null) System.getProperty("user.name") else retval + } + + def runAsUser(func: (Product) => Unit, args: Product) { + runAsUser(func, args, getUserNameFromEnvironment()) + } + + def runAsUser(func: (Product) => Unit, args: Product, user: String) { + + // println("running as user " + jobUserName) + + UserGroupInformation.setConfiguration(yarnConf) + val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(user) + appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] { + def run: AnyRef = { + func(args) + // no return value ... + null + } + }) + } + + // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. + def isYarnMode(): Boolean = { + val yarnMode = System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")) + java.lang.Boolean.valueOf(yarnMode) + } + + // Set an env variable indicating we are running in YARN mode. + // Note that anything with SPARK prefix gets propagated to all (remote) processes + def setYarnMode() { + System.setProperty("SPARK_YARN_MODE", "true") + } + + def setYarnMode(env: HashMap[String, String]) { + env("SPARK_YARN_MODE") = "true" + } +} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala new file mode 100644 index 0000000000000..65361e0ed9952 --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -0,0 +1,342 @@ +package spark.deploy.yarn + +import java.net.Socket +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import scala.collection.JavaConversions._ +import spark.{SparkContext, Logging, Utils} +import org.apache.hadoop.security.UserGroupInformation +import java.security.PrivilegedExceptionAction + +class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { + + def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + + private var rpc: YarnRPC = YarnRPC.create(conf) + private var resourceManager: AMRMProtocol = null + private var appAttemptId: ApplicationAttemptId = null + private var userThread: Thread = null + private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + private var yarnAllocator: YarnAllocationHandler = null + + def run() { + + // Initialization + val jobUserName = Utils.getUserNameFromEnvironment() + logInfo("running as user " + jobUserName) + + // run as user ... + UserGroupInformation.setConfiguration(yarnConf) + val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(jobUserName) + appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] { + def run: AnyRef = { + runImpl() + return null + } + }) + } + + private def runImpl() { + + appAttemptId = getApplicationAttemptId() + resourceManager = registerWithResourceManager() + val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + + // Compute number of threads for akka + val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() + + if (minimumMemory > 0) { + val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD + val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) + + if (numCore > 0) { + // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 + // TODO: Uncomment when hadoop is on a version which has this fixed. + // args.workerCores = numCore + } + } + + // Workaround until hadoop moves to something which has + // https://issues.apache.org/jira/browse/HADOOP-8406 + // ignore result + // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times + // Hence args.workerCores = numCore disabled above. Any better option ? + // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) + + ApplicationMaster.register(this) + // Start the user's JAR + userThread = startUserClass() + + // This a bit hacky, but we need to wait until the spark.master.port property has + // been set by the Thread executing the user class. + waitForSparkMaster() + + // Allocate all containers + allocateWorkers() + + // Wait for the user class to Finish + userThread.join() + + // Finish the ApplicationMaster + finishApplicationMaster() + // TODO: Exit based on success/failure + System.exit(0) + } + + private def getApplicationAttemptId(): ApplicationAttemptId = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + logInfo("ApplicationAttemptId: " + appAttemptId) + return appAttemptId + } + + private def registerWithResourceManager(): AMRMProtocol = { + val rmAddress = NetUtils.createSocketAddr(yarnConf.get( + YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) + logInfo("Connecting to ResourceManager at " + rmAddress) + return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] + } + + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { + logInfo("Registering the ApplicationMaster") + val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) + .asInstanceOf[RegisterApplicationMasterRequest] + appMasterRequest.setApplicationAttemptId(appAttemptId) + // Setting this to master host,port - so that the ApplicationReport at client has some sensible info. + // Users can then monitor stderr/stdout on that node if required. + appMasterRequest.setHost(Utils.localHostName()) + appMasterRequest.setRpcPort(0) + // What do we provide here ? Might make sense to expose something sensible later ? + appMasterRequest.setTrackingUrl("") + return resourceManager.registerApplicationMaster(appMasterRequest) + } + + private def waitForSparkMaster() { + logInfo("Waiting for spark master to be reachable.") + var masterUp = false + while(!masterUp) { + val masterHost = System.getProperty("spark.master.host") + val masterPort = System.getProperty("spark.master.port") + try { + val socket = new Socket(masterHost, masterPort.toInt) + socket.close() + logInfo("Master now available: " + masterHost + ":" + masterPort) + masterUp = true + } catch { + case e: Exception => + logError("Failed to connect to master at " + masterHost + ":" + masterPort) + Thread.sleep(100) + } + } + } + + private def startUserClass(): Thread = { + logInfo("Starting the user JAR in a separate Thread") + val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader) + .getMethod("main", classOf[Array[String]]) + val t = new Thread { + override def run() { + var mainArgs: Array[String] = null + var startIndex = 0 + + // I am sure there is a better 'scala' way to do this .... but I am just trying to get things to work right now ! + if (args.userArgs.isEmpty || args.userArgs.get(0) != "yarn-standalone") { + // ensure that first param is ALWAYS "yarn-standalone" + mainArgs = new Array[String](args.userArgs.size() + 1) + mainArgs.update(0, "yarn-standalone") + startIndex = 1 + } + else { + mainArgs = new Array[String](args.userArgs.size()) + } + + args.userArgs.copyToArray(mainArgs, startIndex, args.userArgs.size()) + + mainMethod.invoke(null, mainArgs) + } + } + t.start() + return t + } + + private def allocateWorkers() { + logInfo("Waiting for spark context initialization") + + try { + var sparkContext: SparkContext = null + ApplicationMaster.sparkContextRef.synchronized { + var count = 0 + while (ApplicationMaster.sparkContextRef.get() == null) { + logInfo("Waiting for spark context initialization ... " + count) + count = count + 1 + ApplicationMaster.sparkContextRef.wait(10000L) + } + sparkContext = ApplicationMaster.sparkContextRef.get() + assert(sparkContext != null) + this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, sparkContext.preferredNodeLocationData) + } + + + logInfo("Allocating " + args.numWorkers + " workers.") + // Wait until all containers have finished + // TODO: This is a bit ugly. Can we make it nicer? + // TODO: Handle container failure + while(yarnAllocator.getNumWorkersRunning < args.numWorkers && + // If user thread exists, then quit ! + userThread.isAlive) { + + this.yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + ApplicationMaster.incrementAllocatorLoop(1) + Thread.sleep(100) + } + } finally { + // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : + // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks + ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) + } + logInfo("All workers have launched.") + + // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout + if (userThread.isAlive){ + // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. + + val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) + // must be <= timeoutInterval/ 2. + // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. + // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. + val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + launchReporterThread(interval) + } + } + + // TODO: We might want to extend this to allocate more containers in case they die ! + private def launchReporterThread(_sleepTime: Long): Thread = { + val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + + val t = new Thread { + override def run() { + while (userThread.isAlive){ + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning + if (missingWorkerCount > 0) { + logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") + yarnAllocator.allocateContainers(missingWorkerCount) + } + else sendProgress() + Thread.sleep(sleepTime) + } + } + } + // setting to daemon status, though this is usually not a good idea. + t.setDaemon(true) + t.start() + logInfo("Started progress reporter thread - sleep time : " + sleepTime) + return t + } + + private def sendProgress() { + logDebug("Sending progress") + // simulated with an allocate request with no nodes requested ... + yarnAllocator.allocateContainers(0) + } + + /* + def printContainers(containers: List[Container]) = { + for (container <- containers) { + logInfo("Launching shell command on a new container." + + ", containerId=" + container.getId() + + ", containerNode=" + container.getNodeId().getHost() + + ":" + container.getNodeId().getPort() + + ", containerNodeURI=" + container.getNodeHttpAddress() + + ", containerState" + container.getState() + + ", containerResourceMemory" + + container.getResource().getMemory()) + } + } + */ + + def finishApplicationMaster() { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + // TODO: Check if the application has failed or succeeded + finishReq.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED) + resourceManager.finishApplicationMaster(finishReq) + } + +} + +object ApplicationMaster { + // number of times to wait for the allocator loop to complete. + // each loop iteration waits for 100ms, so maximum of 3 seconds. + // This is to ensure that we have reasonable number of containers before we start + // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be optimal as more + // containers are available. Might need to handle this better. + private val ALLOCATOR_LOOP_WAIT_COUNT = 30 + def incrementAllocatorLoop(by: Int) { + val count = yarnAllocatorLoop.getAndAdd(by) + if (count >= ALLOCATOR_LOOP_WAIT_COUNT){ + yarnAllocatorLoop.synchronized { + // to wake threads off wait ... + yarnAllocatorLoop.notifyAll() + } + } + } + + private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() + + def register(master: ApplicationMaster) { + applicationMasters.add(master) + } + + val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) + val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) + + def sparkContextInitialized(sc: SparkContext): Boolean = { + var modified = false + sparkContextRef.synchronized { + modified = sparkContextRef.compareAndSet(null, sc) + sparkContextRef.notifyAll() + } + + // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do System.exit + // Should not really have to do this, but it helps yarn to evict resources earlier. + // not to mention, prevent Client declaring failure even though we exit'ed properly. + if (modified) { + Runtime.getRuntime().addShutdownHook(new Thread with Logging { + // This is not just to log, but also to ensure that log system is initialized for this instance when we actually are 'run' + logInfo("Adding shutdown hook for context " + sc) + override def run() { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() + // best case ... + for (master <- applicationMasters) master.finishApplicationMaster + } + } ) + } + + // Wait for initialization to complete and atleast 'some' nodes can get allocated + yarnAllocatorLoop.synchronized { + while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT){ + yarnAllocatorLoop.wait(1000L) + } + } + modified + } + + def main(argStrings: Array[String]) { + val args = new ApplicationMasterArguments(argStrings) + new ApplicationMaster(args).run() + } +} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala new file mode 100644 index 0000000000000..dc89125d8184a --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -0,0 +1,78 @@ +package spark.deploy.yarn + +import spark.util.IntParam +import collection.mutable.ArrayBuffer + +class ApplicationMasterArguments(val args: Array[String]) { + var userJar: String = null + var userClass: String = null + var userArgs: Seq[String] = Seq[String]() + var workerMemory = 1024 + var workerCores = 1 + var numWorkers = 2 + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + val userArgsBuffer = new ArrayBuffer[String]() + + var args = inputArgs + + while (! args.isEmpty) { + + args match { + case ("--jar") :: value :: tail => + userJar = value + args = tail + + case ("--class") :: value :: tail => + userClass = value + args = tail + + case ("--args") :: value :: tail => + userArgsBuffer += value + args = tail + + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + + case ("--worker-memory") :: IntParam(value) :: tail => + workerMemory = value + args = tail + + case ("--worker-cores") :: IntParam(value) :: tail => + workerCores = value + args = tail + + case Nil => + if (userJar == null || userClass == null) { + printUsageAndExit(1) + } + + case _ => + printUsageAndExit(1, args) + } + } + + userArgs = userArgsBuffer.readOnly + } + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + if (unknownParam != null) { + System.err.println("Unknown/unsupported param " + unknownParam) + } + System.err.println( + "Usage: spark.deploy.yarn.ApplicationMaster [options] \n" + + "Options:\n" + + " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --class CLASS_NAME Name of your application's main class (required)\n" + + " --args ARGS Arguments to be passed to your application's main class.\n" + + " Mutliple invocations are possible, each will be passed in order.\n" + + " Note that first argument will ALWAYS be yarn-standalone : will be added if missing.\n" + + " --num-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") + System.exit(exitCode) + } +} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala new file mode 100644 index 0000000000000..7fa67405798cd --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -0,0 +1,326 @@ +package spark.deploy.yarn + +import java.net.{InetSocketAddress, URI} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import scala.collection.mutable.HashMap +import scala.collection.JavaConversions._ +import spark.{Logging, Utils} +import org.apache.hadoop.yarn.util.{Apps, Records, ConverterUtils} +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import spark.deploy.SparkHadoopUtil + +class Client(conf: Configuration, args: ClientArguments) extends Logging { + + def this(args: ClientArguments) = this(new Configuration(), args) + + var applicationsManager: ClientRMProtocol = null + var rpc: YarnRPC = YarnRPC.create(conf) + val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + def run() { + connectToASM() + logClusterResourceDetails() + + val newApp = getNewApplication() + val appId = newApp.getApplicationId() + + verifyClusterResources(newApp) + val appContext = createApplicationSubmissionContext(appId) + val localResources = prepareLocalResources(appId, "spark") + val env = setupLaunchEnv(localResources) + val amContainer = createContainerLaunchContext(newApp, localResources, env) + + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(amContainer) + appContext.setUser(args.amUser) + + submitApp(appContext) + + monitorApplication(appId) + System.exit(0) + } + + + def connectToASM() { + val rmAddress: InetSocketAddress = NetUtils.createSocketAddr( + yarnConf.get(YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS) + ) + logInfo("Connecting to ResourceManager at" + rmAddress) + applicationsManager = rpc.getProxy(classOf[ClientRMProtocol], rmAddress, conf) + .asInstanceOf[ClientRMProtocol] + } + + def logClusterResourceDetails() { + val clusterMetrics: YarnClusterMetrics = getYarnClusterMetrics + logInfo("Got Cluster metric info from ASM, numNodeManagers=" + clusterMetrics.getNumNodeManagers) + +/* + val clusterNodeReports: List[NodeReport] = getNodeReports + logDebug("Got Cluster node info from ASM") + for (node <- clusterNodeReports) { + logDebug("Got node report from ASM for, nodeId=" + node.getNodeId + ", nodeAddress=" + node.getHttpAddress + + ", nodeRackName=" + node.getRackName + ", nodeNumContainers=" + node.getNumContainers + ", nodeHealthStatus=" + node.getNodeHealthStatus) + } +*/ + + val queueInfo: QueueInfo = getQueueInfo(args.amQueue) + logInfo("Queue info .. queueName=" + queueInfo.getQueueName + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity + + ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size + + ", queueChildQueueCount=" + queueInfo.getChildQueues.size) + } + + def getYarnClusterMetrics: YarnClusterMetrics = { + val request: GetClusterMetricsRequest = Records.newRecord(classOf[GetClusterMetricsRequest]) + val response: GetClusterMetricsResponse = applicationsManager.getClusterMetrics(request) + return response.getClusterMetrics + } + + def getNodeReports: List[NodeReport] = { + val request: GetClusterNodesRequest = Records.newRecord(classOf[GetClusterNodesRequest]) + val response: GetClusterNodesResponse = applicationsManager.getClusterNodes(request) + return response.getNodeReports.toList + } + + def getQueueInfo(queueName: String): QueueInfo = { + val request: GetQueueInfoRequest = Records.newRecord(classOf[GetQueueInfoRequest]) + request.setQueueName(queueName) + request.setIncludeApplications(true) + request.setIncludeChildQueues(false) + request.setRecursive(false) + Records.newRecord(classOf[GetQueueInfoRequest]) + return applicationsManager.getQueueInfo(request).getQueueInfo + } + + def getNewApplication(): GetNewApplicationResponse = { + logInfo("Requesting new Application") + val request = Records.newRecord(classOf[GetNewApplicationRequest]) + val response = applicationsManager.getNewApplication(request) + logInfo("Got new ApplicationId: " + response.getApplicationId()) + return response + } + + def verifyClusterResources(app: GetNewApplicationResponse) = { + val maxMem = app.getMaximumResourceCapability().getMemory() + logInfo("Max mem capabililty of resources in this cluster " + maxMem) + + // If the cluster does not have enough memory resources, exit. + val requestedMem = (args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + args.numWorkers * args.workerMemory + if (requestedMem > maxMem) { + logError("Cluster cannot satisfy memory resource request of " + requestedMem) + System.exit(1) + } + } + + def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { + logInfo("Setting up application submission context for ASM") + val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) + appContext.setApplicationId(appId) + appContext.setApplicationName("Spark") + return appContext + } + + def prepareLocalResources(appId: ApplicationId, appName: String): HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + val locaResources = HashMap[String, LocalResource]() + // Upload Spark and the application JAR to the remote file system + // Add them as local resources to the AM + val fs = FileSystem.get(conf) + Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> System.getenv("SPARK_LOG4J_CONF")) + .foreach { case(destName, _localPath) => + val localPath: String = if (_localPath != null) _localPath.trim() else "" + if (! localPath.isEmpty()) { + val src = new Path(localPath) + val pathSuffix = appName + "/" + appId.getId() + destName + val dst = new Path(fs.getHomeDirectory(), pathSuffix) + logInfo("Uploading " + src + " to " + dst) + fs.copyFromLocalFile(false, true, src, dst) + val destStatus = fs.getFileStatus(dst) + + val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + amJarRsrc.setType(LocalResourceType.FILE) + amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(dst)) + amJarRsrc.setTimestamp(destStatus.getModificationTime()) + amJarRsrc.setSize(destStatus.getLen()) + locaResources(destName) = amJarRsrc + } + } + return locaResources + } + + def setupLaunchEnv(localResources: HashMap[String, LocalResource]): HashMap[String, String] = { + logInfo("Setting up the launch environment") + val log4jConfLocalRes = localResources.getOrElse("log4j.properties", null) + + val env = new HashMap[String, String]() + Apps.addToEnvironment(env, Environment.USER.name, args.amUser) + + // If log4j present, ensure ours overrides all others + if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*") + Client.populateHadoopClasspath(yarnConf, env) + SparkHadoopUtil.setYarnMode(env) + env("SPARK_YARN_JAR_PATH") = + localResources("spark.jar").getResource().getScheme.toString() + "://" + + localResources("spark.jar").getResource().getFile().toString() + env("SPARK_YARN_JAR_TIMESTAMP") = localResources("spark.jar").getTimestamp().toString() + env("SPARK_YARN_JAR_SIZE") = localResources("spark.jar").getSize().toString() + + env("SPARK_YARN_USERJAR_PATH") = + localResources("app.jar").getResource().getScheme.toString() + "://" + + localResources("app.jar").getResource().getFile().toString() + env("SPARK_YARN_USERJAR_TIMESTAMP") = localResources("app.jar").getTimestamp().toString() + env("SPARK_YARN_USERJAR_SIZE") = localResources("app.jar").getSize().toString() + + if (log4jConfLocalRes != null) { + env("SPARK_YARN_LOG4J_PATH") = + log4jConfLocalRes.getResource().getScheme.toString() + "://" + log4jConfLocalRes.getResource().getFile().toString() + env("SPARK_YARN_LOG4J_TIMESTAMP") = log4jConfLocalRes.getTimestamp().toString() + env("SPARK_YARN_LOG4J_SIZE") = log4jConfLocalRes.getSize().toString() + } + + // Add each SPARK-* key to the environment + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + return env + } + + def userArgsToString(clientArgs: ClientArguments): String = { + val prefix = " --args " + val args = clientArgs.userArgs + val retval = new StringBuilder() + for (arg <- args){ + retval.append(prefix).append(" '").append(arg).append("' ") + } + + retval.toString + } + + def createContainerLaunchContext(newApp: GetNewApplicationResponse, + localResources: HashMap[String, LocalResource], + env: HashMap[String, String]): ContainerLaunchContext = { + logInfo("Setting up container launch context") + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + amContainer.setLocalResources(localResources) + amContainer.setEnvironment(env) + + val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() + + var amMemory = ((args.amMemory / minResMemory) * minResMemory) + + (if (0 != (args.amMemory % minResMemory)) minResMemory else 0) - YarnAllocationHandler.MEMORY_OVERHEAD + + // Extra options for the JVM + var JAVA_OPTS = "" + + // Add Xmx for am memory + JAVA_OPTS += "-Xmx" + amMemory + "m " + + // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same + // node, spark gc effects all other containers performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is + // limited to subset of cores on a node. + if (env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))) { + // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tenant machines + JAVA_OPTS += " -XX:+UseConcMarkSweepGC " + JAVA_OPTS += " -XX:+CMSIncrementalMode " + JAVA_OPTS += " -XX:+CMSIncrementalPacing " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + } + if (env.isDefinedAt("SPARK_JAVA_OPTS")) { + JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + } + + // Command for the ApplicationMaster + val commands = List[String]("java " + + " -server " + + JAVA_OPTS + + " spark.deploy.yarn.ApplicationMaster" + + " --class " + args.userClass + + " --jar " + args.userJar + + userArgsToString(args) + + " --worker-memory " + args.workerMemory + + " --worker-cores " + args.workerCores + + " --num-workers " + args.numWorkers + + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + logInfo("Command for the ApplicationMaster: " + commands(0)) + amContainer.setCommands(commands) + + val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] + // Memory for the ApplicationMaster + capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + amContainer.setResource(capability) + + return amContainer + } + + def submitApp(appContext: ApplicationSubmissionContext) = { + // Create the request to send to the applications manager + val appRequest = Records.newRecord(classOf[SubmitApplicationRequest]) + .asInstanceOf[SubmitApplicationRequest] + appRequest.setApplicationSubmissionContext(appContext) + // Submit the application to the applications manager + logInfo("Submitting application to ASM") + applicationsManager.submitApplication(appRequest) + } + + def monitorApplication(appId: ApplicationId): Boolean = { + while(true) { + Thread.sleep(1000) + val reportRequest = Records.newRecord(classOf[GetApplicationReportRequest]) + .asInstanceOf[GetApplicationReportRequest] + reportRequest.setApplicationId(appId) + val reportResponse = applicationsManager.getApplicationReport(reportRequest) + val report = reportResponse.getApplicationReport() + + logInfo("Application report from ASM: \n" + + "\t application identifier: " + appId.toString() + "\n" + + "\t appId: " + appId.getId() + "\n" + + "\t clientToken: " + report.getClientToken() + "\n" + + "\t appDiagnostics: " + report.getDiagnostics() + "\n" + + "\t appMasterHost: " + report.getHost() + "\n" + + "\t appQueue: " + report.getQueue() + "\n" + + "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + + "\t appStartTime: " + report.getStartTime() + "\n" + + "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + + "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + + "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + + "\t appUser: " + report.getUser() + ) + + val state = report.getYarnApplicationState() + val dsStatus = report.getFinalApplicationStatus() + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + return true + } + } + return true + } +} + +object Client { + def main(argStrings: Array[String]) { + val args = new ClientArguments(argStrings) + SparkHadoopUtil.setYarnMode() + new Client(args).run + } + + // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps + def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { + for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) + } + } +} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala new file mode 100644 index 0000000000000..53b305f7df0e7 --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -0,0 +1,104 @@ +package spark.deploy.yarn + +import spark.util.MemoryParam +import spark.util.IntParam +import collection.mutable.{ArrayBuffer, HashMap} +import spark.scheduler.{InputFormatInfo, SplitInfo} + +// TODO: Add code and support for ensuring that yarn resource 'asks' are location aware ! +class ClientArguments(val args: Array[String]) { + var userJar: String = null + var userClass: String = null + var userArgs: Seq[String] = Seq[String]() + var workerMemory = 1024 + var workerCores = 1 + var numWorkers = 2 + var amUser = System.getProperty("user.name") + var amQueue = System.getProperty("QUEUE", "default") + var amMemory: Int = 512 + // TODO + var inputFormatInfo: List[InputFormatInfo] = null + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() + val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]() + + var args = inputArgs + + while (! args.isEmpty) { + + args match { + case ("--jar") :: value :: tail => + userJar = value + args = tail + + case ("--class") :: value :: tail => + userClass = value + args = tail + + case ("--args") :: value :: tail => + userArgsBuffer += value + args = tail + + case ("--master-memory") :: MemoryParam(value) :: tail => + amMemory = value + args = tail + + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + + case ("--worker-memory") :: MemoryParam(value) :: tail => + workerMemory = value + args = tail + + case ("--worker-cores") :: IntParam(value) :: tail => + workerCores = value + args = tail + + case ("--user") :: value :: tail => + amUser = value + args = tail + + case ("--queue") :: value :: tail => + amQueue = value + args = tail + + case Nil => + if (userJar == null || userClass == null) { + printUsageAndExit(1) + } + + case _ => + printUsageAndExit(1, args) + } + } + + userArgs = userArgsBuffer.readOnly + inputFormatInfo = inputFormatMap.values.toList + } + + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + if (unknownParam != null) { + System.err.println("Unknown/unsupported param " + unknownParam) + } + System.err.println( + "Usage: spark.deploy.yarn.Client [options] \n" + + "Options:\n" + + " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --class CLASS_NAME Name of your application's main class (required)\n" + + " --args ARGS Arguments to be passed to your application's main class.\n" + + " Mutliple invocations are possible, each will be passed in order.\n" + + " Note that first argument will ALWAYS be yarn-standalone : will be added if missing.\n" + + " --num-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + + " --user USERNAME Run the ApplicationMaster as a different user\n" + ) + System.exit(exitCode) + } + +} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala new file mode 100644 index 0000000000000..5688f1ab66603 --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -0,0 +1,171 @@ +package spark.deploy.yarn + +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment + +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap + +import spark.{Logging, Utils} + +class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String, + slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) + extends Runnable with Logging { + + var rpc: YarnRPC = YarnRPC.create(conf) + var cm: ContainerManager = null + val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + def run = { + logInfo("Starting Worker Container") + cm = connectToCM + startContainer + } + + def startContainer = { + logInfo("Setting up ContainerLaunchContext") + + val ctx = Records.newRecord(classOf[ContainerLaunchContext]) + .asInstanceOf[ContainerLaunchContext] + + ctx.setContainerId(container.getId()) + ctx.setResource(container.getResource()) + val localResources = prepareLocalResources + ctx.setLocalResources(localResources) + + val env = prepareEnvironment + ctx.setEnvironment(env) + + // Extra options for the JVM + var JAVA_OPTS = "" + // Set the JVM memory + val workerMemoryString = workerMemory + "m" + JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " " + if (env.isDefinedAt("SPARK_JAVA_OPTS")) { + JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + } + // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same + // node, spark gc effects all other containers performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is + // limited to subset of cores on a node. +/* + else { + // If no java_opts specified, default to using -XX:+CMSIncrementalMode + // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tennent machines + // The options are based on + // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline + JAVA_OPTS += " -XX:+UseConcMarkSweepGC " + JAVA_OPTS += " -XX:+CMSIncrementalMode " + JAVA_OPTS += " -XX:+CMSIncrementalPacing " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + } +*/ + + ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) + val commands = List[String]("java " + + " -server " + + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. + // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state. + // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ? + " -XX:OnOutOfMemoryError='kill %p' " + + JAVA_OPTS + + " spark.executor.StandaloneExecutorBackend " + + masterAddress + " " + + slaveId + " " + + hostname + " " + + workerCores + + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + logInfo("Setting up worker with commands: " + commands) + ctx.setCommands(commands) + + // Send the start request to the ContainerManager + val startReq = Records.newRecord(classOf[StartContainerRequest]) + .asInstanceOf[StartContainerRequest] + startReq.setContainerLaunchContext(ctx) + cm.startContainer(startReq) + } + + + def prepareLocalResources: HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + val locaResources = HashMap[String, LocalResource]() + + // Spark JAR + val sparkJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + sparkJarResource.setType(LocalResourceType.FILE) + sparkJarResource.setVisibility(LocalResourceVisibility.APPLICATION) + sparkJarResource.setResource(ConverterUtils.getYarnUrlFromURI( + new URI(System.getenv("SPARK_YARN_JAR_PATH")))) + sparkJarResource.setTimestamp(System.getenv("SPARK_YARN_JAR_TIMESTAMP").toLong) + sparkJarResource.setSize(System.getenv("SPARK_YARN_JAR_SIZE").toLong) + locaResources("spark.jar") = sparkJarResource + // User JAR + val userJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + userJarResource.setType(LocalResourceType.FILE) + userJarResource.setVisibility(LocalResourceVisibility.APPLICATION) + userJarResource.setResource(ConverterUtils.getYarnUrlFromURI( + new URI(System.getenv("SPARK_YARN_USERJAR_PATH")))) + userJarResource.setTimestamp(System.getenv("SPARK_YARN_USERJAR_TIMESTAMP").toLong) + userJarResource.setSize(System.getenv("SPARK_YARN_USERJAR_SIZE").toLong) + locaResources("app.jar") = userJarResource + + // Log4j conf - if available + if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) { + val log4jConfResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + log4jConfResource.setType(LocalResourceType.FILE) + log4jConfResource.setVisibility(LocalResourceVisibility.APPLICATION) + log4jConfResource.setResource(ConverterUtils.getYarnUrlFromURI( + new URI(System.getenv("SPARK_YARN_LOG4J_PATH")))) + log4jConfResource.setTimestamp(System.getenv("SPARK_YARN_LOG4J_TIMESTAMP").toLong) + log4jConfResource.setSize(System.getenv("SPARK_YARN_LOG4J_SIZE").toLong) + locaResources("log4j.properties") = log4jConfResource + } + + + logInfo("Prepared Local resources " + locaResources) + return locaResources + } + + def prepareEnvironment: HashMap[String, String] = { + val env = new HashMap[String, String]() + // should we add this ? + Apps.addToEnvironment(env, Environment.USER.name, Utils.getUserNameFromEnvironment()) + + // If log4j present, ensure ours overrides all others + if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) { + // Which is correct ? + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./log4j.properties") + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") + } + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*") + Client.populateHadoopClasspath(yarnConf, env) + + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + return env + } + + def connectToCM: ContainerManager = { + val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() + val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) + logInfo("Connecting to ContainerManager at " + cmHostPortStr) + return rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager] + } + +} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala new file mode 100644 index 0000000000000..cac9dab401aa7 --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala @@ -0,0 +1,547 @@ +package spark.deploy.yarn + +import spark.{Logging, Utils} +import spark.scheduler.SplitInfo +import scala.collection +import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container} +import spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend} +import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} +import org.apache.hadoop.yarn.util.{RackResolver, Records} +import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} +import java.util.concurrent.atomic.AtomicInteger +import org.apache.hadoop.yarn.api.AMRMProtocol +import collection.JavaConversions._ +import collection.mutable.{ArrayBuffer, HashMap, HashSet} +import org.apache.hadoop.conf.Configuration +import java.util.{Collections, Set => JSet} +import java.lang.{Boolean => JBoolean} + +object AllocationType extends Enumeration ("HOST", "RACK", "ANY") { + type AllocationType = Value + val HOST, RACK, ANY = Value +} + +// too many params ? refactor it 'somehow' ? +// needs to be mt-safe +// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive : should make it +// more proactive and decoupled. +// Note that right now, we assume all node asks as uniform in terms of capabilities and priority +// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for more info +// on how we are requesting for containers. +private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceManager: AMRMProtocol, + val appAttemptId: ApplicationAttemptId, + val maxWorkers: Int, val workerMemory: Int, val workerCores: Int, + val preferredHostToCount: Map[String, Int], + val preferredRackToCount: Map[String, Int]) + extends Logging { + + + // These three are locked on allocatedHostToContainersMap. Complementary data structures + // allocatedHostToContainersMap : containers which are running : host, Set + // allocatedContainerToHostMap: container to host mapping + private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]() + private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() + // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an allocated node) + // As with the two data structures above, tightly coupled with them, and to be locked on allocatedHostToContainersMap + private val allocatedRackCount = new HashMap[String, Int]() + + // containers which have been released. + private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() + // containers to be released in next request to RM + private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] + + private val numWorkersRunning = new AtomicInteger() + // Used to generate a unique id per worker + private val workerIdCounter = new AtomicInteger() + private val lastResponseId = new AtomicInteger() + + def getNumWorkersRunning: Int = numWorkersRunning.intValue + + + def isResourceConstraintSatisfied(container: Container): Boolean = { + container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + } + + def allocateContainers(workersToRequest: Int) { + // We need to send the request only once from what I understand ... but for now, not modifying this much. + + // Keep polling the Resource Manager for containers + val amResp = allocateWorkerResources(workersToRequest).getAMResponse + + val _allocatedContainers = amResp.getAllocatedContainers() + if (_allocatedContainers.size > 0) { + + + logDebug("Allocated " + _allocatedContainers.size + " containers, current count " + + numWorkersRunning.get() + ", to-be-released " + releasedContainerList + + ", pendingReleaseContainers : " + pendingReleaseContainers) + logDebug("Cluster Resources: " + amResp.getAvailableResources) + + val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() + + // ignore if not satisfying constraints { + for (container <- _allocatedContainers) { + if (isResourceConstraintSatisfied(container)) { + // allocatedContainers += container + + val host = container.getNodeId.getHost + val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]()) + + containers += container + } + // Add all ignored containers to released list + else releasedContainerList.add(container.getId()) + } + + // Find the appropriate containers to use + // Slightly non trivial groupBy I guess ... + val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (candidateHost <- hostToContainers.keySet) + { + val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) + val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) + + var remainingContainers = hostToContainers.get(candidateHost).getOrElse(null) + assert(remainingContainers != null) + + if (requiredHostCount >= remainingContainers.size){ + // Since we got <= required containers, add all to dataLocalContainers + dataLocalContainers.put(candidateHost, remainingContainers) + // all consumed + remainingContainers = null + } + else if (requiredHostCount > 0) { + // container list has more containers than we need for data locality. + // Split into two : data local container count of (remainingContainers.size - requiredHostCount) + // and rest as remainingContainer + val (dataLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredHostCount) + dataLocalContainers.put(candidateHost, dataLocal) + // remainingContainers = remaining + + // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : + // add remaining to release list. If we have insufficient containers, next allocation cycle + // will reallocate (but wont treat it as data local) + for (container <- remaining) releasedContainerList.add(container.getId()) + remainingContainers = null + } + + // now rack local + if (remainingContainers != null){ + val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + + if (rack != null){ + val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + rackLocalContainers.get(rack).getOrElse(List()).size + + + if (requiredRackCount >= remainingContainers.size){ + // Add all to dataLocalContainers + dataLocalContainers.put(rack, remainingContainers) + // all consumed + remainingContainers = null + } + else if (requiredRackCount > 0) { + // container list has more containers than we need for data locality. + // Split into two : data local container count of (remainingContainers.size - requiredRackCount) + // and rest as remainingContainer + val (rackLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredRackCount) + val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, new ArrayBuffer[Container]()) + + existingRackLocal ++= rackLocal + remainingContainers = remaining + } + } + } + + // If still not consumed, then it is off rack host - add to that list. + if (remainingContainers != null){ + offRackContainers.put(candidateHost, remainingContainers) + } + } + + // Now that we have split the containers into various groups, go through them in order : + // first host local, then rack local and then off rack (everything else). + // Note that the list we create below tries to ensure that not all containers end up within a host + // if there are sufficiently large number of hosts/containers. + + val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size) + allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers) + allocatedContainers ++= ClusterScheduler.prioritizeContainers(rackLocalContainers) + allocatedContainers ++= ClusterScheduler.prioritizeContainers(offRackContainers) + + // Run each of the allocated containers + for (container <- allocatedContainers) { + val numWorkersRunningNow = numWorkersRunning.incrementAndGet() + val workerHostname = container.getNodeId.getHost + val containerId = container.getId + + assert (container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + + if (numWorkersRunningNow > maxWorkers) { + logInfo("Ignoring container " + containerId + " at host " + workerHostname + + " .. we already have required number of containers") + releasedContainerList.add(containerId) + // reset counter back to old value. + numWorkersRunning.decrementAndGet() + } + else { + // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter) + val workerId = workerIdCounter.incrementAndGet().toString + val masterUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.master.host"), System.getProperty("spark.master.port"), + StandaloneSchedulerBackend.ACTOR_NAME) + + logInfo("launching container on " + containerId + " host " + workerHostname) + // just to be safe, simply remove it from pendingReleaseContainers. Should not be there, but .. + pendingReleaseContainers.remove(containerId) + + val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) + allocatedHostToContainersMap.synchronized { + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, new HashSet[ContainerId]()) + + containerSet += containerId + allocatedContainerToHostMap.put(containerId, workerHostname) + if (rack != null) allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + } + + new Thread( + new WorkerRunnable(container, conf, masterUrl, workerId, + workerHostname, workerMemory, workerCores) + ).start() + } + } + logDebug("After allocated " + allocatedContainers.size + " containers (orig : " + + _allocatedContainers.size + "), current count " + numWorkersRunning.get() + + ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers) + } + + + val completedContainers = amResp.getCompletedContainersStatuses() + if (completedContainers.size > 0){ + logDebug("Completed " + completedContainers.size + " containers, current count " + numWorkersRunning.get() + + ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers) + + for (completedContainer <- completedContainers){ + val containerId = completedContainer.getContainerId + + // Was this released by us ? If yes, then simply remove from containerSet and move on. + if (pendingReleaseContainers.containsKey(containerId)) { + pendingReleaseContainers.remove(containerId) + } + else { + // simply decrement count - next iteration of ReporterThread will take care of allocating ! + numWorkersRunning.decrementAndGet() + logInfo("Container completed ? nodeId: " + containerId + ", state " + completedContainer.getState + + " httpaddress: " + completedContainer.getDiagnostics) + } + + allocatedHostToContainersMap.synchronized { + if (allocatedContainerToHostMap.containsKey(containerId)) { + val host = allocatedContainerToHostMap.get(containerId).getOrElse(null) + assert (host != null) + + val containerSet = allocatedHostToContainersMap.get(host).getOrElse(null) + assert (containerSet != null) + + containerSet -= containerId + if (containerSet.isEmpty) allocatedHostToContainersMap.remove(host) + else allocatedHostToContainersMap.update(host, containerSet) + + allocatedContainerToHostMap -= containerId + + // doing this within locked context, sigh ... move to outside ? + val rack = YarnAllocationHandler.lookupRack(conf, host) + if (rack != null) { + val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 + if (rackCount > 0) allocatedRackCount.put(rack, rackCount) + else allocatedRackCount.remove(rack) + } + } + } + } + logDebug("After completed " + completedContainers.size + " containers, current count " + + numWorkersRunning.get() + ", to-be-released " + releasedContainerList + + ", pendingReleaseContainers : " + pendingReleaseContainers) + } + } + + def createRackResourceRequests(hostContainers: List[ResourceRequest]): List[ResourceRequest] = { + // First generate modified racks and new set of hosts under it : then issue requests + val rackToCounts = new HashMap[String, Int]() + + // Within this lock - used to read/write to the rack related maps too. + for (container <- hostContainers) { + val candidateHost = container.getHostName + val candidateNumContainers = container.getNumContainers + assert(YarnAllocationHandler.ANY_HOST != candidateHost) + + val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += candidateNumContainers + rackToCounts.put(rack, count) + } + } + + val requestedContainers: ArrayBuffer[ResourceRequest] = + new ArrayBuffer[ResourceRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts){ + requestedContainers += + createResourceRequest(AllocationType.RACK, rack, count, YarnAllocationHandler.PRIORITY) + } + + requestedContainers.toList + } + + def allocatedContainersOnHost(host: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + } + retval + } + + def allocatedContainersOnRack(rack: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedRackCount.getOrElse(rack, 0) + } + retval + } + + private def allocateWorkerResources(numWorkers: Int): AllocateResponse = { + + var resourceRequests: List[ResourceRequest] = null + + // default. + if (numWorkers <= 0 || preferredHostToCount.isEmpty) { + logDebug("numWorkers: " + numWorkers + ", host preferences ? " + preferredHostToCount.isEmpty) + resourceRequests = List( + createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)) + } + else { + // request for all hosts in preferred nodes and for numWorkers - + // candidates.size, request by default allocation policy. + val hostContainerRequests: ArrayBuffer[ResourceRequest] = + new ArrayBuffer[ResourceRequest](preferredHostToCount.size) + for ((candidateHost, candidateCount) <- preferredHostToCount) { + val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) + + if (requiredCount > 0) { + hostContainerRequests += + createResourceRequest(AllocationType.HOST, candidateHost, requiredCount, YarnAllocationHandler.PRIORITY) + } + } + val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(hostContainerRequests.toList) + + val anyContainerRequests: ResourceRequest = + createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY) + + val containerRequests: ArrayBuffer[ResourceRequest] = + new ArrayBuffer[ResourceRequest](hostContainerRequests.size() + rackContainerRequests.size() + 1) + + containerRequests ++= hostContainerRequests + containerRequests ++= rackContainerRequests + containerRequests += anyContainerRequests + + resourceRequests = containerRequests.toList + } + + val req = Records.newRecord(classOf[AllocateRequest]) + req.setResponseId(lastResponseId.incrementAndGet) + req.setApplicationAttemptId(appAttemptId) + + req.addAllAsks(resourceRequests) + + val releasedContainerList = createReleasedContainerList() + req.addAllReleases(releasedContainerList) + + + + if (numWorkers > 0) { + logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.") + } + else { + logDebug("Empty allocation req .. release : " + releasedContainerList) + } + + for (req <- resourceRequests) { + logInfo("rsrcRequest ... host : " + req.getHostName + ", numContainers : " + req.getNumContainers + + ", p = " + req.getPriority().getPriority + ", capability: " + req.getCapability) + } + resourceManager.allocate(req) + } + + + private def createResourceRequest(requestType: AllocationType.AllocationType, + resource:String, numWorkers: Int, priority: Int): ResourceRequest = { + + // If hostname specified, we need atleast two requests - node local and rack local. + // There must be a third request - which is ANY : that will be specially handled. + requestType match { + case AllocationType.HOST => { + assert (YarnAllocationHandler.ANY_HOST != resource) + + val hostname = resource + val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority) + + // add to host->rack mapping + YarnAllocationHandler.populateRackInfo(conf, hostname) + + nodeLocal + } + + case AllocationType.RACK => { + val rack = resource + createResourceRequestImpl(rack, numWorkers, priority) + } + + case AllocationType.ANY => { + createResourceRequestImpl(YarnAllocationHandler.ANY_HOST, numWorkers, priority) + } + + case _ => throw new IllegalArgumentException("Unexpected/unsupported request type .. " + requestType) + } + } + + private def createResourceRequestImpl(hostname:String, numWorkers: Int, priority: Int): ResourceRequest = { + + val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) + val memCapability = Records.newRecord(classOf[Resource]) + // There probably is some overhead here, let's reserve a bit more memory. + memCapability.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + rsrcRequest.setCapability(memCapability) + + val pri = Records.newRecord(classOf[Priority]) + pri.setPriority(priority) + rsrcRequest.setPriority(pri) + + rsrcRequest.setHostName(hostname) + + rsrcRequest.setNumContainers(java.lang.Math.max(numWorkers, 0)) + rsrcRequest + } + + def createReleasedContainerList(): ArrayBuffer[ContainerId] = { + + val retval = new ArrayBuffer[ContainerId](1) + // iterator on COW list ... + for (container <- releasedContainerList.iterator()){ + retval += container + } + // remove from the original list. + if (! retval.isEmpty) { + releasedContainerList.removeAll(retval) + for (v <- retval) pendingReleaseContainers.put(v, true) + logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + + pendingReleaseContainers) + } + + retval + } +} + +object YarnAllocationHandler { + + val ANY_HOST = "*" + // all requests are issued with same priority : we do not (yet) have any distinction between + // request types (like map/reduce in hadoop for example) + val PRIORITY = 1 + + // Additional memory overhead - in mb + val MEMORY_OVERHEAD = 384 + + // host to rack map - saved from allocation requests + // We are expecting this not to change. + // Note that it is possible for this to change : and RM will indicate that to us via update + // response to allocate. But we are punting on handling that for now. + private val hostToRack = new ConcurrentHashMap[String, String]() + private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() + + def newAllocator(conf: Configuration, + resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { + + val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) + + + new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, + args.workerMemory, args.workerCores, hostToCount, rackToCount) + } + + def newAllocator(conf: Configuration, + resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, + maxWorkers: Int, workerMemory: Int, workerCores: Int, + map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { + + val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) + + new YarnAllocationHandler(conf, resourceManager, appAttemptId, maxWorkers, + workerMemory, workerCores, hostToCount, rackToCount) + } + + // A simple method to copy the split info map. + private def generateNodeToWeight(conf: Configuration, input: collection.Map[String, collection.Set[SplitInfo]]) : + // host to count, rack to count + (Map[String, Int], Map[String, Int]) = { + + if (input == null) return (Map[String, Int](), Map[String, Int]()) + + val hostToCount = new HashMap[String, Int] + val rackToCount = new HashMap[String, Int] + + for ((host, splits) <- input) { + val hostCount = hostToCount.getOrElse(host, 0) + hostToCount.put(host, hostCount + splits.size) + + val rack = lookupRack(conf, host) + if (rack != null){ + val rackCount = rackToCount.getOrElse(host, 0) + rackToCount.put(host, rackCount + splits.size) + } + } + + (hostToCount.toMap, rackToCount.toMap) + } + + def lookupRack(conf: Configuration, host: String): String = { + if (! hostToRack.contains(host)) populateRackInfo(conf, host) + hostToRack.get(host) + } + + def fetchCachedHostsForRack(rack: String): Option[Set[String]] = { + val set = rackToHostSet.get(rack) + if (set == null) return None + + // No better way to get a Set[String] from JSet ? + val convertedSet: collection.mutable.Set[String] = set + Some(convertedSet.toSet) + } + + def populateRackInfo(conf: Configuration, hostname: String) { + Utils.checkHost(hostname) + + if (!hostToRack.containsKey(hostname)) { + // If there are repeated failures to resolve, all to an ignore list ? + val rackInfo = RackResolver.resolve(conf, hostname) + if (rackInfo != null && rackInfo.getNetworkLocation != null) { + val rack = rackInfo.getNetworkLocation + hostToRack.put(hostname, rack) + if (! rackToHostSet.containsKey(rack)) { + rackToHostSet.putIfAbsent(rack, Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) + } + rackToHostSet.get(rack).add(hostname) + + // Since RackResolver caches, we are disabling this for now ... + } /* else { + // right ? Else we will keep calling rack resolver in case we cant resolve rack info ... + hostToRack.put(hostname, null) + } */ + } + } +} diff --git a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala new file mode 100644 index 0000000000000..ed732d36bfc7c --- /dev/null +++ b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -0,0 +1,42 @@ +package spark.scheduler.cluster + +import spark._ +import spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} +import org.apache.hadoop.conf.Configuration + +/** + * + * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of ApplicationMaster, etc is done + */ +private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) { + + def this(sc: SparkContext) = this(sc, new Configuration()) + + // Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate + // Note that only the first creation of SparkContext influences (and ideally, there must be only one SparkContext, right ?) + // Subsequent creations are ignored - since nodes are already allocated by then. + + + // By default, rack is unknown + override def getRackForHost(hostPort: String): Option[String] = { + val host = Utils.parseHostPort(hostPort)._1 + val retval = YarnAllocationHandler.lookupRack(conf, host) + if (retval != null) Some(retval) else None + } + + // By default, if rack is unknown, return nothing + override def getCachedHostsForRack(rack: String): Option[Set[String]] = { + if (rack == None || rack == null) return None + + YarnAllocationHandler.fetchCachedHostsForRack(rack) + } + + override def postStartHook() { + val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) + if (sparkContextInitialized){ + // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + Thread.sleep(3000L) + } + logInfo("YarnClusterScheduler.postStartHook done") + } +} diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala new file mode 100644 index 0000000000000..d4badbc5c4254 --- /dev/null +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -0,0 +1,18 @@ +package spark.deploy + +/** + * Contains util methods to interact with Hadoop from spark. + */ +object SparkHadoopUtil { + + def getUserNameFromEnvironment(): String = { + // defaulting to -D ... + System.getProperty("user.name") + } + + def runAsUser(func: (Product) => Unit, args: Product) { + + // Add support, if exists - for now, simply run func ! + func(args) + } +} diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala index 98525b99c8420..50d6a1c5c9fb9 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/spark/ClosureCleaner.scala @@ -8,12 +8,20 @@ import scala.collection.mutable.Set import org.objectweb.asm.{ClassReader, MethodVisitor, Type} import org.objectweb.asm.commons.EmptyVisitor import org.objectweb.asm.Opcodes._ +import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream} private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it private def getClassReader(cls: Class[_]): ClassReader = { - new ClassReader(cls.getResourceAsStream( - cls.getName.replaceFirst("^.*\\.", "") + ".class")) + // Copy data over, before delegating to ClassReader - else we can run out of open file handles. + val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" + val resourceStream = cls.getResourceAsStream(className) + // todo: Fixme - continuing with earlier behavior ... + if (resourceStream == null) return new ClassReader(resourceStream) + + val baos = new ByteArrayOutputStream(128) + Utils.copyStream(resourceStream, baos, true) + new ClassReader(new ByteArrayInputStream(baos.toByteArray)) } // Check whether a class represents a Scala closure diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala index a953081d245fa..40b0193f1994f 100644 --- a/core/src/main/scala/spark/FetchFailedException.scala +++ b/core/src/main/scala/spark/FetchFailedException.scala @@ -3,18 +3,25 @@ package spark import spark.storage.BlockManagerId private[spark] class FetchFailedException( - val bmAddress: BlockManagerId, - val shuffleId: Int, - val mapId: Int, - val reduceId: Int, + taskEndReason: TaskEndReason, + message: String, cause: Throwable) extends Exception { - - override def getMessage(): String = - "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId) + + def this (bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int, cause: Throwable) = + this(FetchFailed(bmAddress, shuffleId, mapId, reduceId), + "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId), + cause) + + def this (shuffleId: Int, reduceId: Int, cause: Throwable) = + this(FetchFailed(null, shuffleId, -1, reduceId), + "Unable to fetch locations from master: %d %d".format(shuffleId, reduceId), cause) + + override def getMessage(): String = message + override def getCause(): Throwable = cause - def toTaskEndReason: TaskEndReason = - FetchFailed(bmAddress, shuffleId, mapId, reduceId) + def toTaskEndReason: TaskEndReason = taskEndReason + } diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala index 7c1c1bb1440bc..0fc8c314630bc 100644 --- a/core/src/main/scala/spark/Logging.scala +++ b/core/src/main/scala/spark/Logging.scala @@ -68,6 +68,10 @@ trait Logging { if (log.isErrorEnabled) log.error(msg, throwable) } + protected def isTraceEnabled(): Boolean = { + log.isTraceEnabled + } + // Method for ensuring that logging is initialized, to avoid having multiple // threads do it concurrently (as SLF4J initialization is not thread safe). protected def initLogging() { log } diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 866d630a6d27b..6e9da02893c24 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -1,7 +1,6 @@ package spark import java.io._ -import java.util.concurrent.ConcurrentHashMap import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap @@ -12,8 +11,7 @@ import akka.dispatch._ import akka.pattern.ask import akka.remote._ import akka.util.Duration -import akka.util.Timeout -import akka.util.duration._ + import spark.scheduler.MapStatus import spark.storage.BlockManagerId @@ -40,10 +38,12 @@ private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Ac private[spark] class MapOutputTracker extends Logging { + private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + // Set to the MapOutputTrackerActor living on the driver var trackerActor: ActorRef = _ - var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] + private var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] // Incremented every time a fetch fails so that client nodes know to clear // their cache of map output locations if this happens. @@ -52,7 +52,7 @@ private[spark] class MapOutputTracker extends Logging { // Cache a serialized version of the output statuses for each shuffle to send them out faster var cacheGeneration = generation - val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] + private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup) @@ -60,7 +60,6 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. def askTracker(message: Any): Any = { try { - val timeout = 10.seconds val future = trackerActor.ask(message)(timeout) return Await.result(future, timeout) } catch { @@ -77,10 +76,9 @@ private[spark] class MapOutputTracker extends Logging { } def registerShuffle(shuffleId: Int, numMaps: Int) { - if (mapStatuses.get(shuffleId) != None) { + if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) { throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") } - mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)) } def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) { @@ -101,8 +99,9 @@ private[spark] class MapOutputTracker extends Logging { } def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) { - var array = mapStatuses(shuffleId) - if (array != null) { + var arrayOpt = mapStatuses.get(shuffleId) + if (arrayOpt.isDefined && arrayOpt.get != null) { + var array = arrayOpt.get array.synchronized { if (array(mapId) != null && array(mapId).location == bmAddress) { array(mapId) = null @@ -115,13 +114,14 @@ private[spark] class MapOutputTracker extends Logging { } // Remembers which map output locations are currently being fetched on a worker - val fetching = new HashSet[Int] + private val fetching = new HashSet[Int] // Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { 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 fetching.synchronized { if (fetching.contains(shuffleId)) { // Someone else is fetching it; wait for them to be done @@ -132,31 +132,49 @@ private[spark] class MapOutputTracker extends Logging { case e: InterruptedException => } } - return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, mapStatuses(shuffleId)) - } else { + } + + // Either while we waited the fetch happened successfully, or + // someone fetched it in between the get and the fetching.synchronized. + fetchedStatuses = mapStatuses.get(shuffleId).orNull + if (fetchedStatuses == null) { + // We have to do the fetch, get others to wait for us. fetching += shuffleId } } - // We won the race to fetch the output locs; do so - logInfo("Doing the fetch; tracker actor = " + trackerActor) - val host = System.getProperty("spark.hostname", Utils.localHostName) - // This try-finally prevents hangs due to timeouts: - var fetchedStatuses: Array[MapStatus] = null - try { - val fetchedBytes = - askTracker(GetMapOutputStatuses(shuffleId, host)).asInstanceOf[Array[Byte]] - fetchedStatuses = deserializeStatuses(fetchedBytes) - logInfo("Got the output locations") - mapStatuses.put(shuffleId, fetchedStatuses) - } finally { - fetching.synchronized { - fetching -= shuffleId - fetching.notifyAll() + + if (fetchedStatuses == null) { + // We won the race to fetch the output locs; do so + logInfo("Doing the fetch; tracker actor = " + trackerActor) + val hostPort = Utils.localHostPort() + // This try-finally prevents hangs due to timeouts: + var fetchedStatuses: Array[MapStatus] = null + try { + val fetchedBytes = + askTracker(GetMapOutputStatuses(shuffleId, hostPort)).asInstanceOf[Array[Byte]] + fetchedStatuses = deserializeStatuses(fetchedBytes) + logInfo("Got the output locations") + mapStatuses.put(shuffleId, fetchedStatuses) + } finally { + fetching.synchronized { + fetching -= shuffleId + fetching.notifyAll() + } + } + } + if (fetchedStatuses != null) { + fetchedStatuses.synchronized { + return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) } } - return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) + else{ + throw new FetchFailedException(null, shuffleId, -1, reduceId, + new Exception("Missing all output locations for shuffle " + shuffleId)) + } } else { - return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses) + statuses.synchronized { + return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses) + } } } @@ -194,7 +212,8 @@ private[spark] class MapOutputTracker extends Logging { generationLock.synchronized { if (newGen > generation) { logInfo("Updating generation to " + newGen + " and clearing cache") - mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] + // mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] + mapStatuses.clear() generation = newGen } } @@ -232,10 +251,13 @@ private[spark] class MapOutputTracker extends Logging { // 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 serializeStatuses(statuses: Array[MapStatus]): Array[Byte] = { + private def serializeStatuses(statuses: Array[MapStatus]): Array[Byte] = { val out = new ByteArrayOutputStream val objOut = new ObjectOutputStream(new GZIPOutputStream(out)) - objOut.writeObject(statuses) + // Since statuses can be modified in parallel, sync on it + statuses.synchronized { + objOut.writeObject(statuses) + } objOut.close() out.toByteArray } @@ -243,7 +265,10 @@ private[spark] class MapOutputTracker extends Logging { // Opposite of serializeStatuses. def deserializeStatuses(bytes: Array[Byte]): Array[MapStatus] = { val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes))) - objIn.readObject().asInstanceOf[Array[MapStatus]] + objIn.readObject(). + // // drop all null's from status - not sure why they are occuring though. Causes NPE downstream in slave if present + // comment this out - nulls could be due to missing location ? + asInstanceOf[Array[MapStatus]] // .filter( _ != null ) } } @@ -253,14 +278,11 @@ private[spark] object MapOutputTracker { // Convert an array of MapStatuses to locations and sizes for a given reduce ID. If // any of the statuses is null (indicating a missing location due to a failed mapper), // throw a FetchFailedException. - def convertMapStatuses( + private def convertMapStatuses( shuffleId: Int, reduceId: Int, statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = { - if (statuses == null) { - throw new FetchFailedException(null, shuffleId, -1, reduceId, - new Exception("Missing all output locations for shuffle " + shuffleId)) - } + assert (statuses != null) statuses.map { status => if (status == null) { diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 4957a54c1b8af..e853bce2c4254 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -37,7 +37,7 @@ import spark.partial.PartialResult import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import spark.scheduler._ import spark.scheduler.local.LocalScheduler -import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.BlockManagerUI import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -59,7 +59,10 @@ class SparkContext( val appName: String, val sparkHome: String = null, val jars: Seq[String] = Nil, - val environment: Map[String, String] = Map()) + val environment: Map[String, String] = Map(), + // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc) too. + // This is typically generated from InputFormatInfo.computePreferredLocations .. host, set of data-local splits on host + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = scala.collection.immutable.Map()) extends Logging { // Ensure logging is initialized before we spawn any threads @@ -67,7 +70,7 @@ class SparkContext( // Set Spark driver host and port system properties if (System.getProperty("spark.driver.host") == null) { - System.setProperty("spark.driver.host", Utils.localIpAddress) + System.setProperty("spark.driver.host", Utils.localHostName()) } if (System.getProperty("spark.driver.port") == null) { System.setProperty("spark.driver.port", "0") @@ -99,7 +102,7 @@ class SparkContext( // Add each JAR given through the constructor - jars.foreach { addJar(_) } + if (jars != null) jars.foreach { addJar(_) } // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() @@ -111,7 +114,7 @@ class SparkContext( executorEnvs(key) = value } } - executorEnvs ++= environment + if (environment != null) executorEnvs ++= environment // Create and start the scheduler private var taskScheduler: TaskScheduler = { @@ -164,6 +167,22 @@ class SparkContext( } scheduler + case "yarn-standalone" => + val scheduler = try { + val clazz = Class.forName("spark.scheduler.cluster.YarnClusterScheduler") + val cons = clazz.getConstructor(classOf[SparkContext]) + cons.newInstance(this).asInstanceOf[ClusterScheduler] + } catch { + // TODO: Enumerate the exact reasons why it can fail + // But irrespective of it, it means we cannot proceed ! + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + val backend = new StandaloneSchedulerBackend(scheduler, this.env.actorSystem) + scheduler.initialize(backend) + scheduler + case _ => if (MESOS_REGEX.findFirstIn(master).isEmpty) { logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master)) @@ -183,7 +202,7 @@ class SparkContext( } taskScheduler.start() - private var dagScheduler = new DAGScheduler(taskScheduler) + @volatile private var dagScheduler = new DAGScheduler(taskScheduler) dagScheduler.start() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ @@ -207,6 +226,9 @@ class SparkContext( private[spark] var checkpointDir: Option[String] = None + // Post init + taskScheduler.postStartHook() + // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. */ @@ -471,7 +493,7 @@ class SparkContext( */ def getExecutorMemoryStatus: Map[String, (Long, Long)] = { env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) => - (blockManagerId.ip + ":" + blockManagerId.port, mem) + (blockManagerId.host + ":" + blockManagerId.port, mem) } } @@ -527,10 +549,13 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { - if (dagScheduler != null) { + // Do this only if not stopped already - best case effort. + // prevent NPE if stopped more than once. + val dagSchedulerCopy = dagScheduler + dagScheduler = null + if (dagSchedulerCopy != null) { metadataCleaner.cancel() - dagScheduler.stop() - dagScheduler = null + dagSchedulerCopy.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() @@ -546,6 +571,7 @@ class SparkContext( } } + /** * Get Spark's home location from either a value set through the constructor, * or the spark.home Java property, or the SPARK_HOME environment variable diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7157fd26883d3..ffb40bab3a3c4 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -72,6 +72,16 @@ object SparkEnv extends Logging { System.setProperty("spark.driver.port", boundPort.toString) } + // set only if unset until now. + if (System.getProperty("spark.hostPort", null) == null) { + if (!isDriver){ + // unexpected + Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set") + } + Utils.checkHost(hostname) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + } + val classLoader = Thread.currentThread.getContextClassLoader // Create an instance of the class named by the given Java system property, or by @@ -88,9 +98,10 @@ object SparkEnv extends Logging { logInfo("Registering " + name) actorSystem.actorOf(Props(newActor), name = name) } else { - val driverIp: String = System.getProperty("spark.driver.host", "localhost") + val driverHost: String = System.getProperty("spark.driver.host", "localhost") val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt - val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, name) + Utils.checkHost(driverHost, "Expected hostname") + val url = "akka://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) actorSystem.actorFor(url) } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 81daacf958b5a..14bb153d54ab3 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -1,18 +1,18 @@ package spark import java.io._ -import java.net._ +import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} import java.util.{Locale, Random, UUID} -import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} +import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ import scala.io.Source import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -import scala.Some import spark.serializer.SerializerInstance +import spark.deploy.SparkHadoopUtil /** * Various utility methods used by Spark. @@ -68,6 +68,41 @@ private object Utils extends Logging { return buf } + + private val shutdownDeletePaths = new collection.mutable.HashSet[String]() + + // Register the path to be deleted via shutdown hook + def registerShutdownDeleteDir(file: File) { + val absolutePath = file.getAbsolutePath() + shutdownDeletePaths.synchronized { + shutdownDeletePaths += absolutePath + } + } + + // Is the path already registered to be deleted via a shutdown hook ? + def hasShutdownDeleteDir(file: File): Boolean = { + val absolutePath = file.getAbsolutePath() + shutdownDeletePaths.synchronized { + shutdownDeletePaths.contains(absolutePath) + } + } + + // Note: if file is child of some registered path, while not equal to it, then return true; else false + // This is to ensure that two shutdown hooks do not try to delete each others paths - resulting in IOException + // and incomplete cleanup + def hasRootAsShutdownDeleteDir(file: File): Boolean = { + + val absolutePath = file.getAbsolutePath() + + val retval = shutdownDeletePaths.synchronized { + shutdownDeletePaths.find(path => ! absolutePath.equals(path) && absolutePath.startsWith(path) ).isDefined + } + + if (retval) logInfo("path = " + file + ", already present as root for deletion.") + + retval + } + /** Create a temporary directory inside the given parent directory */ def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { var attempts = 0 @@ -86,10 +121,14 @@ private object Utils extends Logging { } } catch { case e: IOException => ; } } + + registerShutdownDeleteDir(dir) + // Add a shutdown hook to delete the temp dir when the JVM exits Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dir " + dir) { override def run() { - Utils.deleteRecursively(dir) + // Attempt to delete if some patch which is parent of this is not already registered. + if (! hasRootAsShutdownDeleteDir(dir)) Utils.deleteRecursively(dir) } }) return dir @@ -227,8 +266,10 @@ private object Utils extends Logging { /** * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). + * Note, this is typically not used from within core spark. */ lazy val localIpAddress: String = findLocalIpAddress() + lazy val localIpAddressHostname: String = getAddressHostName(localIpAddress) private def findLocalIpAddress(): String = { val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") @@ -266,6 +307,8 @@ private object Utils extends Logging { * hostname it reports to the master. */ def setCustomHostname(hostname: String) { + // DEBUG code + Utils.checkHost(hostname) customHostname = Some(hostname) } @@ -273,7 +316,90 @@ private object Utils extends Logging { * Get the local machine's hostname. */ def localHostName(): String = { - customHostname.getOrElse(InetAddress.getLocalHost.getHostName) + // customHostname.getOrElse(InetAddress.getLocalHost.getHostName) + customHostname.getOrElse(localIpAddressHostname) + } + + def getAddressHostName(address: String): String = { + InetAddress.getByName(address).getHostName + } + + + + def localHostPort(): String = { + val retval = System.getProperty("spark.hostPort", null) + if (retval == null) { + logErrorWithStack("spark.hostPort not set but invoking localHostPort") + return localHostName() + } + + retval + } + + // Used by DEBUG code : remove when all testing done + def checkHost(host: String, message: String = "") { + // Currently catches only ipv4 pattern, this is just a debugging tool - not rigourous ! + if (host.matches("^[0-9]+(\\.[0-9]+)*$")) { + Utils.logErrorWithStack("Unexpected to have host " + host + " which matches IP pattern. Message " + message) + } + if (Utils.parseHostPort(host)._2 != 0){ + Utils.logErrorWithStack("Unexpected to have host " + host + " which has port in it. Message " + message) + } + } + + // Used by DEBUG code : remove when all testing done + def checkHostPort(hostPort: String, message: String = "") { + val (host, port) = Utils.parseHostPort(hostPort) + checkHost(host) + if (port <= 0){ + Utils.logErrorWithStack("Unexpected to have port " + port + " which is not valid in " + hostPort + ". Message " + message) + } + } + + def getUserNameFromEnvironment(): String = { + SparkHadoopUtil.getUserNameFromEnvironment + } + + // Used by DEBUG code : remove when all testing done + def logErrorWithStack(msg: String) { + try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } + // temp code for debug + System.exit(-1) + } + + // Typically, this will be of order of number of nodes in cluster + // If not, we should change it to LRUCache or something. + private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() + def parseHostPort(hostPort: String): (String, Int) = { + { + // Check cache first. + var cached = hostPortParseResults.get(hostPort) + if (cached != null) return cached + } + + val indx: Int = hostPort.lastIndexOf(':') + // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now. + // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 + if (-1 == indx) { + val retval = (hostPort, 0) + hostPortParseResults.put(hostPort, retval) + return retval + } + + val retval = (hostPort.substring(0, indx).trim(), hostPort.substring(indx + 1).trim().toInt) + hostPortParseResults.putIfAbsent(hostPort, retval) + hostPortParseResults.get(hostPort) + } + + def addIfNoPort(hostPort: String, port: Int): String = { + if (port <= 0) throw new IllegalArgumentException("Invalid port specified " + port) + + // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now. + // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 + val indx: Int = hostPort.lastIndexOf(':') + if (-1 != indx) return hostPort + + hostPort + ":" + port } private[spark] val daemonThreadFactory: ThreadFactory = diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 9b4d54ab4e046..807119ca8c08a 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -277,6 +277,8 @@ private class BytesToString extends spark.api.java.function.Function[Array[Byte] */ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { + + Utils.checkHost(serverHost, "Expected hostname") override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 8a3e64e4c22fa..51274acb1ed3b 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -4,6 +4,7 @@ import spark.deploy.ExecutorState.ExecutorState import spark.deploy.master.{WorkerInfo, ApplicationInfo} import spark.deploy.worker.ExecutorRunner import scala.collection.immutable.List +import spark.Utils private[spark] sealed trait DeployMessage extends Serializable @@ -19,7 +20,10 @@ case class RegisterWorker( memory: Int, webUiPort: Int, publicAddress: String) - extends DeployMessage + extends DeployMessage { + Utils.checkHost(host, "Required hostname") + assert (port > 0) +} private[spark] case class ExecutorStateChanged( @@ -58,7 +62,9 @@ private[spark] case class RegisteredApplication(appId: String) extends DeployMessage private[spark] -case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) +case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { + Utils.checkHostPort(hostPort, "Required hostport") +} private[spark] case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], @@ -81,6 +87,9 @@ private[spark] case class MasterState(host: String, port: Int, workers: Array[WorkerInfo], activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { + Utils.checkHost(host, "Required hostname") + assert (port > 0) + def uri = "spark://" + host + ":" + port } @@ -92,4 +101,8 @@ private[spark] case object RequestWorkerState private[spark] case class WorkerState(host: String, port: Int, workerId: String, executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, - coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) + coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) { + + Utils.checkHost(host, "Required hostname") + assert (port > 0) +} diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 38a6ebfc242c1..71a641a9efea2 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -12,6 +12,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { def write(obj: WorkerInfo) = JsObject( "id" -> JsString(obj.id), "host" -> JsString(obj.host), + "port" -> JsNumber(obj.port), "webuiaddress" -> JsString(obj.webUiAddress), "cores" -> JsNumber(obj.cores), "coresused" -> JsNumber(obj.coresUsed), diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 22319a96caef7..55bb61b0ccf88 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -18,7 +18,7 @@ import scala.collection.mutable.ArrayBuffer private[spark] class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging { - private val localIpAddress = Utils.localIpAddress + private val localHostname = Utils.localHostName() private val masterActorSystems = ArrayBuffer[ActorSystem]() private val workerActorSystems = ArrayBuffer[ActorSystem]() @@ -26,13 +26,13 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I logInfo("Starting a local Spark cluster with " + numWorkers + " workers.") /* Start the Master */ - val (masterSystem, masterPort) = Master.startSystemAndActor(localIpAddress, 0, 0) + val (masterSystem, masterPort) = Master.startSystemAndActor(localHostname, 0, 0) masterActorSystems += masterSystem - val masterUrl = "spark://" + localIpAddress + ":" + masterPort + val masterUrl = "spark://" + localHostname + ":" + masterPort /* Start the Workers */ for (workerNum <- 1 to numWorkers) { - val (workerSystem, _) = Worker.startSystemAndActor(localIpAddress, 0, 0, coresPerWorker, + val (workerSystem, _) = Worker.startSystemAndActor(localHostname, 0, 0, coresPerWorker, memoryPerWorker, masterUrl, null, Some(workerNum)) workerActorSystems += workerSystem } diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 2fc5e657f9692..072232e33adbb 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -59,10 +59,10 @@ private[spark] class Client( markDisconnected() context.stop(self) - case ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) => + case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) => val fullId = appId + "/" + id - logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, host, cores)) - listener.executorAdded(fullId, workerId, host, cores, memory) + logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) + listener.executorAdded(fullId, workerId, hostPort, cores, memory) case ExecutorUpdated(id, state, message, exitStatus) => val fullId = appId + "/" + id diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index b7008321df564..e8c4083f9dfef 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -12,7 +12,7 @@ private[spark] trait ClientListener { def disconnected(): Unit - def executorAdded(fullId: String, workerId: String, host: String, cores: Int, memory: Int): Unit + def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit } diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index dc004b59ca5ac..ad92532b5849a 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -16,7 +16,7 @@ private[spark] object TestClient { System.exit(0) } - def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int) {} + def executorAdded(id: String, workerId: String, hostPort: String, cores: Int, memory: Int) {} def executorRemoved(id: String, message: String, exitStatus: Option[Int]) {} } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 71b9d0801d594..160afe5239d98 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -15,7 +15,7 @@ import spark.{Logging, SparkException, Utils} import spark.util.AkkaUtils -private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -35,9 +35,11 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor var firstApp: Option[ApplicationInfo] = None + Utils.checkHost(host, "Expected hostname") + val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") - if (envVar != null) envVar else ip + if (envVar != null) envVar else host } // As a temporary workaround before better ways of configuring memory, we allow users to set @@ -46,7 +48,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean override def preStart() { - logInfo("Starting Spark master at spark://" + ip + ":" + port) + logInfo("Starting Spark master at spark://" + host + ":" + port) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) startWebUi() @@ -145,7 +147,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } case RequestMasterState => { - sender ! MasterState(ip, port, workers.toArray, apps.toArray, completedApps.toArray) + sender ! MasterState(host, port, workers.toArray, apps.toArray, completedApps.toArray) } } @@ -211,13 +213,13 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) - exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory) + exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) } def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int, publicAddress: String): WorkerInfo = { // There may be one or more refs to dead workers on this same node (w/ different ID's), remove them. - workers.filter(w => (w.host == host) && (w.state == WorkerState.DEAD)).foreach(workers -= _) + workers.filter(w => (w.host == host && w.port == port) && (w.state == WorkerState.DEAD)).foreach(workers -= _) val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) workers += worker idToWorker(worker.id) = worker @@ -307,7 +309,7 @@ private[spark] object Master { def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) - val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort) + val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) actorSystem.awaitTermination() } diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index 4ceab3fc036da..3d28ecabb4d0e 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -7,13 +7,13 @@ import spark.Utils * Command-line parser for the master. */ private[spark] class MasterArguments(args: Array[String]) { - var ip = Utils.localHostName() + var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 // Check for settings in environment variables - if (System.getenv("SPARK_MASTER_IP") != null) { - ip = System.getenv("SPARK_MASTER_IP") + if (System.getenv("SPARK_MASTER_HOST") != null) { + host = System.getenv("SPARK_MASTER_HOST") } if (System.getenv("SPARK_MASTER_PORT") != null) { port = System.getenv("SPARK_MASTER_PORT").toInt @@ -26,7 +26,13 @@ private[spark] class MasterArguments(args: Array[String]) { def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => - ip = value + Utils.checkHost(value, "ip no longer supported, please use hostname " + value) + host = value + parse(tail) + + case ("--host" | "-h") :: value :: tail => + Utils.checkHost(value, "Please use hostname " + value) + host = value parse(tail) case ("--port" | "-p") :: IntParam(value) :: tail => @@ -54,7 +60,8 @@ private[spark] class MasterArguments(args: Array[String]) { "Usage: Master [options]\n" + "\n" + "Options:\n" + - " -i IP, --ip IP IP address or DNS name to listen on\n" + + " -i HOST, --ip HOST Hostname to listen on (deprecated, please use --host or -h) \n" + + " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + " --webui-port PORT Port for web UI (default: 8080)") System.exit(exitCode) diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 23df1bb463288..0c08c5f417238 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -2,6 +2,7 @@ package spark.deploy.master import akka.actor.ActorRef import scala.collection.mutable +import spark.Utils private[spark] class WorkerInfo( val id: String, @@ -13,6 +14,9 @@ private[spark] class WorkerInfo( val webUiPort: Int, val publicAddress: String) { + Utils.checkHost(host, "Expected hostname") + assert (port > 0) + var executors = new mutable.HashMap[String, ExecutorInfo] // fullId => info var state: WorkerState.Value = WorkerState.ALIVE var coresUsed = 0 @@ -23,6 +27,11 @@ private[spark] class WorkerInfo( def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed + def hostPort: String = { + assert (port > 0) + host + ":" + port + } + def addExecutor(exec: ExecutorInfo) { executors(exec.fullId) = exec coresUsed += exec.cores diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index de11771c8e62d..dfcb9f0d05d5f 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -21,11 +21,13 @@ private[spark] class ExecutorRunner( val memory: Int, val worker: ActorRef, val workerId: String, - val hostname: String, + val hostPort: String, val sparkHome: File, val workDir: File) extends Logging { + Utils.checkHostPort(hostPort, "Expected hostport") + val fullId = appId + "/" + execId var workerThread: Thread = null var process: Process = null @@ -68,7 +70,7 @@ private[spark] class ExecutorRunner( /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */ def substituteVariables(argument: String): String = argument match { case "{{EXECUTOR_ID}}" => execId.toString - case "{{HOSTNAME}}" => hostname + case "{{HOSTPORT}}" => hostPort case "{{CORES}}" => cores.toString case other => other } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 8919d1261cc86..cf4babc892703 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -16,7 +16,7 @@ import spark.deploy.master.Master import java.io.File private[spark] class Worker( - ip: String, + host: String, port: Int, webUiPort: Int, cores: Int, @@ -25,6 +25,9 @@ private[spark] class Worker( workDirPath: String = null) extends Actor with Logging { + Utils.checkHost(host, "Expected hostname") + assert (port > 0) + val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds @@ -39,7 +42,7 @@ private[spark] class Worker( val finishedExecutors = new HashMap[String, ExecutorRunner] val publicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") - if (envVar != null) envVar else ip + if (envVar != null) envVar else host } var coresUsed = 0 @@ -64,7 +67,7 @@ private[spark] class Worker( override def preStart() { logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format( - ip, port, cores, Utils.memoryMegabytesToString(memory))) + host, port, cores, Utils.memoryMegabytesToString(memory))) sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() @@ -75,7 +78,7 @@ private[spark] class Worker( def connectToMaster() { logInfo("Connecting to master " + masterUrl) master = context.actorFor(Master.toAkkaUrl(masterUrl)) - master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) + master ! RegisterWorker(workerId, host, port, cores, memory, webUiPort, publicAddress) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } @@ -106,7 +109,7 @@ private[spark] class Worker( case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) => logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner( - appId, execId, appDesc, cores_, memory_, self, workerId, ip, new File(execSparkHome_), workDir) + appId, execId, appDesc, cores_, memory_, self, workerId, host + ":" + port, new File(execSparkHome_), workDir) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -141,7 +144,7 @@ private[spark] class Worker( masterDisconnected() case RequestWorkerState => { - sender ! WorkerState(ip, port, workerId, executors.values.toList, + sender ! WorkerState(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } @@ -156,7 +159,7 @@ private[spark] class Worker( } def generateWorkerId(): String = { - "worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), ip, port) + "worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), host, port) } override def postStop() { @@ -167,7 +170,7 @@ private[spark] class Worker( private[spark] object Worker { def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) - val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort, args.cores, + val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, args.memory, args.master, args.workDir) actorSystem.awaitTermination() } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala index 08f02bad80d7f..2b96611ee3372 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala @@ -9,7 +9,7 @@ import java.lang.management.ManagementFactory * Command-line parser for the master. */ private[spark] class WorkerArguments(args: Array[String]) { - var ip = Utils.localHostName() + var host = Utils.localHostName() var port = 0 var webUiPort = 8081 var cores = inferDefaultCores() @@ -38,7 +38,13 @@ private[spark] class WorkerArguments(args: Array[String]) { def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => - ip = value + Utils.checkHost(value, "ip no longer supported, please use hostname " + value) + host = value + parse(tail) + + case ("--host" | "-h") :: value :: tail => + Utils.checkHost(value, "Please use hostname " + value) + host = value parse(tail) case ("--port" | "-p") :: IntParam(value) :: tail => @@ -93,7 +99,8 @@ private[spark] class WorkerArguments(args: Array[String]) { " -c CORES, --cores CORES Number of cores to use\n" + " -m MEM, --memory MEM Amount of memory to use (e.g. 1000M, 2G)\n" + " -d DIR, --work-dir DIR Directory to run apps in (default: SPARK_HOME/work)\n" + - " -i IP, --ip IP IP address or DNS name to listen on\n" + + " -i HOST, --ip IP Hostname to listen on (deprecated, please use --host or -h)\n" + + " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: random)\n" + " --webui-port PORT Port for web UI (default: 8081)") System.exit(exitCode) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 3e7407b58d8e6..344face5e6986 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -17,7 +17,7 @@ import java.nio.ByteBuffer * The Mesos executor for Spark. */ private[spark] class Executor(executorId: String, slaveHostname: String, properties: Seq[(String, String)]) extends Logging { - + // Application dependencies (added through SparkContext) that we've fetched so far on this node. // Each map holds the master's timestamp for the version of that file or JAR we got. private val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() @@ -27,6 +27,11 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert initLogging() + // No ip or host:port - just hostname + Utils.checkHost(slaveHostname, "Expected executed slave to be a hostname") + // must not have port specified. + assert (0 == Utils.parseHostPort(slaveHostname)._2) + // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 1047f71c6ae0d..49e1f3f07a995 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -12,23 +12,27 @@ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask import spark.scheduler.cluster.RegisterExecutorFailed import spark.scheduler.cluster.RegisterExecutor +import spark.Utils +import spark.deploy.SparkHadoopUtil private[spark] class StandaloneExecutorBackend( driverUrl: String, executorId: String, - hostname: String, + hostPort: String, cores: Int) extends Actor with ExecutorBackend with Logging { + Utils.checkHostPort(hostPort, "Expected hostport") + var executor: Executor = null var driver: ActorRef = null override def preStart() { logInfo("Connecting to driver: " + driverUrl) driver = context.actorFor(driverUrl) - driver ! RegisterExecutor(executorId, hostname, cores) + driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing } @@ -36,7 +40,8 @@ private[spark] class StandaloneExecutorBackend( override def receive = { case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") - executor = new Executor(executorId, hostname, sparkProperties) + // Make this host instead of hostPort ? + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -63,11 +68,29 @@ private[spark] class StandaloneExecutorBackend( private[spark] object StandaloneExecutorBackend { def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { + SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) + } + + // This will be run 'as' the user + def run0(args: Product) { + assert(4 == args.productArity) + runImpl(args.productElement(0).asInstanceOf[String], + args.productElement(0).asInstanceOf[String], + args.productElement(0).asInstanceOf[String], + args.productElement(0).asInstanceOf[Int]) + } + + private def runImpl(driverUrl: String, executorId: String, hostname: String, cores: Int) { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0) + // Debug code + Utils.checkHost(hostname) + // set it + val sparkHostPort = hostname + ":" + boundPort + System.setProperty("spark.hostPort", sparkHostPort) val actor = actorSystem.actorOf( - Props(new StandaloneExecutorBackend(driverUrl, executorId, hostname, cores)), + Props(new StandaloneExecutorBackend(driverUrl, executorId, sparkHostPort, cores)), name = "Executor") actorSystem.awaitTermination() } diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index d1451bc2124c5..00a0433a441dc 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -13,7 +13,7 @@ import java.net._ private[spark] abstract class Connection(val channel: SocketChannel, val selector: Selector, - val remoteConnectionManagerId: ConnectionManagerId) extends Logging { + val socketRemoteConnectionManagerId: ConnectionManagerId) extends Logging { def this(channel_ : SocketChannel, selector_ : Selector) = { this(channel_, selector_, ConnectionManagerId.fromSocketAddress( @@ -32,16 +32,43 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, var onKeyInterestChangeCallback: (Connection, Int) => Unit = null val remoteAddress = getRemoteAddress() + + // Read channels typically do not register for write and write does not for read + // Now, we do have write registering for read too (temporarily), but this is to detect + // channel close NOT to actually read/consume data on it ! + // How does this work if/when we move to SSL ? + + // What is the interest to register with selector for when we want this connection to be selected + def registerInterest() + // What is the interest to register with selector for when we want this connection to be de-selected + // Traditionally, 0 - but in our case, for example, for close-detection on SendingConnection hack, it will be + // SelectionKey.OP_READ (until we fix it properly) + def unregisterInterest() + + // On receiving a read event, should we change the interest for this channel or not ? + // Will be true for ReceivingConnection, false for SendingConnection. + def changeInterestForRead(): Boolean + + // On receiving a write event, should we change the interest for this channel or not ? + // Will be false for ReceivingConnection, true for SendingConnection. + // Actually, for now, should not get triggered for ReceivingConnection + def changeInterestForWrite(): Boolean + + def getRemoteConnectionManagerId(): ConnectionManagerId = { + socketRemoteConnectionManagerId + } def key() = channel.keyFor(selector) def getRemoteAddress() = channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] - def read() { + // Returns whether we have to register for further reads or not. + def read(): Boolean = { throw new UnsupportedOperationException("Cannot read on connection of type " + this.getClass.toString) } - - def write() { + + // Returns whether we have to register for further writes or not. + def write(): Boolean = { throw new UnsupportedOperationException("Cannot write on connection of type " + this.getClass.toString) } @@ -64,7 +91,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, if (onExceptionCallback != null) { onExceptionCallback(this, e) } else { - logError("Error in connection to " + remoteConnectionManagerId + + logError("Error in connection to " + getRemoteConnectionManagerId() + " and OnExceptionCallback not registered", e) } } @@ -73,7 +100,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, if (onCloseCallback != null) { onCloseCallback(this) } else { - logWarning("Connection to " + remoteConnectionManagerId + + logWarning("Connection to " + getRemoteConnectionManagerId() + " closed and OnExceptionCallback not registered") } @@ -81,7 +108,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, def changeConnectionKeyInterest(ops: Int) { if (onKeyInterestChangeCallback != null) { - onKeyInterestChangeCallback(this, ops) + onKeyInterestChangeCallback(this, ops) } else { throw new Exception("OnKeyInterestChangeCallback not registered") } @@ -122,7 +149,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { messages.synchronized{ /*messages += message*/ messages.enqueue(message) - logDebug("Added [" + message + "] to outbox for sending to [" + remoteConnectionManagerId + "]") + logDebug("Added [" + message + "] to outbox for sending to [" + getRemoteConnectionManagerId() + "]") } } @@ -149,9 +176,9 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } return chunk } else { - /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/ + /*logInfo("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "]")*/ message.finishTime = System.currentTimeMillis - logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + + logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "] in " + message.timeTaken ) } } @@ -170,15 +197,15 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { messages.enqueue(message) nextMessageToBeUsed = nextMessageToBeUsed + 1 if (!message.started) { - logDebug("Starting to send [" + message + "] to [" + remoteConnectionManagerId + "]") + logDebug("Starting to send [" + message + "] to [" + getRemoteConnectionManagerId() + "]") message.started = true message.startTime = System.currentTimeMillis } - logTrace("Sending chunk from [" + message+ "] to [" + remoteConnectionManagerId + "]") + logTrace("Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]") return chunk } else { message.finishTime = System.currentTimeMillis - logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + + logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "] in " + message.timeTaken ) } } @@ -187,26 +214,39 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } } - val outbox = new Outbox(1) + private val outbox = new Outbox(1) val currentBuffers = new ArrayBuffer[ByteBuffer]() /*channel.socket.setSendBufferSize(256 * 1024)*/ - override def getRemoteAddress() = address + override def getRemoteAddress() = address + val DEFAULT_INTEREST = SelectionKey.OP_READ + + override def registerInterest() { + // Registering read too - does not really help in most cases, but for some + // it does - so let us keep it for now. + changeConnectionKeyInterest(SelectionKey.OP_WRITE | DEFAULT_INTEREST) + } + + override def unregisterInterest() { + changeConnectionKeyInterest(DEFAULT_INTEREST) + } + def send(message: Message) { outbox.synchronized { outbox.addMessage(message) if (channel.isConnected) { - changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ) + registerInterest() } } } + // MUST be called within the selector loop def connect() { try{ - channel.connect(address) channel.register(selector, SelectionKey.OP_CONNECT) + channel.connect(address) logInfo("Initiating connection to [" + address + "]") } catch { case e: Exception => { @@ -216,20 +256,33 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } } - def finishConnect() { + def finishConnect(force: Boolean): Boolean = { try { - channel.finishConnect - changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ) + // Typically, this should finish immediately since it was triggered by a connect + // selection - though need not necessarily always complete successfully. + val connected = channel.finishConnect + if (!force && !connected) { + logInfo("finish connect failed [" + address + "], " + outbox.messages.size + " messages pending") + return false + } + + // Fallback to previous behavior - assume finishConnect completed + // This will happen only when finishConnect failed for some repeated number of times (10 or so) + // Is highly unlikely unless there was an unclean close of socket, etc + registerInterest() logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending") + return true } catch { case e: Exception => { logWarning("Error finishing connection to " + address, e) callOnExceptionCallback(e) + // ignore + return true } } } - override def write() { + override def write(): Boolean = { try{ while(true) { if (currentBuffers.size == 0) { @@ -239,8 +292,9 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { currentBuffers ++= chunk.buffers } case None => { - changeConnectionKeyInterest(SelectionKey.OP_READ) - return + // changeConnectionKeyInterest(0) + /*key.interestOps(0)*/ + return false } } } @@ -254,38 +308,53 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { currentBuffers -= buffer } if (writtenBytes < remainingBytes) { - return + // re-register for write. + return true } } } } catch { case e: Exception => { - logWarning("Error writing in connection to " + remoteConnectionManagerId, e) + logWarning("Error writing in connection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() + return false } } + // should not happen - to keep scala compiler happy + return true } - override def read() { + // This is a hack to determine if remote socket was closed or not. + // SendingConnection DOES NOT expect to receive any data - if it does, it is an error + // For a bunch of cases, read will return -1 in case remote socket is closed : hence we + // register for reads to determine that. + override def read(): Boolean = { // We don't expect the other side to send anything; so, we just read to detect an error or EOF. try { val length = channel.read(ByteBuffer.allocate(1)) if (length == -1) { // EOF close() } else if (length > 0) { - logWarning("Unexpected data read from SendingConnection to " + remoteConnectionManagerId) + logWarning("Unexpected data read from SendingConnection to " + getRemoteConnectionManagerId()) } } catch { case e: Exception => - logError("Exception while reading SendingConnection to " + remoteConnectionManagerId, e) + logError("Exception while reading SendingConnection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() } + + false } + + override def changeInterestForRead(): Boolean = false + + override def changeInterestForWrite(): Boolean = true } +// Must be created within selector loop - else deadlock private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) extends Connection(channel_, selector_) { @@ -298,13 +367,13 @@ extends Connection(channel_, selector_) { val newMessage = Message.create(header).asInstanceOf[BufferMessage] newMessage.started = true newMessage.startTime = System.currentTimeMillis - logDebug("Starting to receive [" + newMessage + "] from [" + remoteConnectionManagerId + "]") + logDebug("Starting to receive [" + newMessage + "] from [" + getRemoteConnectionManagerId() + "]") messages += ((newMessage.id, newMessage)) newMessage } val message = messages.getOrElseUpdate(header.id, createNewMessage) - logTrace("Receiving chunk of [" + message + "] from [" + remoteConnectionManagerId + "]") + logTrace("Receiving chunk of [" + message + "] from [" + getRemoteConnectionManagerId() + "]") message.getChunkForReceiving(header.chunkSize) } @@ -316,7 +385,27 @@ extends Connection(channel_, selector_) { messages -= message.id } } - + + @volatile private var inferredRemoteManagerId: ConnectionManagerId = null + override def getRemoteConnectionManagerId(): ConnectionManagerId = { + val currId = inferredRemoteManagerId + if (currId != null) currId else super.getRemoteConnectionManagerId() + } + + // The reciever's remote address is the local socket on remote side : which is NOT the connection manager id of the receiver. + // We infer that from the messages we receive on the receiver socket. + private def processConnectionManagerId(header: MessageChunkHeader) { + val currId = inferredRemoteManagerId + if (header.address == null || currId != null) return + + val managerId = ConnectionManagerId.fromSocketAddress(header.address) + + if (managerId != null) { + inferredRemoteManagerId = managerId + } + } + + val inbox = new Inbox() val headerBuffer: ByteBuffer = ByteBuffer.allocate(MessageChunkHeader.HEADER_SIZE) var onReceiveCallback: (Connection , Message) => Unit = null @@ -324,17 +413,18 @@ extends Connection(channel_, selector_) { channel.register(selector, SelectionKey.OP_READ) - override def read() { + override def read(): Boolean = { try { while (true) { if (currentChunk == null) { val headerBytesRead = channel.read(headerBuffer) if (headerBytesRead == -1) { close() - return + return false } if (headerBuffer.remaining > 0) { - return + // re-register for read event ... + return true } headerBuffer.flip if (headerBuffer.remaining != MessageChunkHeader.HEADER_SIZE) { @@ -342,6 +432,9 @@ extends Connection(channel_, selector_) { } val header = MessageChunkHeader.create(headerBuffer) headerBuffer.clear() + + processConnectionManagerId(header) + header.typ match { case Message.BUFFER_MESSAGE => { if (header.totalSize == 0) { @@ -349,7 +442,8 @@ extends Connection(channel_, selector_) { onReceiveCallback(this, Message.create(header)) } currentChunk = null - return + // re-register for read event ... + return true } else { currentChunk = inbox.getChunk(header).orNull } @@ -362,10 +456,11 @@ extends Connection(channel_, selector_) { val bytesRead = channel.read(currentChunk.buffer) if (bytesRead == 0) { - return + // re-register for read event ... + return true } else if (bytesRead == -1) { close() - return + return false } /*logDebug("Read " + bytesRead + " bytes for the buffer")*/ @@ -376,7 +471,7 @@ extends Connection(channel_, selector_) { if (bufferMessage.isCompletelyReceived) { bufferMessage.flip bufferMessage.finishTime = System.currentTimeMillis - logDebug("Finished receiving [" + bufferMessage + "] from [" + remoteConnectionManagerId + "] in " + bufferMessage.timeTaken) + logDebug("Finished receiving [" + bufferMessage + "] from [" + getRemoteConnectionManagerId() + "] in " + bufferMessage.timeTaken) if (onReceiveCallback != null) { onReceiveCallback(this, bufferMessage) } @@ -387,12 +482,31 @@ extends Connection(channel_, selector_) { } } catch { case e: Exception => { - logWarning("Error reading from connection to " + remoteConnectionManagerId, e) + logWarning("Error reading from connection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() + return false } } + // should not happen - to keep scala compiler happy + return true } def onReceive(callback: (Connection, Message) => Unit) {onReceiveCallback = callback} + + override def changeInterestForRead(): Boolean = true + + override def changeInterestForWrite(): Boolean = { + throw new IllegalStateException("Unexpected invocation right now") + } + + override def registerInterest() { + // Registering read too - does not really help in most cases, but for some + // it does - so let us keep it for now. + changeConnectionKeyInterest(SelectionKey.OP_READ) + } + + override def unregisterInterest() { + changeConnectionKeyInterest(0) + } } diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index b6ec664d7e81b..0c6bdb155972f 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -6,12 +6,12 @@ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ import java.net._ -import java.util.concurrent.Executors +import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} +import scala.collection.mutable.HashSet import scala.collection.mutable.HashMap import scala.collection.mutable.SynchronizedMap import scala.collection.mutable.SynchronizedQueue -import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer import akka.dispatch.{Await, Promise, ExecutionContext, Future} @@ -19,6 +19,10 @@ import akka.util.Duration import akka.util.duration._ private[spark] case class ConnectionManagerId(host: String, port: Int) { + // DEBUG code + Utils.checkHost(host) + assert (port > 0) + def toSocketAddress() = new InetSocketAddress(host, port) } @@ -42,19 +46,37 @@ private[spark] class ConnectionManager(port: Int) extends Logging { def markDone() { completionHandler(this) } } - val selector = SelectorProvider.provider.openSelector() - val handleMessageExecutor = Executors.newFixedThreadPool(System.getProperty("spark.core.connection.handler.threads","20").toInt) - val serverChannel = ServerSocketChannel.open() - val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] - val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] - val messageStatuses = new HashMap[Int, MessageStatus] - val connectionRequests = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] - val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)] - val sendMessageRequests = new Queue[(Message, SendingConnection)] + private val selector = SelectorProvider.provider.openSelector() + + private val handleMessageExecutor = new ThreadPoolExecutor( + System.getProperty("spark.core.connection.handler.threads.min","20").toInt, + System.getProperty("spark.core.connection.handler.threads.max","60").toInt, + System.getProperty("spark.core.connection.handler.threads.keepalive","60").toInt, TimeUnit.SECONDS, + new LinkedBlockingDeque[Runnable]()) + + private val handleReadWriteExecutor = new ThreadPoolExecutor( + System.getProperty("spark.core.connection.io.threads.min","4").toInt, + System.getProperty("spark.core.connection.io.threads.max","32").toInt, + System.getProperty("spark.core.connection.io.threads.keepalive","60").toInt, TimeUnit.SECONDS, + new LinkedBlockingDeque[Runnable]()) + + // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap + private val handleConnectExecutor = new ThreadPoolExecutor( + System.getProperty("spark.core.connection.connect.threads.min","1").toInt, + System.getProperty("spark.core.connection.connect.threads.max","8").toInt, + System.getProperty("spark.core.connection.connect.threads.keepalive","60").toInt, TimeUnit.SECONDS, + new LinkedBlockingDeque[Runnable]()) + + private val serverChannel = ServerSocketChannel.open() + private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] + private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] + private val messageStatuses = new HashMap[Int, MessageStatus] + private val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)] + private val registerRequests = new SynchronizedQueue[SendingConnection] implicit val futureExecContext = ExecutionContext.fromExecutor(Utils.newDaemonCachedThreadPool()) - var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null + private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null serverChannel.configureBlocking(false) serverChannel.socket.setReuseAddress(true) @@ -65,46 +87,139 @@ private[spark] class ConnectionManager(port: Int) extends Logging { val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort) logInfo("Bound socket to port " + serverChannel.socket.getLocalPort() + " with id = " + id) - - val selectorThread = new Thread("connection-manager-thread") { + + private val selectorThread = new Thread("connection-manager-thread") { override def run() = ConnectionManager.this.run() } selectorThread.setDaemon(true) selectorThread.start() - private def run() { - try { - while(!selectorThread.isInterrupted) { - for ((connectionManagerId, sendingConnection) <- connectionRequests) { - sendingConnection.connect() - addConnection(sendingConnection) - connectionRequests -= connectionManagerId + private val writeRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() + + private def triggerWrite(key: SelectionKey) { + val conn = connectionsByKey.getOrElse(key, null) + if (conn == null) return + + writeRunnableStarted.synchronized { + // So that we do not trigger more write events while processing this one. + // The write method will re-register when done. + if (conn.changeInterestForWrite()) conn.unregisterInterest() + if (writeRunnableStarted.contains(key)) { + // key.interestOps(key.interestOps() & ~ SelectionKey.OP_WRITE) + return + } + + writeRunnableStarted += key + } + handleReadWriteExecutor.execute(new Runnable { + override def run() { + var register: Boolean = false + try { + register = conn.write() + } finally { + writeRunnableStarted.synchronized { + writeRunnableStarted -= key + if (register && conn.changeInterestForWrite()) { + conn.registerInterest() + } + } } - sendMessageRequests.synchronized { - while (!sendMessageRequests.isEmpty) { - val (message, connection) = sendMessageRequests.dequeue - connection.send(message) + } + } ) + } + + private val readRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() + + private def triggerRead(key: SelectionKey) { + val conn = connectionsByKey.getOrElse(key, null) + if (conn == null) return + + readRunnableStarted.synchronized { + // So that we do not trigger more read events while processing this one. + // The read method will re-register when done. + if (conn.changeInterestForRead())conn.unregisterInterest() + if (readRunnableStarted.contains(key)) { + return + } + + readRunnableStarted += key + } + handleReadWriteExecutor.execute(new Runnable { + override def run() { + var register: Boolean = false + try { + register = conn.read() + } finally { + readRunnableStarted.synchronized { + readRunnableStarted -= key + if (register && conn.changeInterestForRead()) { + conn.registerInterest() + } } } + } + } ) + } + + private def triggerConnect(key: SelectionKey) { + val conn = connectionsByKey.getOrElse(key, null).asInstanceOf[SendingConnection] + if (conn == null) return + + // prevent other events from being triggered + // Since we are still trying to connect, we do not need to do the additional steps in triggerWrite + conn.changeConnectionKeyInterest(0) + + handleConnectExecutor.execute(new Runnable { + override def run() { + + var tries: Int = 10 + while (tries >= 0) { + if (conn.finishConnect(false)) return + // Sleep ? + Thread.sleep(1) + tries -= 1 + } - while (!keyInterestChangeRequests.isEmpty) { + // fallback to previous behavior : we should not really come here since this method was + // triggered since channel became connectable : but at times, the first finishConnect need not + // succeed : hence the loop to retry a few 'times'. + conn.finishConnect(true) + } + } ) + } + + def run() { + try { + while(!selectorThread.isInterrupted) { + while (! registerRequests.isEmpty) { + val conn: SendingConnection = registerRequests.dequeue + addListeners(conn) + conn.connect() + addConnection(conn) + } + + while(!keyInterestChangeRequests.isEmpty) { val (key, ops) = keyInterestChangeRequests.dequeue - val connection = connectionsByKey(key) - val lastOps = key.interestOps() - key.interestOps(ops) - - def intToOpStr(op: Int): String = { - val opStrs = ArrayBuffer[String]() - if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ" - if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE" - if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT" - if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT" - if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " + val connection = connectionsByKey.getOrElse(key, null) + if (connection != null) { + val lastOps = key.interestOps() + key.interestOps(ops) + + // hot loop - prevent materialization of string if trace not enabled. + if (isTraceEnabled()) { + def intToOpStr(op: Int): String = { + val opStrs = ArrayBuffer[String]() + if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ" + if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE" + if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT" + if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT" + if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " + } + + logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() + + "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") + } } - - logTrace("Changed key for connection to [" + connection.remoteConnectionManagerId + - "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") - } val selectedKeysCount = selector.select() @@ -123,12 +238,15 @@ private[spark] class ConnectionManager(port: Int) extends Logging { if (key.isValid) { if (key.isAcceptable) { acceptConnection(key) - } else if (key.isConnectable) { - connectionsByKey(key).asInstanceOf[SendingConnection].finishConnect() - } else if (key.isReadable) { - connectionsByKey(key).read() - } else if (key.isWritable) { - connectionsByKey(key).write() + } else + if (key.isConnectable) { + triggerConnect(key) + } else + if (key.isReadable) { + triggerRead(key) + } else + if (key.isWritable) { + triggerWrite(key) } } } @@ -138,94 +256,116 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } } - private def acceptConnection(key: SelectionKey) { + def acceptConnection(key: SelectionKey) { val serverChannel = key.channel.asInstanceOf[ServerSocketChannel] - val newChannel = serverChannel.accept() - val newConnection = new ReceivingConnection(newChannel, selector) - newConnection.onReceive(receiveMessage) - newConnection.onClose(removeConnection) - addConnection(newConnection) - logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]") - } - private def addConnection(connection: Connection) { - connectionsByKey += ((connection.key, connection)) - if (connection.isInstanceOf[SendingConnection]) { - val sendingConnection = connection.asInstanceOf[SendingConnection] - connectionsById += ((sendingConnection.remoteConnectionManagerId, sendingConnection)) + var newChannel = serverChannel.accept() + + // accept them all in a tight loop. non blocking accept with no processing, should be fine + while (newChannel != null) { + try { + val newConnection = new ReceivingConnection(newChannel, selector) + newConnection.onReceive(receiveMessage) + addListeners(newConnection) + addConnection(newConnection) + logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]") + } catch { + // might happen in case of issues with registering with selector + case e: Exception => logError("Error in accept loop", e) + } + + newChannel = serverChannel.accept() } + } + + private def addListeners(connection: Connection) { connection.onKeyInterestChange(changeConnectionKeyInterest) connection.onException(handleConnectionError) connection.onClose(removeConnection) } - private def removeConnection(connection: Connection) { + def addConnection(connection: Connection) { + connectionsByKey += ((connection.key, connection)) + } + + def removeConnection(connection: Connection) { connectionsByKey -= connection.key - if (connection.isInstanceOf[SendingConnection]) { - val sendingConnection = connection.asInstanceOf[SendingConnection] - val sendingConnectionManagerId = sendingConnection.remoteConnectionManagerId - logInfo("Removing SendingConnection to " + sendingConnectionManagerId) - - connectionsById -= sendingConnectionManagerId - - messageStatuses.synchronized { - messageStatuses - .values.filter(_.connectionManagerId == sendingConnectionManagerId).foreach(status => { - logInfo("Notifying " + status) - status.synchronized { - status.attempted = true - status.acked = false - status.markDone() - } + + try { + if (connection.isInstanceOf[SendingConnection]) { + val sendingConnection = connection.asInstanceOf[SendingConnection] + val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId() + logInfo("Removing SendingConnection to " + sendingConnectionManagerId) + + connectionsById -= sendingConnectionManagerId + + messageStatuses.synchronized { + messageStatuses + .values.filter(_.connectionManagerId == sendingConnectionManagerId).foreach(status => { + logInfo("Notifying " + status) + status.synchronized { + status.attempted = true + status.acked = false + status.markDone() + } + }) + + messageStatuses.retain((i, status) => { + status.connectionManagerId != sendingConnectionManagerId }) + } + } else if (connection.isInstanceOf[ReceivingConnection]) { + val receivingConnection = connection.asInstanceOf[ReceivingConnection] + val remoteConnectionManagerId = receivingConnection.getRemoteConnectionManagerId() + logInfo("Removing ReceivingConnection to " + remoteConnectionManagerId) + + val sendingConnectionOpt = connectionsById.get(remoteConnectionManagerId) + if (! sendingConnectionOpt.isDefined) { + logError("Corresponding SendingConnectionManagerId not found") + return + } - messageStatuses.retain((i, status) => { - status.connectionManagerId != sendingConnectionManagerId - }) - } - } else if (connection.isInstanceOf[ReceivingConnection]) { - val receivingConnection = connection.asInstanceOf[ReceivingConnection] - val remoteConnectionManagerId = receivingConnection.remoteConnectionManagerId - logInfo("Removing ReceivingConnection to " + remoteConnectionManagerId) - - val sendingConnectionManagerId = connectionsById.keys.find(_.host == remoteConnectionManagerId.host).orNull - if (sendingConnectionManagerId == null) { - logError("Corresponding SendingConnectionManagerId not found") - return - } - logInfo("Corresponding SendingConnectionManagerId is " + sendingConnectionManagerId) - - val sendingConnection = connectionsById(sendingConnectionManagerId) - sendingConnection.close() - connectionsById -= sendingConnectionManagerId - - messageStatuses.synchronized { - for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) { - logInfo("Notifying " + s) - s.synchronized { - s.attempted = true - s.acked = false - s.markDone() + val sendingConnection = sendingConnectionOpt.get + connectionsById -= remoteConnectionManagerId + sendingConnection.close() + + val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId() + + assert (sendingConnectionManagerId == remoteConnectionManagerId) + + messageStatuses.synchronized { + for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) { + logInfo("Notifying " + s) + s.synchronized { + s.attempted = true + s.acked = false + s.markDone() + } } - } - messageStatuses.retain((i, status) => { - status.connectionManagerId != sendingConnectionManagerId - }) + messageStatuses.retain((i, status) => { + status.connectionManagerId != sendingConnectionManagerId + }) + } } + } finally { + // So that the selection keys can be removed. + wakeupSelector() } } - private def handleConnectionError(connection: Connection, e: Exception) { - logInfo("Handling connection error on connection to " + connection.remoteConnectionManagerId) + def handleConnectionError(connection: Connection, e: Exception) { + logInfo("Handling connection error on connection to " + connection.getRemoteConnectionManagerId()) removeConnection(connection) } - private def changeConnectionKeyInterest(connection: Connection, ops: Int) { - keyInterestChangeRequests += ((connection.key, ops)) + def changeConnectionKeyInterest(connection: Connection, ops: Int) { + keyInterestChangeRequests += ((connection.key, ops)) + // so that registerations happen ! + wakeupSelector() } - private def receiveMessage(connection: Connection, message: Message) { + def receiveMessage(connection: Connection, message: Message) { val connectionManagerId = ConnectionManagerId.fromSocketAddress(message.senderAddress) logDebug("Received [" + message + "] from [" + connectionManagerId + "]") val runnable = new Runnable() { @@ -293,18 +433,22 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private def sendMessage(connectionManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, connectionManagerId.port) - val newConnection = connectionRequests.getOrElseUpdate(connectionManagerId, - new SendingConnection(inetSocketAddress, selector, connectionManagerId)) - newConnection + val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId) + registerRequests.enqueue(newConnection) + + newConnection } - val lookupKey = ConnectionManagerId.fromSocketAddress(connectionManagerId.toSocketAddress) - val connection = connectionsById.getOrElse(lookupKey, startNewConnection()) + // I removed the lookupKey stuff as part of merge ... should I re-add it ? We did not find it useful in our test-env ... + // If we do re-add it, we should consistently use it everywhere I guess ? + val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection()) message.senderAddress = id.toSocketAddress() logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") - /*connection.send(message)*/ - sendMessageRequests.synchronized { - sendMessageRequests += ((message, connection)) - } + connection.send(message) + + wakeupSelector() + } + + private def wakeupSelector() { selector.wakeup() } @@ -337,6 +481,8 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logWarning("All connections not cleaned up") } handleMessageExecutor.shutdown() + handleReadWriteExecutor.shutdown() + handleConnectExecutor.shutdown() logInfo("ConnectionManager stopped") } } diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala index 525751b5bf915..34fac9e77699b 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/spark/network/Message.scala @@ -17,7 +17,8 @@ private[spark] class MessageChunkHeader( val other: Int, val address: InetSocketAddress) { lazy val buffer = { - val ip = address.getAddress.getAddress() + // No need to change this, at 'use' time, we do a reverse lookup of the hostname. Refer to network.Connection + val ip = address.getAddress.getAddress() val port = address.getPort() ByteBuffer. allocate(MessageChunkHeader.HEADER_SIZE). diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c54dce51d7839..1440b93e65cf9 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -50,6 +50,11 @@ class DAGScheduler( eventQueue.put(ExecutorLost(execId)) } + // Called by TaskScheduler when a host is added + override def executorGained(execId: String, hostPort: String) { + eventQueue.put(ExecutorGained(execId, hostPort)) + } + // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures. override def taskSetFailed(taskSet: TaskSet, reason: String) { eventQueue.put(TaskSetFailed(taskSet, reason)) @@ -113,7 +118,7 @@ class DAGScheduler( if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map { - locations => locations.map(_.ip).toList + locations => locations.map(_.hostPort).toList }.toArray } cacheLocs(rdd.id) @@ -293,6 +298,9 @@ class DAGScheduler( submitStage(finalStage) } + case ExecutorGained(execId, hostPort) => + handleExecutorGained(execId, hostPort) + case ExecutorLost(execId) => handleExecutorLost(execId) @@ -630,6 +638,14 @@ class DAGScheduler( "(generation " + currentGeneration + ")") } } + + private def handleExecutorGained(execId: String, hostPort: String) { + // remove from failedGeneration(execId) ? + if (failedGeneration.contains(execId)) { + logInfo("Host gained which was in lost list earlier: " + hostPort) + failedGeneration -= execId + } + } /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index ed0b9bf178a89..b46bb863f04d8 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -32,6 +32,10 @@ private[spark] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent +private[spark] case class ExecutorGained(execId: String, hostPort: String) extends DAGSchedulerEvent { + Utils.checkHostPort(hostPort, "Required hostport") +} + private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent private[spark] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala new file mode 100644 index 0000000000000..287f731787f1b --- /dev/null +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -0,0 +1,156 @@ +package spark.scheduler + +import spark.Logging +import scala.collection.immutable.Set +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.hadoop.util.ReflectionUtils +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.conf.Configuration +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.JavaConversions._ + + +/** + * Parses and holds information about inputFormat (and files) specified as a parameter. + */ +class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], + val path: String) extends Logging { + + var mapreduceInputFormat: Boolean = false + var mapredInputFormat: Boolean = false + + validate() + + override def toString(): String = { + "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path + } + + override def hashCode(): Int = { + var hashCode = inputFormatClazz.hashCode + hashCode = hashCode * 31 + path.hashCode + hashCode + } + + // Since we are not doing canonicalization of path, this can be wrong : like relative vs absolute path + // .. which is fine, this is best case effort to remove duplicates - right ? + override def equals(other: Any): Boolean = other match { + case that: InputFormatInfo => { + // not checking config - that should be fine, right ? + this.inputFormatClazz == that.inputFormatClazz && + this.path == that.path + } + case _ => false + } + + private def validate() { + logDebug("validate InputFormatInfo : " + inputFormatClazz + ", path " + path) + + try { + if (classOf[org.apache.hadoop.mapreduce.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) { + logDebug("inputformat is from mapreduce package") + mapreduceInputFormat = true + } + else if (classOf[org.apache.hadoop.mapred.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) { + logDebug("inputformat is from mapred package") + mapredInputFormat = true + } + else { + throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + + " is NOT a supported input format ? does not implement either of the supported hadoop api's") + } + } + catch { + case e: ClassNotFoundException => { + throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + " cannot be found ?", e) + } + } + } + + + // This method does not expect failures, since validate has already passed ... + private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = { + val conf = new JobConf(configuration) + FileInputFormat.setInputPaths(conf, path) + + val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] = + ReflectionUtils.newInstance(inputFormatClazz.asInstanceOf[Class[_]], conf).asInstanceOf[ + org.apache.hadoop.mapreduce.InputFormat[_, _]] + val job = new Job(conf) + + val retval = new ArrayBuffer[SplitInfo]() + val list = instance.getSplits(job) + for (split <- list) { + retval ++= SplitInfo.toSplitInfo(inputFormatClazz, path, split) + } + + return retval.toSet + } + + // This method does not expect failures, since validate has already passed ... + private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = { + val jobConf = new JobConf(configuration) + FileInputFormat.setInputPaths(jobConf, path) + + val instance: org.apache.hadoop.mapred.InputFormat[_, _] = + ReflectionUtils.newInstance(inputFormatClazz.asInstanceOf[Class[_]], jobConf).asInstanceOf[ + org.apache.hadoop.mapred.InputFormat[_, _]] + + val retval = new ArrayBuffer[SplitInfo]() + instance.getSplits(jobConf, jobConf.getNumMapTasks()).foreach( + elem => retval ++= SplitInfo.toSplitInfo(inputFormatClazz, path, elem) + ) + + return retval.toSet + } + + private def findPreferredLocations(): Set[SplitInfo] = { + logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + + ", inputFormatClazz : " + inputFormatClazz) + if (mapreduceInputFormat) { + return prefLocsFromMapreduceInputFormat() + } + else { + assert(mapredInputFormat) + return prefLocsFromMapredInputFormat() + } + } +} + + + + +object InputFormatInfo { + /** + Computes the preferred locations based on input(s) and returned a location to block map. + Typical use of this method for allocation would follow some algo like this + (which is what we currently do in YARN branch) : + a) For each host, count number of splits hosted on that host. + b) Decrement the currently allocated containers on that host. + c) Compute rack info for each host and update rack -> count map based on (b). + d) Allocate nodes based on (c) + e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node + (even if data locality on that is very high) : this is to prevent fragility of job if a single + (or small set of) hosts go down. + + go to (a) until required nodes are allocated. + + If a node 'dies', follow same procedure. + + PS: I know the wording here is weird, hopefully it makes some sense ! + */ + def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] = { + + val nodeToSplit = new HashMap[String, HashSet[SplitInfo]] + for (inputSplit <- formats) { + val splits = inputSplit.findPreferredLocations() + + for (split <- splits){ + val location = split.hostLocation + val set = nodeToSplit.getOrElseUpdate(location, new HashSet[SplitInfo]) + set += split + } + } + + nodeToSplit + } +} diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index beb21a76fe5c8..89dc6640b2fe2 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -70,6 +70,14 @@ private[spark] class ResultTask[T, U]( rdd.partitions(partition) } + // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. + val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + + { + // DEBUG code + preferredLocs.foreach (host => Utils.checkHost(host, "preferredLocs : " + preferredLocs)) + } + override def run(attemptId: Long): U = { val context = new TaskContext(stageId, partition, attemptId) metrics = Some(context.taskMetrics) @@ -80,7 +88,7 @@ private[spark] class ResultTask[T, U]( } } - override def preferredLocations: Seq[String] = locs + override def preferredLocations: Seq[String] = preferredLocs override def toString = "ResultTask(" + stageId + ", " + partition + ")" diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 36d087a4d009c..7dc6da4573391 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -77,13 +77,21 @@ private[spark] class ShuffleMapTask( var rdd: RDD[_], var dep: ShuffleDependency[_,_], var partition: Int, - @transient var locs: Seq[String]) + @transient private var locs: Seq[String]) extends Task[MapStatus](stageId) with Externalizable with Logging { protected def this() = this(0, null, null, 0, null) + // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. + private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + + { + // DEBUG code + preferredLocs.foreach (host => Utils.checkHost(host, "preferredLocs : " + preferredLocs)) + } + var split = if (rdd == null) { null } else { @@ -154,7 +162,7 @@ private[spark] class ShuffleMapTask( } } - override def preferredLocations: Seq[String] = locs + override def preferredLocations: Seq[String] = preferredLocs override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partition) } diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/spark/scheduler/SplitInfo.scala new file mode 100644 index 0000000000000..6abfb7a1f7150 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/SplitInfo.scala @@ -0,0 +1,61 @@ +package spark.scheduler + +import collection.mutable.ArrayBuffer + +// information about a specific split instance : handles both split instances. +// So that we do not need to worry about the differences. +class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, + val length: Long, val underlyingSplit: Any) { + override def toString(): String = { + "SplitInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + + ", hostLocation : " + hostLocation + ", path : " + path + + ", length : " + length + ", underlyingSplit " + underlyingSplit + } + + override def hashCode(): Int = { + var hashCode = inputFormatClazz.hashCode + hashCode = hashCode * 31 + hostLocation.hashCode + hashCode = hashCode * 31 + path.hashCode + // ignore overflow ? It is hashcode anyway ! + hashCode = hashCode * 31 + (length & 0x7fffffff).toInt + hashCode + } + + // This is practically useless since most of the Split impl's dont seem to implement equals :-( + // So unless there is identity equality between underlyingSplits, it will always fail even if it + // is pointing to same block. + override def equals(other: Any): Boolean = other match { + case that: SplitInfo => { + this.hostLocation == that.hostLocation && + this.inputFormatClazz == that.inputFormatClazz && + this.path == that.path && + this.length == that.length && + // other split specific checks (like start for FileSplit) + this.underlyingSplit == that.underlyingSplit + } + case _ => false + } +} + +object SplitInfo { + + def toSplitInfo(inputFormatClazz: Class[_], path: String, + mapredSplit: org.apache.hadoop.mapred.InputSplit): Seq[SplitInfo] = { + val retval = new ArrayBuffer[SplitInfo]() + val length = mapredSplit.getLength + for (host <- mapredSplit.getLocations) { + retval += new SplitInfo(inputFormatClazz, host, path, length, mapredSplit) + } + retval + } + + def toSplitInfo(inputFormatClazz: Class[_], path: String, + mapreduceSplit: org.apache.hadoop.mapreduce.InputSplit): Seq[SplitInfo] = { + val retval = new ArrayBuffer[SplitInfo]() + val length = mapreduceSplit.getLength + for (host <- mapreduceSplit.getLocations) { + retval += new SplitInfo(inputFormatClazz, host, path, length, mapreduceSplit) + } + retval + } +} diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index d549b184b088d..7787b547620f6 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -10,6 +10,10 @@ package spark.scheduler private[spark] trait TaskScheduler { def start(): Unit + // Invoked after system has successfully initialized (typically in spark context). + // Yarn uses this to bootstrap allocation of resources based on preferred locations, wait for slave registerations, etc. + def postStartHook() { } + // Disconnect from the cluster. def stop(): Unit diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 771518dddfaca..b75d3736cf5d0 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -14,6 +14,9 @@ private[spark] trait TaskSchedulerListener { def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit + // A node was added to the cluster. + def executorGained(execId: String, hostPort: String): Unit + // A node was lost from the cluster. def executorLost(execId: String): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 26fdef101bb27..2e18d46edce4f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -1,6 +1,6 @@ package spark.scheduler.cluster -import java.io.{File, FileInputStream, FileOutputStream} +import java.lang.{Boolean => JBoolean} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -25,6 +25,30 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong + // How often to revive offers in case there are pending tasks - that is how often to try to get + // tasks scheduled in case there are nodes available : default 0 is to disable it - to preserve existing behavior + // Note that this is required due to delayed scheduling due to data locality waits, etc. + // TODO: rename property ? + val TASK_REVIVAL_INTERVAL = System.getProperty("spark.tasks.revive.interval", "0").toLong + + /* + This property controls how aggressive we should be to modulate waiting for host local task scheduling. + To elaborate, currently there is a time limit (3 sec def) to ensure that spark attempts to wait for host locality of tasks before + scheduling on other nodes. We have modified this in yarn branch such that offers to task set happen in prioritized order : + host-local, rack-local and then others + But once all available host local (and no pref) tasks are scheduled, instead of waiting for 3 sec before + scheduling to other nodes (which degrades performance for time sensitive tasks and on larger clusters), we can + modulate that : to also allow rack local nodes or any node. The default is still set to HOST - so that previous behavior is + maintained. This is to allow tuning the tension between pulling rdd data off node and scheduling computation asap. + + TODO: rename property ? The value is one of + - HOST_LOCAL (default, no change w.r.t current behavior), + - RACK_LOCAL and + - ANY + + Note that this property makes more sense when used in conjugation with spark.tasks.revive.interval > 0 : else it is not very effective. + */ + val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "HOST_LOCAL")) val activeTaskSets = new HashMap[String, TaskSetManager] var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -33,9 +57,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val taskIdToExecutorId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] - var hasReceivedTask = false - var hasLaunchedTask = false - val starvationTimer = new Timer(true) + @volatile private var hasReceivedTask = false + @volatile private var hasLaunchedTask = false + private val starvationTimer = new Timer(true) // Incrementing Mesos task IDs val nextTaskId = new AtomicLong(0) @@ -43,11 +67,16 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Which executor IDs we have executors on val activeExecutorIds = new HashSet[String] + // TODO: We might want to remove this and merge it with execId datastructures - but later. + // Which hosts in the cluster are alive (contains hostPort's) + private val hostPortsAlive = new HashSet[String] + private val hostToAliveHostPorts = new HashMap[String, HashSet[String]] + // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host - val executorsByHost = new HashMap[String, HashSet[String]] + val executorsByHostPort = new HashMap[String, HashSet[String]] - val executorIdToHost = new HashMap[String, String] + val executorIdToHostPort = new HashMap[String, String] // JAR server, if any JARs were added by the user to the SparkContext var jarServer: HttpServer = null @@ -75,11 +104,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def start() { backend.start() - if (System.getProperty("spark.speculation", "false") == "true") { + if (JBoolean.getBoolean("spark.speculation")) { new Thread("ClusterScheduler speculation check") { setDaemon(true) override def run() { + logInfo("Starting speculative execution thread") while (true) { try { Thread.sleep(SPECULATION_INTERVAL) @@ -91,6 +121,27 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } }.start() } + + + // Change to always run with some default if TASK_REVIVAL_INTERVAL <= 0 ? + if (TASK_REVIVAL_INTERVAL > 0) { + new Thread("ClusterScheduler task offer revival check") { + setDaemon(true) + + override def run() { + logInfo("Starting speculative task offer revival thread") + while (true) { + try { + Thread.sleep(TASK_REVIVAL_INTERVAL) + } catch { + case e: InterruptedException => {} + } + + if (hasPendingTasks()) backend.reviveOffers() + } + } + }.start() + } } override def submitTasks(taskSet: TaskSet) { @@ -139,22 +190,92 @@ private[spark] class ClusterScheduler(val sc: SparkContext) SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname for (o <- offers) { - executorIdToHost(o.executorId) = o.hostname - if (!executorsByHost.contains(o.hostname)) { - executorsByHost(o.hostname) = new HashSet() + // DEBUG Code + Utils.checkHostPort(o.hostPort) + + executorIdToHostPort(o.executorId) = o.hostPort + if (! executorsByHostPort.contains(o.hostPort)) { + executorsByHostPort(o.hostPort) = new HashSet[String]() } + + hostPortsAlive += o.hostPort + hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(o.hostPort)._1, new HashSet[String]).add(o.hostPort) + executorGained(o.executorId, o.hostPort) } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray var launchedTask = false + + for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { + + // Split offers based on host local, rack local and off-rack tasks. + val hostLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + val otherOffers = new HashMap[String, ArrayBuffer[Int]]() + + for (i <- 0 until offers.size) { + val hostPort = offers(i).hostPort + // DEBUG code + Utils.checkHostPort(hostPort) + val host = Utils.parseHostPort(hostPort)._1 + val numHostLocalTasks = math.max(0, math.min(manager.numPendingTasksForHost(hostPort), availableCpus(i))) + if (numHostLocalTasks > 0){ + val list = hostLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) + for (j <- 0 until numHostLocalTasks) list += i + } + + val numRackLocalTasks = math.max(0, + // Remove host local tasks (which are also rack local btw !) from this + math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numHostLocalTasks, availableCpus(i))) + if (numRackLocalTasks > 0){ + val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) + for (j <- 0 until numRackLocalTasks) list += i + } + if (numHostLocalTasks <= 0 && numRackLocalTasks <= 0){ + // add to others list - spread even this across cluster. + val list = otherOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) + list += i + } + } + + val offersPriorityList = new ArrayBuffer[Int]( + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) + // First host local, then rack, then others + val numHostLocalOffers = { + val hostLocalPriorityList = ClusterScheduler.prioritizeContainers(hostLocalOffers) + offersPriorityList ++= hostLocalPriorityList + hostLocalPriorityList.size + } + val numRackLocalOffers = { + val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) + offersPriorityList ++= rackLocalPriorityList + rackLocalPriorityList.size + } + offersPriorityList ++= ClusterScheduler.prioritizeContainers(otherOffers) + + var lastLoop = false + val lastLoopIndex = TASK_SCHEDULING_AGGRESSION match { + case TaskLocality.HOST_LOCAL => numHostLocalOffers + case TaskLocality.RACK_LOCAL => numRackLocalOffers + numHostLocalOffers + case TaskLocality.ANY => offersPriorityList.size + } + do { launchedTask = false - for (i <- 0 until offers.size) { + var loopCount = 0 + for (i <- offersPriorityList) { val execId = offers(i).executorId - val host = offers(i).hostname - manager.slaveOffer(execId, host, availableCpus(i)) match { + val hostPort = offers(i).hostPort + + // If last loop and within the lastLoopIndex, expand scope - else use null (which will use default/existing) + val overrideLocality = if (lastLoop && loopCount < lastLoopIndex) TASK_SCHEDULING_AGGRESSION else null + + // If last loop, override waiting for host locality - we scheduled all local tasks already and there might be more available ... + loopCount += 1 + + manager.slaveOffer(execId, hostPort, availableCpus(i), overrideLocality) match { case Some(task) => tasks(i) += task val tid = task.taskId @@ -162,15 +283,31 @@ private[spark] class ClusterScheduler(val sc: SparkContext) taskSetTaskIds(manager.taskSet.id) += tid taskIdToExecutorId(tid) = execId activeExecutorIds += execId - executorsByHost(host) += execId + executorsByHostPort(hostPort) += execId availableCpus(i) -= 1 launchedTask = true - + case None => {} } } + // Loop once more - when lastLoop = true, then we try to schedule task on all nodes irrespective of + // data locality (we still go in order of priority : but that would not change anything since + // if data local tasks had been available, we would have scheduled them already) + if (lastLoop) { + // prevent more looping + launchedTask = false + } else if (!lastLoop && !launchedTask) { + // Do this only if TASK_SCHEDULING_AGGRESSION != HOST_LOCAL + if (TASK_SCHEDULING_AGGRESSION != TaskLocality.HOST_LOCAL) { + // fudge launchedTask to ensure we loop once more + launchedTask = true + // dont loop anymore + lastLoop = true + } + } } while (launchedTask) } + if (tasks.size > 0) { hasLaunchedTask = true } @@ -256,10 +393,15 @@ private[spark] class ClusterScheduler(val sc: SparkContext) if (jarServer != null) { jarServer.stop() } + + // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. + // TODO: Do something better ! + Thread.sleep(5000L) } override def defaultParallelism() = backend.defaultParallelism() + // Check for speculatable tasks in all our active jobs. def checkSpeculatableTasks() { var shouldRevive = false @@ -273,12 +415,20 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } + // Check for pending tasks in all our active jobs. + def hasPendingTasks(): Boolean = { + synchronized { + activeTaskSetsQueue.exists( _.hasPendingTasks() ) + } + } + def executorLost(executorId: String, reason: ExecutorLossReason) { var failedExecutor: Option[String] = None + synchronized { if (activeExecutorIds.contains(executorId)) { - val host = executorIdToHost(executorId) - logError("Lost executor %s on %s: %s".format(executorId, host, reason)) + val hostPort = executorIdToHostPort(executorId) + logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) removeExecutor(executorId) failedExecutor = Some(executorId) } else { @@ -296,19 +446,95 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - /** Get a list of hosts that currently have executors */ - def hostsAlive: scala.collection.Set[String] = executorsByHost.keySet - /** Remove an executor from all our data structures and mark it as lost */ private def removeExecutor(executorId: String) { activeExecutorIds -= executorId - val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) + val hostPort = executorIdToHostPort(executorId) + if (hostPortsAlive.contains(hostPort)) { + // DEBUG Code + Utils.checkHostPort(hostPort) + + hostPortsAlive -= hostPort + hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(hostPort)._1, new HashSet[String]).remove(hostPort) + } + + val execs = executorsByHostPort.getOrElse(hostPort, new HashSet) execs -= executorId if (execs.isEmpty) { - executorsByHost -= host + executorsByHostPort -= hostPort } - executorIdToHost -= executorId - activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) + executorIdToHostPort -= executorId + activeTaskSetsQueue.foreach(_.executorLost(executorId, hostPort)) + } + + def executorGained(execId: String, hostPort: String) { + listener.executorGained(execId, hostPort) + } + + def getExecutorsAliveOnHost(host: String): Option[Set[String]] = { + val retval = hostToAliveHostPorts.get(host) + if (retval.isDefined) { + return Some(retval.get.toSet) + } + + None + } + + // By default, rack is unknown + def getRackForHost(value: String): Option[String] = None + + // By default, (cached) hosts for rack is unknown + def getCachedHostsForRack(rack: String): Option[Set[String]] = None +} + + +object ClusterScheduler { + + // Used to 'spray' available containers across the available set to ensure too many containers on same host + // are not used up. Used in yarn mode and in task scheduling (when there are multiple containers available + // to execute a task) + // For example: yarn can returns more containers than we would have requested under ANY, this method + // prioritizes how to use the allocated containers. + // flatten the map such that the array buffer entries are spread out across the returned value. + // given == , , , , , i + // the return value would be something like : h1c1, h2c1, h3c1, h4c1, h5c1, h1c2, h2c2, h3c2, h1c3, h2c3, h1c4, h1c5 + // We then 'use' the containers in this order (consuming only the top K from this list where + // K = number to be user). This is to ensure that if we have multiple eligible allocations, + // they dont end up allocating all containers on a small number of hosts - increasing probability of + // multiple container failure when a host goes down. + // Note, there is bias for keys with higher number of entries in value to be picked first (by design) + // Also note that invocation of this method is expected to have containers of same 'type' + // (host-local, rack-local, off-rack) and not across types : so that reordering is simply better from + // the available list - everything else being same. + // That is, we we first consume data local, then rack local and finally off rack nodes. So the + // prioritization from this method applies to within each category + def prioritizeContainers[K, T] (map: HashMap[K, ArrayBuffer[T]]): List[T] = { + val _keyList = new ArrayBuffer[K](map.size) + _keyList ++= map.keys + + // order keyList based on population of value in map + val keyList = _keyList.sortWith( + (left, right) => map.get(left).getOrElse(Set()).size > map.get(right).getOrElse(Set()).size + ) + + val retval = new ArrayBuffer[T](keyList.size * 2) + var index = 0 + var found = true + + while (found){ + found = false + for (key <- keyList) { + val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) + assert(containerList != null) + // Get the index'th entry for this host - if present + if (index < containerList.size){ + retval += containerList.apply(index) + found = true + } + } + index += 1 + } + + retval.toList } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index bb289c9cf391b..6b61152ed0374 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -27,7 +27,7 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTPORT}}", "{{CORES}}") val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) @@ -57,9 +57,9 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def executorAdded(executorId: String, workerId: String, host: String, cores: Int, memory: Int) { - logInfo("Granted executor ID %s on host %s with %d cores, %s RAM".format( - executorId, host, cores, Utils.memoryMegabytesToString(memory))) + override def executorAdded(executorId: String, workerId: String, hostPort: String, cores: Int, memory: Int) { + logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( + executorId, hostPort, cores, Utils.memoryMegabytesToString(memory))) } override def executorRemoved(executorId: String, message: String, exitStatus: Option[Int]) { diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index d7660678248b2..333529484460e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -3,6 +3,7 @@ package spark.scheduler.cluster import spark.TaskState.TaskState import java.nio.ByteBuffer import spark.util.SerializableBuffer +import spark.Utils private[spark] sealed trait StandaloneClusterMessage extends Serializable @@ -19,8 +20,10 @@ case class RegisterExecutorFailed(message: String) extends StandaloneClusterMess // Executors to driver private[spark] -case class RegisterExecutor(executorId: String, host: String, cores: Int) - extends StandaloneClusterMessage +case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) + extends StandaloneClusterMessage { + Utils.checkHostPort(hostPort, "Expected host port") +} private[spark] case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 7a428e3361976..c20276a605796 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -5,8 +5,9 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ import akka.util.duration._ import akka.pattern.ask +import akka.util.Duration -import spark.{SparkException, Logging, TaskState} +import spark.{Utils, SparkException, Logging, TaskState} import akka.dispatch.Await import java.util.concurrent.atomic.AtomicInteger import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} @@ -24,12 +25,12 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor var totalCoreCount = new AtomicInteger(0) class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { - val executorActor = new HashMap[String, ActorRef] - val executorAddress = new HashMap[String, Address] - val executorHost = new HashMap[String, String] - val freeCores = new HashMap[String, Int] - val actorToExecutorId = new HashMap[ActorRef, String] - val addressToExecutorId = new HashMap[Address, String] + private val executorActor = new HashMap[String, ActorRef] + private val executorAddress = new HashMap[String, Address] + private val executorHostPort = new HashMap[String, String] + private val freeCores = new HashMap[String, Int] + private val actorToExecutorId = new HashMap[ActorRef, String] + private val addressToExecutorId = new HashMap[Address, String] override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() @@ -37,7 +38,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } def receive = { - case RegisterExecutor(executorId, host, cores) => + case RegisterExecutor(executorId, hostPort, cores) => + Utils.checkHostPort(hostPort, "Host port expected " + hostPort) if (executorActor.contains(executorId)) { sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) } else { @@ -45,7 +47,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor sender ! RegisteredExecutor(sparkProperties) context.watch(sender) executorActor(executorId) = sender - executorHost(executorId) = host + executorHostPort(executorId) = hostPort freeCores(executorId) = cores executorAddress(executorId) = sender.path.address actorToExecutorId(sender) = executorId @@ -85,13 +87,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor // Make fake resource offers on all executors def makeOffers() { launchTasks(scheduler.resourceOffers( - executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) + executorHostPort.toArray.map {case (id, hostPort) => new WorkerOffer(id, hostPort, freeCores(id))})) } // Make fake resource offers on just one executor def makeOffers(executorId: String) { launchTasks(scheduler.resourceOffers( - Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId))))) + Seq(new WorkerOffer(executorId, executorHostPort(executorId), freeCores(executorId))))) } // Launch tasks returned by a set of resource offers @@ -110,9 +112,9 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor actorToExecutorId -= executorActor(executorId) addressToExecutorId -= executorAddress(executorId) executorActor -= executorId - executorHost -= executorId + executorHostPort -= executorId freeCores -= executorId - executorHost -= executorId + executorHostPort -= executorId totalCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) } @@ -128,7 +130,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor while (iterator.hasNext) { val entry = iterator.next val (key, value) = (entry.getKey.toString, entry.getValue.toString) - if (key.startsWith("spark.")) { + if (key.startsWith("spark.") && !key.equals("spark.hostPort")) { properties += ((key, value)) } } @@ -136,10 +138,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor Props(new DriverActor(properties)), name = StandaloneSchedulerBackend.ACTOR_NAME) } + private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + override def stop() { try { if (driverActor != null) { - val timeout = 5.seconds val future = driverActor.ask(StopDriver)(timeout) Await.result(future, timeout) } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index dfe3c5a85bc25..718f26bfbd74e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,5 +1,7 @@ package spark.scheduler.cluster +import spark.Utils + /** * Information about a running task attempt inside a TaskSet. */ @@ -9,8 +11,11 @@ class TaskInfo( val index: Int, val launchTime: Long, val executorId: String, - val host: String, - val preferred: Boolean) { + val hostPort: String, + val taskLocality: TaskLocality.TaskLocality) { + + Utils.checkHostPort(hostPort, "Expected hostport") + var finishTime: Long = 0 var failed = false diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index c9f2c488048ca..27e713e2c43dd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -1,7 +1,6 @@ package spark.scheduler.cluster -import java.util.Arrays -import java.util.{HashMap => JHashMap} +import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -14,6 +13,36 @@ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer +private[spark] object TaskLocality extends Enumeration("HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { + + val HOST_LOCAL, RACK_LOCAL, ANY = Value + + type TaskLocality = Value + + def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { + + constraint match { + case TaskLocality.HOST_LOCAL => condition == TaskLocality.HOST_LOCAL + case TaskLocality.RACK_LOCAL => condition == TaskLocality.HOST_LOCAL || condition == TaskLocality.RACK_LOCAL + // For anything else, allow + case _ => true + } + } + + def parse(str: String): TaskLocality = { + // better way to do this ? + try { + TaskLocality.withName(str) + } catch { + case nEx: NoSuchElementException => { + logWarning("Invalid task locality specified '" + str + "', defaulting to HOST_LOCAL"); + // default to preserve earlier behavior + HOST_LOCAL + } + } + } +} + /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ @@ -47,14 +76,22 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node. These collections are actually + // List of pending tasks for each node (hyper local to container). These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put // back at the head of the stack. They are also only cleaned up lazily; // when a task is launched, it remains in all the pending lists except // the one that it was launched from, but gets removed from them later. - val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] + private val pendingTasksForHostPort = new HashMap[String, ArrayBuffer[Int]] + + // List of pending tasks for each node. + // Essentially, similar to pendingTasksForHostPort, except at host level + private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] + + // List of pending tasks for each node based on rack locality. + // Essentially, similar to pendingTasksForHost, except at rack level + private val pendingRackLocalTasksForHost = new HashMap[String, ArrayBuffer[Int]] // List containing pending tasks with no locality preferences val pendingTasksWithNoPrefs = new ArrayBuffer[Int] @@ -96,26 +133,117 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } + private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, rackLocal: Boolean = false): ArrayBuffer[String] = { + // DEBUG code + _taskPreferredLocations.foreach(h => Utils.checkHost(h, "taskPreferredLocation " + _taskPreferredLocations)) + + val taskPreferredLocations = if (! rackLocal) _taskPreferredLocations else { + // Expand set to include all 'seen' rack local hosts. + // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // Best case effort, and maybe sort of kludge for now ... rework it later ? + val hosts = new HashSet[String] + _taskPreferredLocations.foreach(h => { + val rackOpt = scheduler.getRackForHost(h) + if (rackOpt.isDefined) { + val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) + if (hostsOpt.isDefined) { + hosts ++= hostsOpt.get + } + } + + // Ensure that irrespective of what scheduler says, host is always added ! + hosts += h + }) + + hosts + } + + val retval = new ArrayBuffer[String] + scheduler.synchronized { + for (prefLocation <- taskPreferredLocations) { + val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(prefLocation) + if (aliveLocationsOpt.isDefined) { + retval ++= aliveLocationsOpt.get + } + } + } + + retval + } + // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { - val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive - if (locations.size == 0) { + // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate + // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. + val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched) + val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, true) + + if (rackLocalLocations.size == 0) { + // Current impl ensures this. + assert (hostLocalLocations.size == 0) pendingTasksWithNoPrefs += index } else { - for (host <- locations) { - val list = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) + + // host locality + for (hostPort <- hostLocalLocations) { + // DEBUG Code + Utils.checkHostPort(hostPort) + + val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer()) + hostPortList += index + + val host = Utils.parseHostPort(hostPort)._1 + val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) + hostList += index + } + + // rack locality + for (rackLocalHostPort <- rackLocalLocations) { + // DEBUG Code + Utils.checkHostPort(rackLocalHostPort) + + val rackLocalHost = Utils.parseHostPort(rackLocalHostPort)._1 + val list = pendingRackLocalTasksForHost.getOrElseUpdate(rackLocalHost, ArrayBuffer()) list += index } } + allPendingTasks += index } + // Return the pending tasks list for a given host port (hyper local), or an empty list if + // there is no map entry for that host + private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { + // DEBUG Code + Utils.checkHostPort(hostPort) + pendingTasksForHostPort.getOrElse(hostPort, ArrayBuffer()) + } + // Return the pending tasks list for a given host, or an empty list if // there is no map entry for that host - private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { + private def getPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { + val host = Utils.parseHostPort(hostPort)._1 pendingTasksForHost.getOrElse(host, ArrayBuffer()) } + // Return the pending tasks (rack level) list for a given host, or an empty list if + // there is no map entry for that host + private def getRackLocalPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { + val host = Utils.parseHostPort(hostPort)._1 + pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) + } + + // Number of pending tasks for a given host (which would be data local) + def numPendingTasksForHost(hostPort: String): Int = { + getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + // Number of pending rack local tasks for a given host + def numRackLocalPendingTasksForHost(hostPort: String): Int = { + getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + // Dequeue a pending task from the given list and return its index. // Return None if the list is empty. // This method also cleans up any tasks in the list that have already @@ -132,26 +260,49 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } // Return a speculative task for a given host if any are available. The task should not have an - // attempt running on this host, in case the host is slow. In addition, if localOnly is set, the - // task must have a preference for this host (or no preferred locations at all). - private def findSpeculativeTask(host: String, localOnly: Boolean): Option[Int] = { - val hostsAlive = sched.hostsAlive + // attempt running on this host, in case the host is slow. In addition, if locality is set, the + // task must have a preference for this host/rack/no preferred locations at all. + private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + + assert (TaskLocality.isAllowed(locality, TaskLocality.HOST_LOCAL)) speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set - val localTask = speculatableTasks.find { - index => - val locations = tasks(index).preferredLocations.toSet & hostsAlive - val attemptLocs = taskAttempts(index).map(_.host) - (locations.size == 0 || locations.contains(host)) && !attemptLocs.contains(host) + + if (speculatableTasks.size > 0) { + val localTask = speculatableTasks.find { + index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched) + val attemptLocs = taskAttempts(index).map(_.hostPort) + (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) + } + + if (localTask != None) { + speculatableTasks -= localTask.get + return localTask } - if (localTask != None) { - speculatableTasks -= localTask.get - return localTask - } - if (!localOnly && speculatableTasks.size > 0) { - val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.host).contains(host)) - if (nonLocalTask != None) { - speculatableTasks -= nonLocalTask.get - return nonLocalTask + + // check for rack locality + if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + val rackTask = speculatableTasks.find { + index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, true) + val attemptLocs = taskAttempts(index).map(_.hostPort) + locations.contains(hostPort) && !attemptLocs.contains(hostPort) + } + + if (rackTask != None) { + speculatableTasks -= rackTask.get + return rackTask + } + } + + // Any task ... + if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { + // Check for attemptLocs also ? + val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) + if (nonLocalTask != None) { + speculatableTasks -= nonLocalTask.get + return nonLocalTask + } } } return None @@ -159,59 +310,103 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. - private def findTask(host: String, localOnly: Boolean): Option[Int] = { - val localTask = findTaskFromList(getPendingTasksForHost(host)) + private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) if (localTask != None) { return localTask } + + if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort)) + if (rackLocalTask != None) { + return rackLocalTask + } + } + + // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. + // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) if (noPrefTask != None) { return noPrefTask } - if (!localOnly) { + + if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { val nonLocalTask = findTaskFromList(allPendingTasks) if (nonLocalTask != None) { return nonLocalTask } } + // Finally, if all else has failed, find a speculative task - return findSpeculativeTask(host, localOnly) + return findSpeculativeTask(hostPort, locality) } // Does a host count as a preferred location for a task? This is true if // either the task has preferred locations and this host is one, or it has // no preferred locations (in which we still count the launch as preferred). - private def isPreferredLocation(task: Task[_], host: String): Boolean = { + private def isPreferredLocation(task: Task[_], hostPort: String): Boolean = { val locs = task.preferredLocations - return (locs.contains(host) || locs.isEmpty) + // DEBUG code + locs.foreach(h => Utils.checkHost(h, "preferredLocation " + locs)) + + if (locs.contains(hostPort) || locs.isEmpty) return true + + val host = Utils.parseHostPort(hostPort)._1 + locs.contains(host) + } + + // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). + // This is true if either the task has preferred locations and this host is one, or it has + // no preferred locations (in which we still count the launch as preferred). + def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { + + val locs = task.preferredLocations + + // DEBUG code + locs.foreach(h => Utils.checkHost(h, "preferredLocation " + locs)) + + val preferredRacks = new HashSet[String]() + for (preferredHost <- locs) { + val rack = sched.getRackForHost(preferredHost) + if (None != rack) preferredRacks += rack.get + } + + if (preferredRacks.isEmpty) return false + + val hostRack = sched.getRackForHost(hostPort) + + return None != hostRack && preferredRacks.contains(hostRack.get) } // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { - val time = System.currentTimeMillis - val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) + // If explicitly specified, use that + val locality = if (overrideLocality != null) overrideLocality else { + // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... + val time = System.currentTimeMillis + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.HOST_LOCAL else TaskLocality.ANY + } - findTask(host, localOnly) match { + findTask(hostPort, locality) match { case Some(index) => { // Found a task; do some bookkeeping and return a Mesos task for it val task = tasks(index) val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch - val preferred = isPreferredLocation(task, host) - val prefStr = if (preferred) { - "preferred" - } else { - "non-preferred, not one of " + task.preferredLocations.mkString(", ") - } - logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format( - taskSet.id, index, taskId, execId, host, prefStr)) + val taskLocality = if (isPreferredLocation(task, hostPort)) TaskLocality.HOST_LOCAL else + if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else TaskLocality.ANY + val prefStr = taskLocality.toString + logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( + taskSet.id, index, taskId, execId, hostPort, prefStr)) // Do various bookkeeping copiesRunning(index) += 1 - val info = new TaskInfo(taskId, index, time, execId, host, preferred) + val time = System.currentTimeMillis + val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) - if (preferred) { + if (TaskLocality.HOST_LOCAL == taskLocality) { lastPreferredLaunchTime = time } // Serialize and return the task @@ -355,17 +550,15 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe sched.taskSetFinished(this) } - def executorLost(execId: String, hostname: String) { + def executorLost(execId: String, hostPort: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) - val newHostsAlive = sched.hostsAlive // If some task has preferred locations only on hostname, and there are no more executors there, // put it in the no-prefs list to avoid the wait from delay scheduling - if (!newHostsAlive.contains(hostname)) { - for (index <- getPendingTasksForHost(hostname)) { - val newLocs = tasks(index).preferredLocations.toSet & newHostsAlive - if (newLocs.isEmpty) { - pendingTasksWithNoPrefs += index - } + for (index <- getPendingTasksForHostPort(hostPort)) { + val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, true) + if (newLocs.isEmpty) { + assert (findPreferredLocations(tasks(index).preferredLocations, sched).isEmpty) + pendingTasksWithNoPrefs += index } } // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage @@ -419,7 +612,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe !speculatableTasks.contains(index)) { logInfo( "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( - taskSet.id, index, info.host, threshold)) + taskSet.id, index, info.hostPort, threshold)) speculatableTasks += index foundTasks = true } @@ -427,4 +620,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } return foundTasks } + + def hasPendingTasks(): Boolean = { + numTasks > 0 && tasksFinished < numTasks + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala index 3c3afcbb14d3f..c47824315ccbd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala @@ -4,5 +4,5 @@ package spark.scheduler.cluster * Represents free resources available on an executor. */ private[spark] -class WorkerOffer(val executorId: String, val hostname: String, val cores: Int) { +class WorkerOffer(val executorId: String, val hostPort: String, val cores: Int) { } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 9e1bde3fbe44e..f060a940a9269 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -7,7 +7,7 @@ import scala.collection.mutable.HashMap import spark._ import spark.executor.ExecutorURLClassLoader import spark.scheduler._ -import spark.scheduler.cluster.TaskInfo +import spark.scheduler.cluster.{TaskLocality, TaskInfo} /** * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally @@ -53,7 +53,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon def runTask(task: Task[_], idInJob: Int, attemptId: Int) { logInfo("Running " + task) - val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local", true) + val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.HOST_LOCAL) // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 210061e9726b4..10e70723db549 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -37,17 +37,27 @@ class BlockManager( maxMemory: Long) extends Logging { - class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { - var pending: Boolean = true - var size: Long = -1L - var failed: Boolean = false + private class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { + @volatile var pending: Boolean = true + @volatile var size: Long = -1L + @volatile var initThread: Thread = null + @volatile var failed = false + + setInitThread() + + private def setInitThread() { + // Set current thread as init thread - waitForReady will not block this thread + // (in case there is non trivial initialization which ends up calling waitForReady as part of + // initialization itself) + this.initThread = Thread.currentThread() + } /** * Wait for this BlockInfo to be marked as ready (i.e. block is finished writing). * Return true if the block is available, false otherwise. */ def waitForReady(): Boolean = { - if (pending) { + if (initThread != Thread.currentThread() && pending) { synchronized { while (pending) this.wait() } @@ -57,19 +67,26 @@ class BlockManager( /** Mark this BlockInfo as ready (i.e. block is finished writing) */ def markReady(sizeInBytes: Long) { + assert (pending) + size = sizeInBytes + initThread = null + failed = false + initThread = null + pending = false synchronized { - pending = false - failed = false - size = sizeInBytes this.notifyAll() } } /** Mark this BlockInfo as ready but failed */ def markFailure() { + assert (pending) + size = 0 + initThread = null + failed = true + initThread = null + pending = false synchronized { - failed = true - pending = false this.notifyAll() } } @@ -101,7 +118,7 @@ class BlockManager( val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties - val host = System.getProperty("spark.hostname", Utils.localHostName()) + val hostPort = Utils.localHostPort() val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) @@ -212,9 +229,12 @@ class BlockManager( * Tell the master about the current storage status of a block. This will send a block update * message reflecting the current status, *not* the desired storage level in its block info. * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. + * + * droppedMemorySize exists to account for when block is dropped from memory to disk (so it is still valid). + * This ensures that update in master will compensate for the increase in memory on slave. */ - def reportBlockStatus(blockId: String, info: BlockInfo) { - val needReregister = !tryToReportBlockStatus(blockId, info) + def reportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L) { + val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize) if (needReregister) { logInfo("Got told to reregister updating block " + blockId) // Reregistering will report our new block for free. @@ -228,7 +248,7 @@ class BlockManager( * which will be true if the block was successfully recorded and false if * the slave needs to re-register. */ - private def tryToReportBlockStatus(blockId: String, info: BlockInfo): Boolean = { + private def tryToReportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = { val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { info.level match { case null => @@ -237,7 +257,7 @@ class BlockManager( val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication) - val memSize = if (inMem) memoryStore.getSize(blockId) else 0L + val memSize = if (inMem) memoryStore.getSize(blockId) else droppedMemorySize val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L (storageLevel, memSize, diskSize, info.tellMaster) } @@ -257,7 +277,7 @@ class BlockManager( def getLocations(blockId: String): Seq[String] = { val startTimeMs = System.currentTimeMillis var managers = master.getLocations(blockId) - val locations = managers.map(_.ip) + val locations = managers.map(_.hostPort) logDebug("Got block locations in " + Utils.getUsedTimeMs(startTimeMs)) return locations } @@ -267,7 +287,7 @@ class BlockManager( */ def getLocations(blockIds: Array[String]): Array[Seq[String]] = { val startTimeMs = System.currentTimeMillis - val locations = master.getLocations(blockIds).map(_.map(_.ip).toSeq).toArray + val locations = master.getLocations(blockIds).map(_.map(_.hostPort).toSeq).toArray logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) return locations } @@ -339,6 +359,8 @@ class BlockManager( case Some(bytes) => // Put a copy of the block back in memory before returning it. Note that we can't // put the ByteBuffer returned by the disk store as that's a memory-mapped file. + // The use of rewind assumes this. + assert (0 == bytes.position()) val copyForMemory = ByteBuffer.allocate(bytes.limit) copyForMemory.put(bytes) memoryStore.putBytes(blockId, copyForMemory, level) @@ -411,6 +433,7 @@ class BlockManager( // Read it as a byte buffer into memory first, then return it diskStore.getBytes(blockId) match { case Some(bytes) => + assert (0 == bytes.position()) if (level.useMemory) { if (level.deserialized) { memoryStore.putBytes(blockId, bytes, level) @@ -450,7 +473,7 @@ class BlockManager( for (loc <- locations) { logDebug("Getting remote block " + blockId + " from " + loc) val data = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(loc.ip, loc.port)) + GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) if (data != null) { return Some(dataDeserialize(blockId, data)) } @@ -501,17 +524,17 @@ class BlockManager( throw new IllegalArgumentException("Storage level is null or invalid") } - val oldBlock = blockInfo.get(blockId).orNull - if (oldBlock != null && oldBlock.waitForReady()) { - logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - return oldBlock.size - } - // Remember the block's storage level so that we can correctly drop it to disk if it needs // to be dropped right after it got put into memory. Note, however, that other threads will // not be able to get() this block until we call markReady on its BlockInfo. val myInfo = new BlockInfo(level, tellMaster) - blockInfo.put(blockId, myInfo) + // Do atomically ! + val oldBlockOpt = blockInfo.putIfAbsent(blockId, myInfo) + + if (oldBlockOpt.isDefined && oldBlockOpt.get.waitForReady()) { + logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + return oldBlockOpt.get.size + } val startTimeMs = System.currentTimeMillis @@ -531,6 +554,7 @@ class BlockManager( logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") + var marked = false try { if (level.useMemory) { // Save it just to memory first, even if it also has useDisk set to true; we will later @@ -555,20 +579,20 @@ class BlockManager( // Now that the block is in either the memory or disk store, let other threads read it, // and tell the master about it. + marked = true myInfo.markReady(size) if (tellMaster) { reportBlockStatus(blockId, myInfo) } - } catch { + } finally { // If we failed at putting the block to memory/disk, notify other possible readers // that it has failed, and then remove it from the block info map. - case e: Exception => { + if (! marked) { // Note that the remove must happen before markFailure otherwise another thread // could've inserted a new BlockInfo before we remove it. blockInfo.remove(blockId) myInfo.markFailure() - logWarning("Putting block " + blockId + " failed", e) - throw e + logWarning("Putting block " + blockId + " failed") } } } @@ -611,16 +635,17 @@ class BlockManager( throw new IllegalArgumentException("Storage level is null or invalid") } - if (blockInfo.contains(blockId)) { - logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - return - } - // Remember the block's storage level so that we can correctly drop it to disk if it needs // to be dropped right after it got put into memory. Note, however, that other threads will // not be able to get() this block until we call markReady on its BlockInfo. val myInfo = new BlockInfo(level, tellMaster) - blockInfo.put(blockId, myInfo) + // Do atomically ! + val prevInfo = blockInfo.putIfAbsent(blockId, myInfo) + if (prevInfo != null) { + // Should we check for prevInfo.waitForReady() here ? + logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + return + } val startTimeMs = System.currentTimeMillis @@ -639,6 +664,7 @@ class BlockManager( logDebug("PutBytes for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") + var marked = false try { if (level.useMemory) { // Store it only in memory at first, even if useDisk is also set to true @@ -649,22 +675,24 @@ class BlockManager( diskStore.putBytes(blockId, bytes, level) } + // assert (0 == bytes.position(), "" + bytes) + // Now that the block is in either the memory or disk store, let other threads read it, // and tell the master about it. + marked = true myInfo.markReady(bytes.limit) if (tellMaster) { reportBlockStatus(blockId, myInfo) } - } catch { + } finally { // If we failed at putting the block to memory/disk, notify other possible readers // that it has failed, and then remove it from the block info map. - case e: Exception => { + if (! marked) { // Note that the remove must happen before markFailure otherwise another thread // could've inserted a new BlockInfo before we remove it. blockInfo.remove(blockId) myInfo.markFailure() - logWarning("Putting block " + blockId + " failed", e) - throw e + logWarning("Putting block " + blockId + " failed") } } } @@ -698,7 +726,7 @@ class BlockManager( logDebug("Try to replicate BlockId " + blockId + " once; The size of the data is " + data.limit() + " Bytes. To node: " + peer) if (!BlockManagerWorker.syncPutBlock(PutBlock(blockId, data, tLevel), - new ConnectionManagerId(peer.ip, peer.port))) { + new ConnectionManagerId(peer.host, peer.port))) { logError("Failed to call syncPutBlock to " + peer) } logDebug("Replicated BlockId " + blockId + " once used " + @@ -730,6 +758,14 @@ class BlockManager( val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { + // required ? As of now, this will be invoked only for blocks which are ready + // But in case this changes in future, adding for consistency sake. + if (! info.waitForReady() ) { + // If we get here, the block write failed. + logWarning("Block " + blockId + " was marked as failure. Nothing to drop") + return + } + val level = info.level if (level.useDisk && !diskStore.contains(blockId)) { logInfo("Writing block " + blockId + " to disk") @@ -740,12 +776,13 @@ class BlockManager( diskStore.putBytes(blockId, bytes, level) } } + val droppedMemorySize = memoryStore.getSize(blockId) val blockWasRemoved = memoryStore.remove(blockId) if (!blockWasRemoved) { logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") } if (info.tellMaster) { - reportBlockStatus(blockId, info) + reportBlockStatus(blockId, info, droppedMemorySize) } if (!level.useDisk) { // The block is completely gone from this node; forget it so we can put() it again later. @@ -938,8 +975,8 @@ class BlockFetcherIterator( def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) - val cmId = new ConnectionManagerId(req.address.ip, req.address.port) + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort)) + val cmId = new ConnectionManagerId(req.address.host, req.address.port) val blockMessageArray = new BlockMessageArray(req.blocks.map { case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) }) diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index f2f1e77d41a65..f4a2181490f9f 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -2,6 +2,7 @@ package spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap +import spark.Utils /** * This class represent an unique identifier for a BlockManager. @@ -13,7 +14,7 @@ import java.util.concurrent.ConcurrentHashMap */ private[spark] class BlockManagerId private ( private var executorId_ : String, - private var ip_ : String, + private var host_ : String, private var port_ : Int ) extends Externalizable { @@ -21,32 +22,45 @@ private[spark] class BlockManagerId private ( def executorId: String = executorId_ - def ip: String = ip_ + if (null != host_){ + Utils.checkHost(host_, "Expected hostname") + assert (port_ > 0) + } + + def hostPort: String = { + // DEBUG code + Utils.checkHost(host) + assert (port > 0) + + host + ":" + port + } + + def host: String = host_ def port: Int = port_ override def writeExternal(out: ObjectOutput) { out.writeUTF(executorId_) - out.writeUTF(ip_) + out.writeUTF(host_) out.writeInt(port_) } override def readExternal(in: ObjectInput) { executorId_ = in.readUTF() - ip_ = in.readUTF() + host_ = in.readUTF() port_ = in.readInt() } @throws(classOf[IOException]) private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) - override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, ip, port) + override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, host, port) - override def hashCode: Int = (executorId.hashCode * 41 + ip.hashCode) * 41 + port + override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port override def equals(that: Any) = that match { case id: BlockManagerId => - executorId == id.executorId && port == id.port && ip == id.ip + executorId == id.executorId && port == id.port && host == id.host case _ => false } @@ -55,8 +69,8 @@ private[spark] class BlockManagerId private ( private[spark] object BlockManagerId { - def apply(execId: String, ip: String, port: Int) = - getCachedBlockManagerId(new BlockManagerId(execId, ip, port)) + def apply(execId: String, host: String, port: Int) = + getCachedBlockManagerId(new BlockManagerId(execId, host, port)) def apply(in: ObjectInput) = { val obj = new BlockManagerId() @@ -67,11 +81,7 @@ private[spark] object BlockManagerId { val blockManagerIdCache = new ConcurrentHashMap[BlockManagerId, BlockManagerId]() def getCachedBlockManagerId(id: BlockManagerId): BlockManagerId = { - if (blockManagerIdCache.containsKey(id)) { - blockManagerIdCache.get(id) - } else { - blockManagerIdCache.put(id, id) - id - } + blockManagerIdCache.putIfAbsent(id, id) + blockManagerIdCache.get(id) } } diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 2830bc62975ae..3ce1e6e257796 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -332,8 +332,8 @@ object BlockManagerMasterActor { // Mapping from block id to its status. private val _blocks = new JHashMap[String, BlockStatus] - logInfo("Registering block manager %s:%d with %s RAM".format( - blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(maxMem))) + logInfo("Registering block manager %s with %s RAM".format( + blockManagerId.hostPort, Utils.memoryBytesToString(maxMem))) def updateLastSeenMs() { _lastSeenMs = System.currentTimeMillis() @@ -358,13 +358,13 @@ object BlockManagerMasterActor { _blocks.put(blockId, BlockStatus(storageLevel, memSize, diskSize)) if (storageLevel.useMemory) { _remainingMem -= memSize - logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), + logInfo("Added %s in memory on %s (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.memoryBytesToString(memSize), Utils.memoryBytesToString(_remainingMem))) } if (storageLevel.useDisk) { - logInfo("Added %s on disk on %s:%d (size: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) + logInfo("Added %s on disk on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.memoryBytesToString(diskSize))) } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. @@ -372,13 +372,13 @@ object BlockManagerMasterActor { _blocks.remove(blockId) if (blockStatus.storageLevel.useMemory) { _remainingMem += blockStatus.memSize - logInfo("Removed %s on %s:%d in memory (size: %s, free: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), + logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.memoryBytesToString(memSize), Utils.memoryBytesToString(_remainingMem))) } if (blockStatus.storageLevel.useDisk) { - logInfo("Removed %s on %s:%d on disk (size: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) + logInfo("Removed %s on %s on disk (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.memoryBytesToString(diskSize))) } } } diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index a25decb123d86..ee0c5ff9a2d1c 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -115,6 +115,7 @@ private[spark] object BlockMessageArray { val newBuffer = ByteBuffer.allocate(totalSize) newBuffer.clear() bufferMessage.buffers.foreach(buffer => { + assert (0 == buffer.position()) newBuffer.put(buffer) buffer.rewind() }) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index ddbf8821ad15a..c9553d2e0fa9e 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -20,6 +20,9 @@ import spark.Utils private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { + private val mapMode = MapMode.READ_ONLY + private var mapOpenMode = "r" + val MAX_DIR_CREATION_ATTEMPTS: Int = 10 val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt @@ -35,7 +38,10 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) getFile(blockId).length() } - override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + override def putBytes(blockId: String, _bytes: ByteBuffer, level: StorageLevel) { + // So that we do not modify the input offsets ! + // duplicate does not copy buffer, so inexpensive + val bytes = _bytes.duplicate() logDebug("Attempting to put block " + blockId) val startTime = System.currentTimeMillis val file = createFile(blockId) @@ -49,6 +55,18 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) blockId, Utils.memoryBytesToString(bytes.limit), (finishTime - startTime))) } + private def getFileBytes(file: File): ByteBuffer = { + val length = file.length() + val channel = new RandomAccessFile(file, mapOpenMode).getChannel() + val buffer = try { + channel.map(mapMode, 0, length) + } finally { + channel.close() + } + + buffer + } + override def putValues( blockId: String, values: ArrayBuffer[Any], @@ -70,9 +88,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) if (returnValues) { // Return a byte buffer for the contents of the file - val channel = new RandomAccessFile(file, "r").getChannel() - val buffer = channel.map(MapMode.READ_ONLY, 0, length) - channel.close() + val buffer = getFileBytes(file) PutResult(length, Right(buffer)) } else { PutResult(length, null) @@ -81,10 +97,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def getBytes(blockId: String): Option[ByteBuffer] = { val file = getFile(blockId) - val length = file.length().toInt - val channel = new RandomAccessFile(file, "r").getChannel() - val bytes = channel.map(MapMode.READ_ONLY, 0, length) - channel.close() + val bytes = getFileBytes(file) Some(bytes) } @@ -96,7 +109,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val file = getFile(blockId) if (file.exists()) { file.delete() - true } else { false } @@ -175,11 +187,12 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } private def addShutdownHook() { + localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir) ) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") try { - localDirs.foreach(localDir => Utils.deleteRecursively(localDir)) + localDirs.foreach(localDir => if (! Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir)) } catch { case t: Throwable => logError("Exception while deleting local spark dirs", t) } diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index 949588476c201..eba5ee507ff0c 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -31,7 +31,9 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + override def putBytes(blockId: String, _bytes: ByteBuffer, level: StorageLevel) { + // Work on a duplicate - since the original input might be used elsewhere. + val bytes = _bytes.duplicate() bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index 3b5a77ab228bb..cc0c354e7e9ea 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -123,11 +123,7 @@ object StorageLevel { val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = { - if (storageLevelCache.containsKey(level)) { - storageLevelCache.get(level) - } else { - storageLevelCache.put(level, level) - level - } + storageLevelCache.putIfAbsent(level, level) + storageLevelCache.get(level) } } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 3e805b78314c7..9fb7e001badcb 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -11,7 +11,7 @@ import cc.spray.{SprayCanRootService, HttpService} import cc.spray.can.server.HttpServer import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler import akka.dispatch.Await -import spark.SparkException +import spark.{Utils, SparkException} import java.util.concurrent.TimeoutException /** @@ -31,7 +31,10 @@ private[spark] object AkkaUtils { val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt - val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean + val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + // 10 seconds is the default akka timeout, but in a cluster, we need higher by default. + val akkaWriteTimeout = System.getProperty("spark.akka.writeTimeout", "30").toInt + val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] @@ -45,8 +48,9 @@ private[spark] object AkkaUtils { akka.remote.netty.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d akka.remote.log-remote-lifecycle-events = %s + akka.remote.netty.write-timeout = %ds """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, - if (lifecycleEvents) "on" else "off")) + lifecycleEvents, akkaWriteTimeout)) val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader) @@ -60,8 +64,9 @@ private[spark] object AkkaUtils { /** * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to * handle requests. Returns the bound port or throws a SparkException on failure. + * TODO: Not changing ip to host here - is it required ? */ - def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, + def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, name: String = "HttpServer"): ActorRef = { val ioWorker = new IoWorker(actorSystem).start() val httpService = actorSystem.actorOf(Props(new HttpService(route))) diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index 188f8910da8d5..92dfaa6e6f3d1 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -3,6 +3,7 @@ package spark.util import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Map +import spark.scheduler.MapStatus /** * This is a custom implementation of scala.collection.mutable.Map which stores the insertion @@ -42,6 +43,13 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging { this } + // Should we return previous value directly or as Option ? + def putIfAbsent(key: A, value: B): Option[B] = { + val prev = internalMap.putIfAbsent(key, (value, currentTime)) + if (prev != null) Some(prev._1) else None + } + + override def -= (key: A): this.type = { internalMap.remove(key) this diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html index ac51a39a5199d..b9b9f08810930 100644 --- a/core/src/main/twirl/spark/deploy/master/index.scala.html +++ b/core/src/main/twirl/spark/deploy/master/index.scala.html @@ -2,7 +2,7 @@ @import spark.deploy.master._ @import spark.Utils -@spark.common.html.layout(title = "Spark Master on " + state.host) { +@spark.common.html.layout(title = "Spark Master on " + state.host + ":" + state.port) {
    diff --git a/core/src/main/twirl/spark/deploy/worker/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html index c39f769a7387f..0e66af9284762 100644 --- a/core/src/main/twirl/spark/deploy/worker/index.scala.html +++ b/core/src/main/twirl/spark/deploy/worker/index.scala.html @@ -1,7 +1,7 @@ @(worker: spark.deploy.WorkerState) @import spark.Utils -@spark.common.html.layout(title = "Spark Worker on " + worker.host) { +@spark.common.html.layout(title = "Spark Worker on " + worker.host + ":" + worker.port) {
    diff --git a/core/src/main/twirl/spark/storage/worker_table.scala.html b/core/src/main/twirl/spark/storage/worker_table.scala.html index d54b8de4cc813..cd72a688c1ffb 100644 --- a/core/src/main/twirl/spark/storage/worker_table.scala.html +++ b/core/src/main/twirl/spark/storage/worker_table.scala.html @@ -12,7 +12,7 @@
    @for(status <- workersStatusList) { - + diff --git a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html index be69e9bf028b0..46277ca421bd2 100644 --- a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html @@ -4,7 +4,7 @@ From 42b1953c5385135467b87e79266a0f6c23d63e7b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 4 May 2013 13:23:31 -0700 Subject: [PATCH 541/696] Fix SPARK-630: app details page shows finished executors as running. --- core/src/main/scala/spark/deploy/master/Master.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 160afe5239d98..707fe57983e3f 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -275,6 +275,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) exec.worker.actor ! KillExecutor(exec.application.id, exec.id) + exec.state = ExecutorState.KILLED } app.markFinished(state) app.driver ! ApplicationRemoved(state.toString) From 0a2bed356b9ea604d317be97a6747588c5af29e4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 4 May 2013 21:50:08 -0700 Subject: [PATCH 542/696] Fixed flaky unpersist test in DistributedSuite. --- core/src/test/scala/spark/DistributedSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index a13c88cfb4bd5..4df3bb5b6793b 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -276,7 +276,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter // is racing this thread to remove entries from the driver. } } - assert(sc.getRDDStorageInfo.isEmpty === true) } } From e014c1d1cb4bd1037dc674ef474d0197267b399b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 5 May 2013 11:30:36 -0700 Subject: [PATCH 543/696] Fix SPARK-670: EC2 start command should require -i option. --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 9f2daad2b6323..7affe6fffc9e1 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -103,7 +103,7 @@ def parse_args(): parser.print_help() sys.exit(1) (action, cluster_name) = args - if opts.identity_file == None and action in ['launch', 'login']: + if opts.identity_file == None and action in ['launch', 'login', 'start']: print >> stderr, ("ERROR: The -i or --identity-file argument is " + "required for " + action) sys.exit(1) From 22a5063ae45300cdd759f51877dac850008e16ee Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 5 May 2013 12:19:11 -0700 Subject: [PATCH 544/696] switch from separating appUI host & port to combining into just appUiUrl --- .../main/scala/spark/deploy/ApplicationDescription.scala | 3 +-- core/src/main/scala/spark/deploy/client/TestClient.scala | 2 +- .../main/scala/spark/deploy/master/ApplicationInfo.scala | 6 +----- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 2 +- core/src/main/scala/spark/storage/BlockManagerUI.scala | 2 ++ .../main/twirl/spark/deploy/master/app_details.scala.html | 2 +- 7 files changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index bb9e7b3bba6f8..4aff0aedc1e2b 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -6,8 +6,7 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, val sparkHome: String, - val appUIHost: String, - val appUIPort: Int) + val appUiUrl: String) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index e4ab01dd2a2a2..f195082808aab 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -25,7 +25,7 @@ private[spark] object TestClient { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "localhost", 0) + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener val client = new Client(actorSystem, url, desc, listener) client.start() diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3ee1b60351054..e28b007e30686 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -11,8 +11,7 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUIHost: String, - val appUIPort: Int) + val appUiUrl: String) { var state = ApplicationState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] @@ -63,7 +62,4 @@ private[spark] class ApplicationInfo( } } - - def appUIAddress = "http://" + this.appUIHost + ":" + this.appUIPort - } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 9f2d3da495bd4..6f58ad16af724 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -244,7 +244,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUIHost, desc.appUIPort) + val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl) apps += app idToApp(app.id) = app actorToApp(driver) = app diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 5d7d1feb7456a..955ee5d806c57 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -32,7 +32,7 @@ private[spark] class SparkDeploySchedulerBackend( val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) val appDesc = - new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.host, sc.ui.port) + new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 13158e426245d..e02281344a8f8 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -74,4 +74,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } } } + + private[spark] def appUIAddress = "http://" + host + ":" + port } diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 02086b476fe6c..15eabc9834e1b 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -22,7 +22,7 @@
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • From 0fd84965f66aa37d2ae14da799b86a5c8ed1cb32 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 6 May 2013 15:35:18 -0700 Subject: [PATCH 545/696] Added EmptyRDD. --- core/src/main/scala/spark/rdd/EmptyRDD.scala | 16 ++++++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 14 +++++++++++++- 2 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/spark/rdd/EmptyRDD.scala diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala new file mode 100644 index 0000000000000..e4dd3a7fa7dce --- /dev/null +++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala @@ -0,0 +1,16 @@ +package spark.rdd + +import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} + + +/** + * An RDD that is empty, i.e. has no element in it. + */ +class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) { + + override def getPartitions: Array[Partition] = Array.empty + + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + throw new UnsupportedOperationException("empty RDD") + } +} diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index cee6312572710..2ce757b13cdd4 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -5,7 +5,7 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.time.{Span, Millis} import spark.SparkContext._ -import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD} +import spark.rdd.{CoalescedRDD, CoGroupedRDD, EmptyRDD, PartitionPruningRDD, ShuffledRDD} class RDDSuite extends FunSuite with LocalSparkContext { @@ -147,6 +147,18 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(rdd.collect().toList === List(1, 2, 3, 4)) } + test("empty RDD") { + sc = new SparkContext("local", "test") + val empty = new EmptyRDD[Int](sc) + assert(empty.count === 0) + assert(empty.collect().size === 0) + + val thrown = intercept[UnsupportedOperationException]{ + empty.reduce(_+_) + } + assert(thrown.getMessage.contains("empty")) + } + test("cogrouped RDDs") { sc = new SparkContext("local", "test") val rdd1 = sc.makeRDD(Array((1, "one"), (1, "another one"), (2, "two"), (3, "three")), 2) From 64d4d2b036447f42bfcd3bac5687c79a3b0661ca Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 6 May 2013 16:30:46 -0700 Subject: [PATCH 546/696] Added tests for joins, cogroups, and unions for EmptyRDD. --- core/src/test/scala/spark/RDDSuite.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 2ce757b13cdd4..a761dd77c5118 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -157,6 +157,14 @@ class RDDSuite extends FunSuite with LocalSparkContext { empty.reduce(_+_) } assert(thrown.getMessage.contains("empty")) + + val emptyKv = new EmptyRDD[(Int, Int)](sc) + val rdd = sc.parallelize(1 to 2, 2).map(x => (x, x)) + assert(rdd.join(emptyKv).collect().size === 0) + assert(rdd.rightOuterJoin(emptyKv).collect().size === 0) + assert(rdd.leftOuterJoin(emptyKv).collect().size === 2) + assert(rdd.cogroup(emptyKv).collect().size === 2) + assert(rdd.union(emptyKv).collect().size === 2) } test("cogrouped RDDs") { From 4d8919d33056a006ebf6b1ddb0509aeccaa828d7 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 18 Apr 2013 14:58:38 -0700 Subject: [PATCH 547/696] Update Maven build to Scala 2.9.3 --- core/pom.xml | 4 ++-- pom.xml | 11 +++++------ 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index da26d674ec02e..9a019b5a42e53 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -73,7 +73,7 @@ cc.spray - spray-json_${scala.version} + spray-json_2.9.2 org.tomdz.twirl @@ -81,7 +81,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 org.apache.mesos diff --git a/pom.xml b/pom.xml index c3323ffad0f71..3936165d784ed 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ UTF-8 1.5 - 2.9.2 + 2.9.3 0.9.0-incubating 2.0.3 1.0-M2.1 @@ -238,7 +238,7 @@ cc.spray - spray-json_${scala.version} + spray-json_2.9.2 ${spray.json.version} @@ -248,7 +248,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 0.4.1 @@ -277,7 +277,7 @@ org.scalatest scalatest_${scala.version} - 1.8 + 1.9.1 test @@ -289,7 +289,7 @@ org.scalacheck scalacheck_${scala.version} - 1.9 + 1.10.0 test @@ -513,7 +513,6 @@ hadoop1 - 1 From a3d5f922109caa878f8350fe0634514b8af55cbc Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Tue, 7 May 2013 11:43:06 -0500 Subject: [PATCH 548/696] Switch to using SparkContext method to create RDD --- examples/src/main/scala/spark/examples/HBaseTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index d94b25828d017..9bad876860b43 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -24,9 +24,9 @@ object HBaseTest { admin.createTable(tableDesc) } - val hBaseRDD = new NewHadoopRDD(sc, classOf[TableInputFormat], + val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], - classOf[org.apache.hadoop.hbase.client.Result], conf) + classOf[org.apache.hadoop.hbase.client.Result]) hBaseRDD.count() From aacca1b8a85bd073ce185a06d6470b070761b2f4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 18 Apr 2013 14:58:38 -0700 Subject: [PATCH 549/696] Update Maven build to Scala 2.9.3 --- core/pom.xml | 4 ++-- pom.xml | 11 +++++------ 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index da26d674ec02e..9a019b5a42e53 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -73,7 +73,7 @@ cc.spray - spray-json_${scala.version} + spray-json_2.9.2 org.tomdz.twirl @@ -81,7 +81,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 org.apache.mesos diff --git a/pom.xml b/pom.xml index c3323ffad0f71..3936165d784ed 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ UTF-8 1.5 - 2.9.2 + 2.9.3 0.9.0-incubating 2.0.3 1.0-M2.1 @@ -238,7 +238,7 @@ cc.spray - spray-json_${scala.version} + spray-json_2.9.2 ${spray.json.version} @@ -248,7 +248,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 0.4.1 @@ -277,7 +277,7 @@ org.scalatest scalatest_${scala.version} - 1.8 + 1.9.1 test @@ -289,7 +289,7 @@ org.scalacheck scalacheck_${scala.version} - 1.9 + 1.10.0 test @@ -513,7 +513,6 @@ hadoop1 - 1 From 8b7948517182ced5b3681dfc668732364ebccc38 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 17:02:32 -0700 Subject: [PATCH 550/696] Moved BlockFetcherIterator to its own file. --- .../spark/storage/BlockFetcherIterator.scala | 361 ++++++++++++++++++ .../scala/spark/storage/BlockManager.scala | 340 +---------------- 2 files changed, 362 insertions(+), 339 deletions(-) create mode 100644 core/src/main/scala/spark/storage/BlockFetcherIterator.scala diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala new file mode 100644 index 0000000000000..30990d9a389f3 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -0,0 +1,361 @@ +package spark.storage + +import java.nio.ByteBuffer +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet +import scala.collection.mutable.Queue + +import spark.Logging +import spark.Utils +import spark.SparkException + +import spark.network.BufferMessage +import spark.network.ConnectionManagerId +import spark.network.netty.ShuffleCopier + +import spark.serializer.Serializer +import io.netty.buffer.ByteBuf + + +trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] + with Logging with BlockFetchTracker { + + def initialize() + +} + + + +object BlockFetcherIterator { + + // A request to fetch one or more blocks, complete with their sizes + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + // the block (since we want all deserializaton to happen in the calling thread); can also + // represent a fetch failure if size == -1. + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } + + class BasicBlockFetcherIterator( + private val blockManager: BlockManager, + val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer + ) extends BlockFetcherIterator { + + import blockManager._ + + private var _remoteBytesRead = 0l + private var _remoteFetchTime = 0l + private var _fetchWaitTime = 0l + + if (blocksByAddress == null) { + throw new IllegalArgumentException("BlocksByAddress is null") + } + var totalBlocks = blocksByAddress.map(_._2.size).sum + logDebug("Getting " + totalBlocks + " blocks") + var startTime = System.currentTimeMillis + val localBlockIds = new ArrayBuffer[String]() + val remoteBlockIds = new HashSet[String]() + + // A queue to hold our results. + val results = new LinkedBlockingQueue[FetchResult] + + // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + // the number of bytes in flight is limited to maxBytesInFlight + val fetchRequests = new Queue[FetchRequest] + + // Current bytes in flight from our requests + var bytesInFlight = 0L + + def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort)) + val cmId = new ConnectionManagerId(req.address.host, req.address.port) + val blockMessageArray = new BlockMessageArray(req.blocks.map { + case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) + }) + bytesInFlight += req.size + val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + val fetchStart = System.currentTimeMillis() + val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) + future.onSuccess { + case Some(message) => { + val fetchDone = System.currentTimeMillis() + _remoteFetchTime += fetchDone - fetchStart + val bufferMessage = message.asInstanceOf[BufferMessage] + val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + for (blockMessage <- blockMessageArray) { + if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { + throw new SparkException( + "Unexpected message " + blockMessage.getType + " received from " + cmId) + } + val blockId = blockMessage.getId + results.put(new FetchResult(blockId, sizeMap(blockId), + () => dataDeserialize(blockId, blockMessage.getData, serializer))) + _remoteBytesRead += req.size + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } + } + case None => { + logError("Could not get block(s) from " + cmId) + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + } + } + + def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + curBlocks += ((blockId, size)) + curRequestSize += size + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + remoteRequests + } + + def getLocalBlocks(){ + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + for (id <- localBlockIds) { + getLocal(id) match { + case Some(iter) => { + results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + logDebug("Got local block " + id) + } + case None => { + throw new BlockException(id, "Could not get block " + id + " from local machine") + } + } + } + } + + override def initialize(){ + // Split local and remote blocks. + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + + val numGets = remoteBlockIds.size - fetchRequests.size + logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + + } + + //an iterator that will read fetched blocks off the queue as they arrive. + @volatile protected var resultsGotten = 0 + + def hasNext: Boolean = resultsGotten < totalBlocks + + def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val startFetchWait = System.currentTimeMillis() + val result = results.take() + val stopFetchWait = System.currentTimeMillis() + _fetchWaitTime += (stopFetchWait - startFetchWait) + if (! result.failed) bytesInFlight -= result.size + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + + + //methods to profile the block fetching + def numLocalBlocks = localBlockIds.size + def numRemoteBlocks = remoteBlockIds.size + + def remoteFetchTime = _remoteFetchTime + def fetchWaitTime = _fetchWaitTime + + def remoteBytesRead = _remoteBytesRead + + } + + + class NettyBlockFetcherIterator( + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer + ) extends BasicBlockFetcherIterator(blockManager,blocksByAddress,serializer) { + + import blockManager._ + + val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] + + def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, + results : LinkedBlockingQueue[FetchResult]){ + results.put(new FetchResult( + blockId, blockSize, () => dataDeserialize(blockId, blockData, serializer) )) + } + + def startCopiers (numCopiers: Int): List [ _ <: Thread]= { + (for ( i <- Range(0,numCopiers) ) yield { + val copier = new Thread { + override def run(){ + try { + while(!isInterrupted && !fetchRequestsSync.isEmpty) { + sendRequest(fetchRequestsSync.take()) + } + } catch { + case x: InterruptedException => logInfo("Copier Interrupted") + //case _ => throw new SparkException("Exception Throw in Shuffle Copier") + } + } + } + copier.start + copier + }).toList + } + + //keep this to interrupt the threads when necessary + def stopCopiers(copiers : List[_ <: Thread]) { + for (copier <- copiers) { + copier.interrupt() + } + } + + override def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host)) + val cmId = new ConnectionManagerId(req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cpier = new ShuffleCopier + cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) + logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) + } + + override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val originalTotalBlocks = totalBlocks; + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + if (size > 0) { + curBlocks += ((blockId, size)) + curRequestSize += size + } else if (size == 0){ + //here we changes the totalBlocks + totalBlocks -= 1 + } else { + throw new SparkException("Negative block size "+blockId) + } + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") + remoteRequests + } + + var copiers : List[_ <: Thread] = null + + override def initialize(){ + // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + for (request <- Utils.randomize(remoteRequests)) { + fetchRequestsSync.put(request) + } + + copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) + logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val result = results.take() + // if all the results has been retrieved + // shutdown the copiers + if (resultsGotten == totalBlocks) { + if( copiers != null ) + stopCopiers(copiers) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + } + + def apply(t: String, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer): BlockFetcherIterator = { + val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress, serializer) } + else { new BasicBlockFetcherIterator(blockManager,blocksByAddress, serializer) } + iter.initialize() + iter + } +} + diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 433e939656676..a189c1a025002 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -2,9 +2,8 @@ package spark.storage import java.io.{InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import akka.actor.{ActorSystem, Cancellable, Props} @@ -23,8 +22,6 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer -import spark.network.netty.ShuffleCopier -import io.netty.buffer.ByteBuf private[spark] class BlockManager( @@ -977,338 +974,3 @@ object BlockManager extends Logging { } } - -trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { - def initialize -} - -object BlockFetcherIterator { - - // A request to fetch one or more blocks, complete with their sizes - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - -class BasicBlockFetcherIterator( - private val blockManager: BlockManager, - val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer -) extends BlockFetcherIterator { - - import blockManager._ - - private var _remoteBytesRead = 0l - private var _remoteFetchTime = 0l - private var _fetchWaitTime = 0l - - if (blocksByAddress == null) { - throw new IllegalArgumentException("BlocksByAddress is null") - } - var totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + totalBlocks + " blocks") - var startTime = System.currentTimeMillis - val localBlockIds = new ArrayBuffer[String]() - val remoteBlockIds = new HashSet[String]() - - // A queue to hold our results. - val results = new LinkedBlockingQueue[FetchResult] - - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight - val fetchRequests = new Queue[FetchRequest] - - // Current bytes in flight from our requests - var bytesInFlight = 0L - - def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort)) - val cmId = new ConnectionManagerId(req.address.host, req.address.port) - val blockMessageArray = new BlockMessageArray(req.blocks.map { - case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) - }) - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val fetchStart = System.currentTimeMillis() - val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onSuccess { - case Some(message) => { - val fetchDone = System.currentTimeMillis() - _remoteFetchTime += fetchDone - fetchStart - val bufferMessage = message.asInstanceOf[BufferMessage] - val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - throw new SparkException( - "Unexpected message " + blockMessage.getType + " received from " + cmId) - } - val blockId = blockMessage.getId - results.put(new FetchResult(blockId, sizeMap(blockId), - () => dataDeserialize(blockId, blockMessage.getData, serializer))) - _remoteBytesRead += req.size - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - } - } - case None => { - logError("Could not get block(s) from " + cmId) - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } - } - } - } - - def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - remoteRequests - } - - def getLocalBlocks(){ - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - for (id <- localBlockIds) { - getLocal(id) match { - case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") - } - } - } - } - - def initialize(){ - // Split local and remote blocks. - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - - val numGets = remoteBlockIds.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - - } - - //an iterator that will read fetched blocks off the queue as they arrive. - @volatile private var resultsGotten = 0 - - def hasNext: Boolean = resultsGotten < totalBlocks - - def next(): (String, Option[Iterator[Any]]) = { - resultsGotten += 1 - val startFetchWait = System.currentTimeMillis() - val result = results.take() - val stopFetchWait = System.currentTimeMillis() - _fetchWaitTime += (stopFetchWait - startFetchWait) - if (! result.failed) bytesInFlight -= result.size - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - - - //methods to profile the block fetching - def numLocalBlocks = localBlockIds.size - def numRemoteBlocks = remoteBlockIds.size - - def remoteFetchTime = _remoteFetchTime - def fetchWaitTime = _fetchWaitTime - - def remoteBytesRead = _remoteBytesRead - -} - -class NettyBlockFetcherIterator( - blockManager: BlockManager, - blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer -) extends BasicBlockFetcherIterator(blockManager,blocksByAddress,serializer) { - - import blockManager._ - - val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] - - def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, - results : LinkedBlockingQueue[FetchResult]){ - results.put(new FetchResult( - blockId, blockSize, () => dataDeserialize(blockId, blockData, serializer) )) - } - - def startCopiers (numCopiers: Int): List [ _ <: Thread]= { - (for ( i <- Range(0,numCopiers) ) yield { - val copier = new Thread { - override def run(){ - try { - while(!isInterrupted && !fetchRequestsSync.isEmpty) { - sendRequest(fetchRequestsSync.take()) - } - } catch { - case x: InterruptedException => logInfo("Copier Interrupted") - //case _ => throw new SparkException("Exception Throw in Shuffle Copier") - } - } - } - copier.start - copier - }).toList - } - - //keep this to interrupt the threads when necessary - def stopCopiers(copiers : List[_ <: Thread]) { - for (copier <- copiers) { - copier.interrupt() - } - } - - override def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) - val cmId = new ConnectionManagerId(req.address.ip, System.getProperty("spark.shuffle.sender.port", "6653").toInt) - val cpier = new ShuffleCopier - cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) - logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.ip ) - } - - override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = totalBlocks; - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - if (size > 0) { - curBlocks += ((blockId, size)) - curRequestSize += size - } else if (size == 0){ - //here we changes the totalBlocks - totalBlocks -= 1 - } else { - throw new SparkException("Negative block size "+blockId) - } - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") - remoteRequests - } - - var copiers : List[_ <: Thread] = null - - override def initialize(){ - // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - for (request <- Utils.randomize(remoteRequests)) { - fetchRequestsSync.put(request) - } - - copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } - - override def next(): (String, Option[Iterator[Any]]) = { - resultsGotten += 1 - val result = results.take() - // if all the results has been retrieved - // shutdown the copiers - if (resultsGotten == totalBlocks) { - if( copiers != null ) - stopCopiers(copiers) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - } - - def apply(t: String, - blockManager: BlockManager, - blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer): BlockFetcherIterator = { - val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - else { new BasicBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - iter.initialize - iter - } - -} From 0e5cc30868bcf933f2980c4cfe29abc3d8fe5887 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:18:24 -0700 Subject: [PATCH 551/696] Cleaned up BlockManager and BlockFetcherIterator from Shane's PR. --- .../spark/storage/BlockFetchTracker.scala | 12 +- .../spark/storage/BlockFetcherIterator.scala | 167 +++++++++--------- .../scala/spark/storage/BlockManager.scala | 22 ++- 3 files changed, 102 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala index 993aece1f7f76..0718156b1ba2f 100644 --- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala +++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala @@ -1,10 +1,10 @@ package spark.storage private[spark] trait BlockFetchTracker { - def totalBlocks : Int - def numLocalBlocks: Int - def numRemoteBlocks: Int - def remoteFetchTime : Long - def fetchWaitTime: Long - def remoteBytesRead : Long + def totalBlocks : Int + def numLocalBlocks: Int + def numRemoteBlocks: Int + def remoteFetchTime : Long + def fetchWaitTime: Long + def remoteBytesRead : Long } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 30990d9a389f3..43f835237ca32 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -7,27 +7,36 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue +import io.netty.buffer.ByteBuf + import spark.Logging import spark.Utils import spark.SparkException - import spark.network.BufferMessage import spark.network.ConnectionManagerId import spark.network.netty.ShuffleCopier - import spark.serializer.Serializer -import io.netty.buffer.ByteBuf +/** + * A block fetcher iterator interface. There are two implementations: + * + * BasicBlockFetcherIterator: uses a custom-built NIO communication layer. + * NettyBlockFetcherIterator: uses Netty (OIO) as the communication layer. + * + * Eventually we would like the two to converge and use a single NIO-based communication layer, + * but extensive tests show that under some circumstances (e.g. large shuffles with lots of cores), + * NIO would perform poorly and thus the need for the Netty OIO one. + */ + +private[storage] trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { - def initialize() - } - +private[storage] object BlockFetcherIterator { // A request to fetch one or more blocks, complete with their sizes @@ -45,8 +54,8 @@ object BlockFetcherIterator { class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer - ) extends BlockFetcherIterator { + serializer: Serializer) + extends BlockFetcherIterator { import blockManager._ @@ -57,23 +66,24 @@ object BlockFetcherIterator { if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") } - var totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + totalBlocks + " blocks") - var startTime = System.currentTimeMillis - val localBlockIds = new ArrayBuffer[String]() - val remoteBlockIds = new HashSet[String]() + + protected var _totalBlocks = blocksByAddress.map(_._2.size).sum + logDebug("Getting " + _totalBlocks + " blocks") + protected var startTime = System.currentTimeMillis + protected val localBlockIds = new ArrayBuffer[String]() + protected val remoteBlockIds = new HashSet[String]() // A queue to hold our results. - val results = new LinkedBlockingQueue[FetchResult] + protected val results = new LinkedBlockingQueue[FetchResult] // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight - val fetchRequests = new Queue[FetchRequest] + private val fetchRequests = new Queue[FetchRequest] // Current bytes in flight from our requests - var bytesInFlight = 0L + private var bytesInFlight = 0L - def sendRequest(req: FetchRequest) { + protected def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort)) val cmId = new ConnectionManagerId(req.address.host, req.address.port) @@ -111,7 +121,7 @@ object BlockFetcherIterator { } } - def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { + protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] @@ -148,14 +158,15 @@ object BlockFetcherIterator { remoteRequests } - def getLocalBlocks(){ + protected def getLocalBlocks() { // Get the local blocks while remote blocks are being fetched. Note that it's okay to do // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight for (id <- localBlockIds) { getLocal(id) match { case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + // Pass 0 as size since it's not in flight + results.put(new FetchResult(id, 0, () => iter)) logDebug("Got local block " + id) } case None => { @@ -165,7 +176,7 @@ object BlockFetcherIterator { } } - override def initialize(){ + override def initialize() { // Split local and remote blocks. val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order @@ -184,15 +195,14 @@ object BlockFetcherIterator { startTime = System.currentTimeMillis getLocalBlocks() logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } //an iterator that will read fetched blocks off the queue as they arrive. @volatile protected var resultsGotten = 0 - def hasNext: Boolean = resultsGotten < totalBlocks + override def hasNext: Boolean = resultsGotten < _totalBlocks - def next(): (String, Option[Iterator[Any]]) = { + override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 val startFetchWait = System.currentTimeMillis() val result = results.take() @@ -206,74 +216,73 @@ object BlockFetcherIterator { (result.blockId, if (result.failed) None else Some(result.deserialize())) } - - //methods to profile the block fetching - def numLocalBlocks = localBlockIds.size - def numRemoteBlocks = remoteBlockIds.size - - def remoteFetchTime = _remoteFetchTime - def fetchWaitTime = _fetchWaitTime - - def remoteBytesRead = _remoteBytesRead - + // Implementing BlockFetchTracker trait. + override def totalBlocks: Int = _totalBlocks + override def numLocalBlocks: Int = localBlockIds.size + override def numRemoteBlocks: Int = remoteBlockIds.size + override def remoteFetchTime: Long = _remoteFetchTime + override def fetchWaitTime: Long = _fetchWaitTime + override def remoteBytesRead: Long = _remoteBytesRead } - + // End of BasicBlockFetcherIterator class NettyBlockFetcherIterator( blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer - ) extends BasicBlockFetcherIterator(blockManager,blocksByAddress,serializer) { + serializer: Serializer) + extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer) { import blockManager._ val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] - def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, - results : LinkedBlockingQueue[FetchResult]){ - results.put(new FetchResult( - blockId, blockSize, () => dataDeserialize(blockId, blockData, serializer) )) - } - - def startCopiers (numCopiers: Int): List [ _ <: Thread]= { + private def startCopiers(numCopiers: Int): List[_ <: Thread] = { (for ( i <- Range(0,numCopiers) ) yield { - val copier = new Thread { - override def run(){ - try { - while(!isInterrupted && !fetchRequestsSync.isEmpty) { + val copier = new Thread { + override def run(){ + try { + while(!isInterrupted && !fetchRequestsSync.isEmpty) { sendRequest(fetchRequestsSync.take()) - } - } catch { - case x: InterruptedException => logInfo("Copier Interrupted") - //case _ => throw new SparkException("Exception Throw in Shuffle Copier") } - } + } catch { + case x: InterruptedException => logInfo("Copier Interrupted") + //case _ => throw new SparkException("Exception Throw in Shuffle Copier") + } } - copier.start - copier + } + copier.start + copier }).toList } //keep this to interrupt the threads when necessary - def stopCopiers(copiers : List[_ <: Thread]) { + private def stopCopiers() { for (copier <- copiers) { copier.interrupt() } } - override def sendRequest(req: FetchRequest) { + override protected def sendRequest(req: FetchRequest) { + + def putResult(blockId: String, blockSize: Long, blockData: ByteBuf) { + val fetchResult = new FetchResult(blockId, blockSize, + () => dataDeserialize(blockId, blockData.nioBuffer, serializer)) + results.put(fetchResult) + } + logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host)) - val cmId = new ConnectionManagerId(req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cmId = new ConnectionManagerId( + req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) val cpier = new ShuffleCopier - cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) + cpier.getBlocks(cmId, req.blocks, putResult) logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) } - override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { + override protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = totalBlocks; + val originalTotalBlocks = _totalBlocks; val remoteRequests = new ArrayBuffer[FetchRequest] for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { @@ -293,11 +302,11 @@ object BlockFetcherIterator { if (size > 0) { curBlocks += ((blockId, size)) curRequestSize += size - } else if (size == 0){ + } else if (size == 0) { //here we changes the totalBlocks - totalBlocks -= 1 + _totalBlocks -= 1 } else { - throw new SparkException("Negative block size "+blockId) + throw new BlockException(blockId, "Negative block size " + size) } if (curRequestSize >= minRequestSize) { // Add this FetchRequest @@ -312,13 +321,14 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") + logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + + originalTotalBlocks + " blocks") remoteRequests } - var copiers : List[_ <: Thread] = null + private var copiers: List[_ <: Thread] = null - override def initialize(){ + override def initialize() { // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order @@ -327,7 +337,8 @@ object BlockFetcherIterator { } copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) + logInfo("Started " + fetchRequestsSync.size + " remote gets in " + + Utils.getUsedTimeMs(startTime)) // Get Local Blocks startTime = System.currentTimeMillis @@ -338,24 +349,12 @@ object BlockFetcherIterator { override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 val result = results.take() - // if all the results has been retrieved - // shutdown the copiers - if (resultsGotten == totalBlocks) { - if( copiers != null ) - stopCopiers(copiers) + // if all the results has been retrieved, shutdown the copiers + if (resultsGotten == _totalBlocks && copiers != null) { + stopCopiers() } (result.blockId, if (result.failed) None else Some(result.deserialize())) } } - - def apply(t: String, - blockManager: BlockManager, - blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer): BlockFetcherIterator = { - val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - else { new BasicBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - iter.initialize() - iter - } + // End of NettyBlockFetcherIterator } - diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index a189c1a025002..e0dec3a8bbf9c 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -23,8 +23,7 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer -private[spark] -class BlockManager( +private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, @@ -494,11 +493,16 @@ class BlockManager( def getMultiple( blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], serializer: Serializer) : BlockFetcherIterator = { - if(System.getProperty("spark.shuffle.use.netty", "false").toBoolean){ - return BlockFetcherIterator("netty",this, blocksByAddress, serializer) - } else { - return BlockFetcherIterator("", this, blocksByAddress, serializer) - } + + val iter = + if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) { + new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) + } else { + new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) + } + + iter.initialize() + iter } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -942,8 +946,8 @@ class BlockManager( } } -private[spark] -object BlockManager extends Logging { + +private[spark] object BlockManager extends Logging { val ID_GENERATOR = new IdGenerator From 9e64396ca4c24804d5fd4e96212eed54530ca409 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:30:54 -0700 Subject: [PATCH 552/696] Cleaned up the Java files from Shane's PR. --- .../java/spark/network/netty/FileClient.java | 45 +++++------- .../netty/FileClientChannelInitializer.java | 11 +-- .../network/netty/FileClientHandler.java | 11 +-- .../java/spark/network/netty/FileServer.java | 73 +++++++++---------- .../netty/FileServerChannelInitializer.java | 22 ++---- .../network/netty/FileServerHandler.java | 33 ++++----- .../spark/network/netty/PathResolver.java | 4 +- .../main/scala/spark/storage/DiskStore.scala | 4 +- 8 files changed, 85 insertions(+), 118 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index d0c5081dd2a53..3a62dacbc8e89 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -1,42 +1,40 @@ package spark.network.netty; import io.netty.bootstrap.Bootstrap; -import io.netty.channel.AbstractChannel; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelOption; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; -import java.util.Arrays; -public class FileClient { +class FileClient { private FileClientHandler handler = null; private Channel channel = null; private Bootstrap bootstrap = null; - public FileClient(FileClientHandler handler){ + public FileClient(FileClientHandler handler) { this.handler = handler; } - - public void init(){ - bootstrap = new Bootstrap(); - bootstrap.group(new OioEventLoopGroup()) + + public void init() { + bootstrap = new Bootstrap(); + bootstrap.group(new OioEventLoopGroup()) .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) .handler(new FileClientChannelInitializer(handler)); - } + } public static final class ChannelCloseListener implements ChannelFutureListener { private FileClient fc = null; + public ChannelCloseListener(FileClient fc){ this.fc = fc; } + @Override public void operationComplete(ChannelFuture future) { if (fc.bootstrap!=null){ @@ -46,44 +44,39 @@ public void operationComplete(ChannelFuture future) { } } - public void connect(String host, int port){ + public void connect(String host, int port) { try { - // Start the connection attempt. channel = bootstrap.connect(host, port).sync().channel(); // ChannelFuture cf = channel.closeFuture(); //cf.addListener(new ChannelCloseListener(this)); } catch (InterruptedException e) { close(); - } + } } - - public void waitForClose(){ + + public void waitForClose() { try { channel.closeFuture().sync(); } catch (InterruptedException e){ e.printStackTrace(); } - } + } - public void sendRequest(String file){ + public void sendRequest(String file) { //assert(file == null); //assert(channel == null); - channel.write(file+"\r\n"); + channel.write(file + "\r\n"); } - public void close(){ + public void close() { if(channel != null) { - channel.close(); - channel = null; + channel.close(); + channel = null; } if ( bootstrap!=null) { bootstrap.shutdown(); bootstrap = null; } } - - } - - diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java index 50e5704619a60..af25baf641110 100644 --- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java @@ -3,15 +3,10 @@ import io.netty.buffer.BufType; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.netty.handler.codec.string.StringEncoder; -import io.netty.util.CharsetUtil; -import io.netty.handler.logging.LoggingHandler; -import io.netty.handler.logging.LogLevel; -public class FileClientChannelInitializer extends - ChannelInitializer { +class FileClientChannelInitializer extends ChannelInitializer { private FileClientHandler fhandler; @@ -23,7 +18,7 @@ public FileClientChannelInitializer(FileClientHandler handler) { public void initChannel(SocketChannel channel) { // file no more than 2G channel.pipeline() - .addLast("encoder", new StringEncoder(BufType.BYTE)) - .addLast("handler", fhandler); + .addLast("encoder", new StringEncoder(BufType.BYTE)) + .addLast("handler", fhandler); } } diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java index 911c8b32b550f..2069dee5caa59 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -3,12 +3,9 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundByteHandlerAdapter; -import io.netty.util.CharsetUtil; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.logging.Logger; -public abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { +abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { private FileHeader currentHeader = null; @@ -19,7 +16,7 @@ public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) { // Use direct buffer if possible. return ctx.alloc().ioBuffer(); } - + @Override public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) { // get header @@ -27,8 +24,8 @@ public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) { currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE())); } // get file - if(in.readableBytes() >= currentHeader.fileLen()){ - handle(ctx,in,currentHeader); + if(in.readableBytes() >= currentHeader.fileLen()) { + handle(ctx, in, currentHeader); currentHeader = null; ctx.close(); } diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index 38af3050967eb..647b26bf8a155 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -1,58 +1,51 @@ package spark.network.netty; -import java.io.File; import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; import io.netty.channel.Channel; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioServerSocketChannel; -import io.netty.handler.logging.LogLevel; -import io.netty.handler.logging.LoggingHandler; + /** * Server that accept the path of a file an echo back its content. */ -public class FileServer { +class FileServer { + + private ServerBootstrap bootstrap = null; + private Channel channel = null; + private PathResolver pResolver; - private ServerBootstrap bootstrap = null; - private Channel channel = null; - private PathResolver pResolver; + public FileServer(PathResolver pResolver) { + this.pResolver = pResolver; + } - public FileServer(PathResolver pResolver){ - this.pResolver = pResolver; + public void run(int port) { + // Configure the server. + bootstrap = new ServerBootstrap(); + try { + bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) + .channel(OioServerSocketChannel.class) + .option(ChannelOption.SO_BACKLOG, 100) + .option(ChannelOption.SO_RCVBUF, 1500) + .childHandler(new FileServerChannelInitializer(pResolver)); + // Start the server. + channel = bootstrap.bind(port).sync().channel(); + channel.closeFuture().sync(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } finally{ + bootstrap.shutdown(); } + } - public void run(int port) { - // Configure the server. - bootstrap = new ServerBootstrap(); - try { - bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) - .channel(OioServerSocketChannel.class) - .option(ChannelOption.SO_BACKLOG, 100) - .option(ChannelOption.SO_RCVBUF, 1500) - .childHandler(new FileServerChannelInitializer(pResolver)); - // Start the server. - channel = bootstrap.bind(port).sync().channel(); - channel.closeFuture().sync(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally{ - bootstrap.shutdown(); - } + public void stop() { + if (channel!=null) { + channel.close(); } - - public void stop(){ - if (channel!=null){ - channel.close(); - } - if (bootstrap != null){ - bootstrap.shutdown(); - } + if (bootstrap != null) { + bootstrap.shutdown(); } + } } - - diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java index 9d0618ff1c492..8f1f5c65cd757 100644 --- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java @@ -1,21 +1,15 @@ package spark.network.netty; -import java.io.File; -import io.netty.buffer.BufType; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.string.StringDecoder; -import io.netty.handler.codec.string.StringEncoder; import io.netty.handler.codec.DelimiterBasedFrameDecoder; import io.netty.handler.codec.Delimiters; -import io.netty.util.CharsetUtil; -import io.netty.handler.logging.LoggingHandler; -import io.netty.handler.logging.LogLevel; +import io.netty.handler.codec.string.StringDecoder; + -public class FileServerChannelInitializer extends - ChannelInitializer { +class FileServerChannelInitializer extends ChannelInitializer { - PathResolver pResolver; + PathResolver pResolver; public FileServerChannelInitializer(PathResolver pResolver) { this.pResolver = pResolver; @@ -24,10 +18,8 @@ public FileServerChannelInitializer(PathResolver pResolver) { @Override public void initChannel(SocketChannel channel) { channel.pipeline() - .addLast("framer", new DelimiterBasedFrameDecoder( - 8192, Delimiters.lineDelimiter())) - .addLast("strDecoder", new StringDecoder()) - .addLast("handler", new FileServerHandler(pResolver)); - + .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter())) + .addLast("strDecoder", new StringDecoder()) + .addLast("handler", new FileServerHandler(pResolver)); } } diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java index e1083e87a26de..a78eddb1b58b9 100644 --- a/core/src/main/java/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/spark/network/netty/FileServerHandler.java @@ -1,17 +1,17 @@ package spark.network.netty; +import java.io.File; +import java.io.FileInputStream; + import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundMessageHandlerAdapter; import io.netty.channel.DefaultFileRegion; -import io.netty.handler.stream.ChunkedFile; -import java.io.File; -import java.io.FileInputStream; -public class FileServerHandler extends - ChannelInboundMessageHandlerAdapter { - PathResolver pResolver; - +class FileServerHandler extends ChannelInboundMessageHandlerAdapter { + + PathResolver pResolver; + public FileServerHandler(PathResolver pResolver){ this.pResolver = pResolver; } @@ -21,8 +21,8 @@ public void messageReceived(ChannelHandlerContext ctx, String blockId) { String path = pResolver.getAbsolutePath(blockId); // if getFilePath returns null, close the channel if (path == null) { - //ctx.close(); - return; + //ctx.close(); + return; } File file = new File(path); if (file.exists()) { @@ -33,23 +33,21 @@ public void messageReceived(ChannelHandlerContext ctx, String blockId) { return; } long length = file.length(); - if (length > Integer.MAX_VALUE || length <= 0 ) { + if (length > Integer.MAX_VALUE || length <= 0) { //logger.info("too large file : " + file.getAbsolutePath() + " of size "+ length); ctx.write(new FileHeader(0, blockId).buffer()); ctx.flush(); - return; + return; } int len = new Long(length).intValue(); //logger.info("Sending block "+blockId+" filelen = "+len); //logger.info("header = "+ (new FileHeader(len, blockId)).buffer()); ctx.write((new FileHeader(len, blockId)).buffer()); try { - ctx.sendFile(new DefaultFileRegion(new FileInputStream(file) - .getChannel(), 0, file.length())); + ctx.sendFile(new DefaultFileRegion(new FileInputStream(file) + .getChannel(), 0, file.length())); } catch (Exception e) { - // TODO Auto-generated catch block - //logger.warning("Exception when sending file : " - //+ file.getAbsolutePath()); + //logger.warning("Exception when sending file : " + file.getAbsolutePath()); e.printStackTrace(); } } else { @@ -58,8 +56,7 @@ public void messageReceived(ChannelHandlerContext ctx, String blockId) { } ctx.flush(); } - - + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { cause.printStackTrace(); diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java index 5d5eda006eb19..302411672cf96 100755 --- a/core/src/main/java/spark/network/netty/PathResolver.java +++ b/core/src/main/java/spark/network/netty/PathResolver.java @@ -1,12 +1,12 @@ package spark.network.netty; + public interface PathResolver { /** * Get the absolute path of the file - * + * * @param fileId * @return the absolute path of file */ public String getAbsolutePath(String fileId); - } diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 82bcbd5bc2c68..be33d4260e495 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -288,7 +288,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt val pResolver = new PathResolver { - def getAbsolutePath(blockId:String):String = { + override def getAbsolutePath(blockId: String): String = { if (!blockId.startsWith("shuffle_")) { return null } @@ -298,7 +298,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) shuffleSender = new Thread { override def run() = { val sender = new ShuffleSender(port,pResolver) - logInfo("created ShuffleSender binding to port : "+ port) + logInfo("Created ShuffleSender binding to port : "+ port) sender.start } } From 547dcbe494ce7a888f636cf2596243be37b567b1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:39:33 -0700 Subject: [PATCH 553/696] Cleaned up Scala files in network/netty from Shane's PR. --- .../spark/network/netty/ShuffleCopier.scala | 50 +++++++++---------- .../spark/network/netty/ShuffleSender.scala | 18 ++++--- 2 files changed, 35 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index d8d35bfeecc0e..a91f5a886d732 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -1,23 +1,21 @@ package spark.network.netty +import java.util.concurrent.Executors + import io.netty.buffer.ByteBuf import io.netty.channel.ChannelHandlerContext -import io.netty.channel.ChannelInboundByteHandlerAdapter import io.netty.util.CharsetUtil -import java.util.concurrent.atomic.AtomicInteger -import java.util.logging.Logger import spark.Logging import spark.network.ConnectionManagerId -import java.util.concurrent.Executors + private[spark] class ShuffleCopier extends Logging { - def getBlock(cmId: ConnectionManagerId, - blockId: String, - resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + def getBlock(cmId: ConnectionManagerId, blockId: String, + resultCollectCallback: (String, Long, ByteBuf) => Unit) { - val handler = new ShuffleClientHandler(resultCollectCallback) + val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) val fc = new FileClient(handler) fc.init() fc.connect(cmId.host, cmId.port) @@ -28,29 +26,28 @@ private[spark] class ShuffleCopier extends Logging { def getBlocks(cmId: ConnectionManagerId, blocks: Seq[(String, Long)], - resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + resultCollectCallback: (String, Long, ByteBuf) => Unit) { - blocks.map { - case(blockId,size) => { - getBlock(cmId,blockId,resultCollectCallback) - } + for ((blockId, size) <- blocks) { + getBlock(cmId, blockId, resultCollectCallback) } } } -private[spark] class ShuffleClientHandler(val resultCollectCallBack: (String, Long, ByteBuf) => Unit ) extends FileClientHandler with Logging { - - def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { - logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); - resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) - } -} private[spark] object ShuffleCopier extends Logging { - def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) = { - logInfo("File: " + blockId + " content is : \" " - + content.toString(CharsetUtil.UTF_8) + "\"") + private class ShuffleClientHandler(resultCollectCallBack: (String, Long, ByteBuf) => Unit) + extends FileClientHandler with Logging { + + override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { + logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); + resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) + } + } + + def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) { + logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") } def runGetBlock(host:String, port:Int, file:String){ @@ -71,18 +68,17 @@ private[spark] object ShuffleCopier extends Logging { val host = args(0) val port = args(1).toInt val file = args(2) - val threads = if (args.length>3) args(3).toInt else 10 + val threads = if (args.length > 3) args(3).toInt else 10 val copiers = Executors.newFixedThreadPool(80) - for (i <- Range(0,threads)){ + for (i <- Range(0, threads)) { val runnable = new Runnable() { def run() { - runGetBlock(host,port,file) + runGetBlock(host, port, file) } } copiers.execute(runnable) } copiers.shutdown } - } diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala index c1986812e93e8..dc87fefc56794 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -1,12 +1,13 @@ package spark.network.netty -import spark.Logging import java.io.File +import spark.Logging + -private[spark] class ShuffleSender(val port: Int, val pResolver:PathResolver) extends Logging { +private[spark] class ShuffleSender(val port: Int, val pResolver: PathResolver) extends Logging { val server = new FileServer(pResolver) - + Runtime.getRuntime().addShutdownHook( new Thread() { override def run() { @@ -20,17 +21,22 @@ private[spark] class ShuffleSender(val port: Int, val pResolver:PathResolver) ex } } + private[spark] object ShuffleSender { + def main(args: Array[String]) { if (args.length < 3) { - System.err.println("Usage: ShuffleSender ") + System.err.println( + "Usage: ShuffleSender ") System.exit(1) } + val port = args(0).toInt val subDirsPerLocalDir = args(1).toInt - val localDirs = args.drop(2) map {new File(_)} + val localDirs = args.drop(2).map(new File(_)) + val pResovler = new PathResolver { - def getAbsolutePath(blockId:String):String = { + override def getAbsolutePath(blockId: String): String = { if (!blockId.startsWith("shuffle_")) { throw new Exception("Block " + blockId + " is not a shuffle block") } From 8388e8dd7ab5e55ea67b329d9359ba2147d796b0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:40:35 -0700 Subject: [PATCH 554/696] Minor style fix in DiskStore... --- core/src/main/scala/spark/storage/DiskStore.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index be33d4260e495..933eeaa2162c6 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -283,7 +283,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) }) } - private def startShuffleBlockSender (){ + private def startShuffleBlockSender() { try { val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt @@ -297,7 +297,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } shuffleSender = new Thread { override def run() = { - val sender = new ShuffleSender(port,pResolver) + val sender = new ShuffleSender(port, pResolver) logInfo("Created ShuffleSender binding to port : "+ port) sender.start } From 5d70ee4663b5589611aef107f914a94f301c7d2a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 22:42:15 -0700 Subject: [PATCH 555/696] Cleaned up connection manager (moved many classes to their own files). --- .../scala/spark/network/BufferMessage.scala | 94 +++++++++ .../main/scala/spark/network/Connection.scala | 137 ++++++++------ .../spark/network/ConnectionManager.scala | 53 ++---- .../spark/network/ConnectionManagerId.scala | 21 ++ .../main/scala/spark/network/Message.scala | 179 ++---------------- .../scala/spark/network/MessageChunk.scala | 25 +++ .../spark/network/MessageChunkHeader.scala | 58 ++++++ 7 files changed, 315 insertions(+), 252 deletions(-) create mode 100644 core/src/main/scala/spark/network/BufferMessage.scala create mode 100644 core/src/main/scala/spark/network/ConnectionManagerId.scala create mode 100644 core/src/main/scala/spark/network/MessageChunk.scala create mode 100644 core/src/main/scala/spark/network/MessageChunkHeader.scala diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/spark/network/BufferMessage.scala new file mode 100644 index 0000000000000..7b0e489a6c0cc --- /dev/null +++ b/core/src/main/scala/spark/network/BufferMessage.scala @@ -0,0 +1,94 @@ +package spark.network + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import spark.storage.BlockManager + + +private[spark] +class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) + extends Message(Message.BUFFER_MESSAGE, id_) { + + val initialSize = currentSize() + var gotChunkForSendingOnce = false + + def size = initialSize + + def currentSize() = { + if (buffers == null || buffers.isEmpty) { + 0 + } else { + buffers.map(_.remaining).reduceLeft(_ + _) + } + } + + def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] = { + if (maxChunkSize <= 0) { + throw new Exception("Max chunk size is " + maxChunkSize) + } + + if (size == 0 && gotChunkForSendingOnce == false) { + val newChunk = new MessageChunk( + new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null) + gotChunkForSendingOnce = true + return Some(newChunk) + } + + while(!buffers.isEmpty) { + val buffer = buffers(0) + if (buffer.remaining == 0) { + BlockManager.dispose(buffer) + buffers -= buffer + } else { + val newBuffer = if (buffer.remaining <= maxChunkSize) { + buffer.duplicate() + } else { + buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer] + } + buffer.position(buffer.position + newBuffer.remaining) + val newChunk = new MessageChunk(new MessageChunkHeader( + typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) + gotChunkForSendingOnce = true + return Some(newChunk) + } + } + None + } + + def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] = { + // STRONG ASSUMPTION: BufferMessage created when receiving data has ONLY ONE data buffer + if (buffers.size > 1) { + throw new Exception("Attempting to get chunk from message with multiple data buffers") + } + val buffer = buffers(0) + if (buffer.remaining > 0) { + if (buffer.remaining < chunkSize) { + throw new Exception("Not enough space in data buffer for receiving chunk") + } + val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer] + buffer.position(buffer.position + newBuffer.remaining) + val newChunk = new MessageChunk(new MessageChunkHeader( + typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) + return Some(newChunk) + } + None + } + + def flip() { + buffers.foreach(_.flip) + } + + def hasAckId() = (ackId != 0) + + def isCompletelyReceived() = !buffers(0).hasRemaining + + override def toString = { + if (hasAckId) { + "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")" + } else { + "BufferMessage(id = " + id + ", size = " + size + ")" + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index 00a0433a441dc..6e28f677a31af 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -13,12 +13,13 @@ import java.net._ private[spark] abstract class Connection(val channel: SocketChannel, val selector: Selector, - val socketRemoteConnectionManagerId: ConnectionManagerId) extends Logging { + val socketRemoteConnectionManagerId: ConnectionManagerId) + extends Logging { + def this(channel_ : SocketChannel, selector_ : Selector) = { this(channel_, selector_, - ConnectionManagerId.fromSocketAddress( - channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] - )) + ConnectionManagerId.fromSocketAddress( + channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress])) } channel.configureBlocking(false) @@ -32,17 +33,19 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, var onKeyInterestChangeCallback: (Connection, Int) => Unit = null val remoteAddress = getRemoteAddress() - + // Read channels typically do not register for write and write does not for read // Now, we do have write registering for read too (temporarily), but this is to detect // channel close NOT to actually read/consume data on it ! // How does this work if/when we move to SSL ? - + // What is the interest to register with selector for when we want this connection to be selected def registerInterest() - // What is the interest to register with selector for when we want this connection to be de-selected - // Traditionally, 0 - but in our case, for example, for close-detection on SendingConnection hack, it will be - // SelectionKey.OP_READ (until we fix it properly) + + // What is the interest to register with selector for when we want this connection to + // be de-selected + // Traditionally, 0 - but in our case, for example, for close-detection on SendingConnection hack, + // it will be SelectionKey.OP_READ (until we fix it properly) def unregisterInterest() // On receiving a read event, should we change the interest for this channel or not ? @@ -64,12 +67,14 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, // Returns whether we have to register for further reads or not. def read(): Boolean = { - throw new UnsupportedOperationException("Cannot read on connection of type " + this.getClass.toString) + throw new UnsupportedOperationException( + "Cannot read on connection of type " + this.getClass.toString) } // Returns whether we have to register for further writes or not. def write(): Boolean = { - throw new UnsupportedOperationException("Cannot write on connection of type " + this.getClass.toString) + throw new UnsupportedOperationException( + "Cannot write on connection of type " + this.getClass.toString) } def close() { @@ -81,11 +86,17 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, callOnCloseCallback() } - def onClose(callback: Connection => Unit) {onCloseCallback = callback} + def onClose(callback: Connection => Unit) { + onCloseCallback = callback + } - def onException(callback: (Connection, Exception) => Unit) {onExceptionCallback = callback} + def onException(callback: (Connection, Exception) => Unit) { + onExceptionCallback = callback + } - def onKeyInterestChange(callback: (Connection, Int) => Unit) {onKeyInterestChangeCallback = callback} + def onKeyInterestChange(callback: (Connection, Int) => Unit) { + onKeyInterestChangeCallback = callback + } def callOnExceptionCallback(e: Exception) { if (onExceptionCallback != null) { @@ -95,7 +106,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, " and OnExceptionCallback not registered", e) } } - + def callOnCloseCallback() { if (onCloseCallback != null) { onCloseCallback(this) @@ -132,24 +143,25 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, print(" (" + position + ", " + length + ")") buffer.position(curPosition) } - } -private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, - remoteId_ : ConnectionManagerId) -extends Connection(SocketChannel.open, selector_, remoteId_) { +private[spark] +class SendingConnection(val address: InetSocketAddress, selector_ : Selector, + remoteId_ : ConnectionManagerId) + extends Connection(SocketChannel.open, selector_, remoteId_) { class Outbox(fair: Int = 0) { val messages = new Queue[Message]() - val defaultChunkSize = 65536 //32768 //16384 + val defaultChunkSize = 65536 //32768 //16384 var nextMessageToBeUsed = 0 def addMessage(message: Message) { - messages.synchronized{ + messages.synchronized{ /*messages += message*/ messages.enqueue(message) - logDebug("Added [" + message + "] to outbox for sending to [" + getRemoteConnectionManagerId() + "]") + logDebug("Added [" + message + "] to outbox for sending to " + + "[" + getRemoteConnectionManagerId() + "]") } } @@ -174,7 +186,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { message.started = true message.startTime = System.currentTimeMillis } - return chunk + return chunk } else { /*logInfo("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "]")*/ message.finishTime = System.currentTimeMillis @@ -185,7 +197,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } None } - + private def getChunkRR(): Option[MessageChunk] = { messages.synchronized { while (!messages.isEmpty) { @@ -197,12 +209,14 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { messages.enqueue(message) nextMessageToBeUsed = nextMessageToBeUsed + 1 if (!message.started) { - logDebug("Starting to send [" + message + "] to [" + getRemoteConnectionManagerId() + "]") + logDebug( + "Starting to send [" + message + "] to [" + getRemoteConnectionManagerId() + "]") message.started = true message.startTime = System.currentTimeMillis } - logTrace("Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]") - return chunk + logTrace( + "Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]") + return chunk } else { message.finishTime = System.currentTimeMillis logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + @@ -213,7 +227,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { None } } - + private val outbox = new Outbox(1) val currentBuffers = new ArrayBuffer[ByteBuffer]() @@ -228,11 +242,11 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { // it does - so let us keep it for now. changeConnectionKeyInterest(SelectionKey.OP_WRITE | DEFAULT_INTEREST) } - + override def unregisterInterest() { changeConnectionKeyInterest(DEFAULT_INTEREST) } - + def send(message: Message) { outbox.synchronized { outbox.addMessage(message) @@ -262,12 +276,14 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { // selection - though need not necessarily always complete successfully. val connected = channel.finishConnect if (!force && !connected) { - logInfo("finish connect failed [" + address + "], " + outbox.messages.size + " messages pending") + logInfo( + "finish connect failed [" + address + "], " + outbox.messages.size + " messages pending") return false } // Fallback to previous behavior - assume finishConnect completed - // This will happen only when finishConnect failed for some repeated number of times (10 or so) + // This will happen only when finishConnect failed for some repeated number of times + // (10 or so) // Is highly unlikely unless there was an unclean close of socket, etc registerInterest() logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending") @@ -283,13 +299,13 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } override def write(): Boolean = { - try{ - while(true) { + try { + while (true) { if (currentBuffers.size == 0) { outbox.synchronized { outbox.getChunk() match { case Some(chunk) => { - currentBuffers ++= chunk.buffers + currentBuffers ++= chunk.buffers } case None => { // changeConnectionKeyInterest(0) @@ -299,7 +315,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } } } - + if (currentBuffers.size > 0) { val buffer = currentBuffers(0) val remainingBytes = buffer.remaining @@ -314,7 +330,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } } } catch { - case e: Exception => { + case e: Exception => { logWarning("Error writing in connection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() @@ -336,7 +352,8 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { if (length == -1) { // EOF close() } else if (length > 0) { - logWarning("Unexpected data read from SendingConnection to " + getRemoteConnectionManagerId()) + logWarning( + "Unexpected data read from SendingConnection to " + getRemoteConnectionManagerId()) } } catch { case e: Exception => @@ -355,30 +372,32 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { // Must be created within selector loop - else deadlock -private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) -extends Connection(channel_, selector_) { - +private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) + extends Connection(channel_, selector_) { + class Inbox() { val messages = new HashMap[Int, BufferMessage]() - + def getChunk(header: MessageChunkHeader): Option[MessageChunk] = { - + def createNewMessage: BufferMessage = { val newMessage = Message.create(header).asInstanceOf[BufferMessage] newMessage.started = true newMessage.startTime = System.currentTimeMillis - logDebug("Starting to receive [" + newMessage + "] from [" + getRemoteConnectionManagerId() + "]") + logDebug( + "Starting to receive [" + newMessage + "] from [" + getRemoteConnectionManagerId() + "]") messages += ((newMessage.id, newMessage)) newMessage } - + val message = messages.getOrElseUpdate(header.id, createNewMessage) - logTrace("Receiving chunk of [" + message + "] from [" + getRemoteConnectionManagerId() + "]") + logTrace( + "Receiving chunk of [" + message + "] from [" + getRemoteConnectionManagerId() + "]") message.getChunkForReceiving(header.chunkSize) } - + def getMessageForChunk(chunk: MessageChunk): Option[BufferMessage] = { - messages.get(chunk.header.id) + messages.get(chunk.header.id) } def removeMessage(message: Message) { @@ -387,12 +406,14 @@ extends Connection(channel_, selector_) { } @volatile private var inferredRemoteManagerId: ConnectionManagerId = null + override def getRemoteConnectionManagerId(): ConnectionManagerId = { val currId = inferredRemoteManagerId if (currId != null) currId else super.getRemoteConnectionManagerId() } - // The reciever's remote address is the local socket on remote side : which is NOT the connection manager id of the receiver. + // The reciever's remote address is the local socket on remote side : which is NOT + // the connection manager id of the receiver. // We infer that from the messages we receive on the receiver socket. private def processConnectionManagerId(header: MessageChunkHeader) { val currId = inferredRemoteManagerId @@ -428,7 +449,8 @@ extends Connection(channel_, selector_) { } headerBuffer.flip if (headerBuffer.remaining != MessageChunkHeader.HEADER_SIZE) { - throw new Exception("Unexpected number of bytes (" + headerBuffer.remaining + ") in the header") + throw new Exception( + "Unexpected number of bytes (" + headerBuffer.remaining + ") in the header") } val header = MessageChunkHeader.create(headerBuffer) headerBuffer.clear() @@ -451,9 +473,9 @@ extends Connection(channel_, selector_) { case _ => throw new Exception("Message of unknown type received") } } - + if (currentChunk == null) throw new Exception("No message chunk to receive data") - + val bytesRead = channel.read(currentChunk.buffer) if (bytesRead == 0) { // re-register for read event ... @@ -464,14 +486,15 @@ extends Connection(channel_, selector_) { } /*logDebug("Read " + bytesRead + " bytes for the buffer")*/ - + if (currentChunk.buffer.remaining == 0) { /*println("Filled buffer at " + System.currentTimeMillis)*/ val bufferMessage = inbox.getMessageForChunk(currentChunk).get if (bufferMessage.isCompletelyReceived) { bufferMessage.flip bufferMessage.finishTime = System.currentTimeMillis - logDebug("Finished receiving [" + bufferMessage + "] from [" + getRemoteConnectionManagerId() + "] in " + bufferMessage.timeTaken) + logDebug("Finished receiving [" + bufferMessage + "] from " + + "[" + getRemoteConnectionManagerId() + "] in " + bufferMessage.timeTaken) if (onReceiveCallback != null) { onReceiveCallback(this, bufferMessage) } @@ -481,7 +504,7 @@ extends Connection(channel_, selector_) { } } } catch { - case e: Exception => { + case e: Exception => { logWarning("Error reading from connection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() @@ -491,7 +514,7 @@ extends Connection(channel_, selector_) { // should not happen - to keep scala compiler happy return true } - + def onReceive(callback: (Connection, Message) => Unit) {onReceiveCallback = callback} override def changeInterestForRead(): Boolean = true @@ -505,7 +528,7 @@ extends Connection(channel_, selector_) { // it does - so let us keep it for now. changeConnectionKeyInterest(SelectionKey.OP_READ) } - + override def unregisterInterest() { changeConnectionKeyInterest(0) } diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 0eb03630d00fe..624a094856778 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -18,20 +18,7 @@ import akka.dispatch.{Await, Promise, ExecutionContext, Future} import akka.util.Duration import akka.util.duration._ -private[spark] case class ConnectionManagerId(host: String, port: Int) { - // DEBUG code - Utils.checkHost(host) - assert (port > 0) - def toSocketAddress() = new InetSocketAddress(host, port) -} - -private[spark] object ConnectionManagerId { - def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { - new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort()) - } -} - private[spark] class ConnectionManager(port: Int) extends Logging { class MessageStatus( @@ -45,7 +32,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { def markDone() { completionHandler(this) } } - + private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( @@ -80,7 +67,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { serverChannel.configureBlocking(false) serverChannel.socket.setReuseAddress(true) - serverChannel.socket.setReceiveBufferSize(256 * 1024) + serverChannel.socket.setReceiveBufferSize(256 * 1024) serverChannel.socket.bind(new InetSocketAddress(port)) serverChannel.register(selector, SelectionKey.OP_ACCEPT) @@ -351,7 +338,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { case e: Exception => logError("Error in select loop", e) } } - + def acceptConnection(key: SelectionKey) { val serverChannel = key.channel.asInstanceOf[ServerSocketChannel] @@ -463,7 +450,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { def receiveMessage(connection: Connection, message: Message) { val connectionManagerId = ConnectionManagerId.fromSocketAddress(message.senderAddress) - logDebug("Received [" + message + "] from [" + connectionManagerId + "]") + logDebug("Received [" + message + "] from [" + connectionManagerId + "]") val runnable = new Runnable() { val creationTime = System.currentTimeMillis def run() { @@ -483,11 +470,11 @@ private[spark] class ConnectionManager(port: Int) extends Logging { if (bufferMessage.hasAckId) { val sentMessageStatus = messageStatuses.synchronized { messageStatuses.get(bufferMessage.ackId) match { - case Some(status) => { - messageStatuses -= bufferMessage.ackId + case Some(status) => { + messageStatuses -= bufferMessage.ackId status } - case None => { + case None => { throw new Exception("Could not find reference for received ack message " + message.id) null } @@ -507,7 +494,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logDebug("Not calling back as callback is null") None } - + if (ackMessage.isDefined) { if (!ackMessage.get.isInstanceOf[BufferMessage]) { logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + ackMessage.get.getClass()) @@ -517,7 +504,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } } - sendMessage(connectionManagerId, ackMessage.getOrElse { + sendMessage(connectionManagerId, ackMessage.getOrElse { Message.createBufferMessage(bufferMessage.id) }) } @@ -588,17 +575,17 @@ private[spark] object ConnectionManager { def main(args: Array[String]) { val manager = new ConnectionManager(9999) - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None }) - + /*testSequentialSending(manager)*/ /*System.gc()*/ /*testParallelSending(manager)*/ /*System.gc()*/ - + /*testParallelDecreasingSending(manager)*/ /*System.gc()*/ @@ -610,9 +597,9 @@ private[spark] object ConnectionManager { println("--------------------------") println("Sequential Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 - + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip @@ -628,7 +615,7 @@ private[spark] object ConnectionManager { println("--------------------------") println("Parallel Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) @@ -643,12 +630,12 @@ private[spark] object ConnectionManager { if (!g.isDefined) println("Failed") }) val finishTime = System.currentTimeMillis - + val mb = size * count / 1024.0 / 1024.0 val ms = finishTime - startTime val tput = mb * 1000.0 / ms println("--------------------------") - println("Started at " + startTime + ", finished at " + finishTime) + println("Started at " + startTime + ", finished at " + finishTime) println("Sent " + count + " messages of size " + size + " in " + ms + " ms (" + tput + " MB/s)") println("--------------------------") println() @@ -658,7 +645,7 @@ private[spark] object ConnectionManager { println("--------------------------") println("Parallel Decreasing Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 val buffers = Array.tabulate(count)(i => ByteBuffer.allocate(size * (i + 1)).put(Array.tabulate[Byte](size * (i + 1))(x => x.toByte))) buffers.foreach(_.flip) @@ -673,7 +660,7 @@ private[spark] object ConnectionManager { if (!g.isDefined) println("Failed") }) val finishTime = System.currentTimeMillis - + val ms = finishTime - startTime val tput = mb * 1000.0 / ms println("--------------------------") @@ -687,7 +674,7 @@ private[spark] object ConnectionManager { println("--------------------------") println("Continuous Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/spark/network/ConnectionManagerId.scala new file mode 100644 index 0000000000000..b554e84251c14 --- /dev/null +++ b/core/src/main/scala/spark/network/ConnectionManagerId.scala @@ -0,0 +1,21 @@ +package spark.network + +import java.net.InetSocketAddress + +import spark.Utils + + +private[spark] case class ConnectionManagerId(host: String, port: Int) { + // DEBUG code + Utils.checkHost(host) + assert (port > 0) + + def toSocketAddress() = new InetSocketAddress(host, port) +} + + +private[spark] object ConnectionManagerId { + def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { + new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort()) + } +} diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala index 34fac9e77699b..d4f03610ebfdf 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/spark/network/Message.scala @@ -1,56 +1,10 @@ package spark.network -import spark._ - -import scala.collection.mutable.ArrayBuffer - import java.nio.ByteBuffer -import java.net.InetAddress import java.net.InetSocketAddress -import storage.BlockManager - -private[spark] class MessageChunkHeader( - val typ: Long, - val id: Int, - val totalSize: Int, - val chunkSize: Int, - val other: Int, - val address: InetSocketAddress) { - lazy val buffer = { - // No need to change this, at 'use' time, we do a reverse lookup of the hostname. Refer to network.Connection - val ip = address.getAddress.getAddress() - val port = address.getPort() - ByteBuffer. - allocate(MessageChunkHeader.HEADER_SIZE). - putLong(typ). - putInt(id). - putInt(totalSize). - putInt(chunkSize). - putInt(other). - putInt(ip.size). - put(ip). - putInt(port). - position(MessageChunkHeader.HEADER_SIZE). - flip.asInstanceOf[ByteBuffer] - } - - override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + - " and sizes " + totalSize + " / " + chunkSize + " bytes" -} -private[spark] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { - val size = if (buffer == null) 0 else buffer.remaining - lazy val buffers = { - val ab = new ArrayBuffer[ByteBuffer]() - ab += header.buffer - if (buffer != null) { - ab += buffer - } - ab - } +import scala.collection.mutable.ArrayBuffer - override def toString = "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")" -} private[spark] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null @@ -59,120 +13,16 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { var finishTime = -1L def size: Int - + def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] - + def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] - + def timeTaken(): String = (finishTime - startTime).toString + " ms" override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" } -private[spark] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) -extends Message(Message.BUFFER_MESSAGE, id_) { - - val initialSize = currentSize() - var gotChunkForSendingOnce = false - - def size = initialSize - - def currentSize() = { - if (buffers == null || buffers.isEmpty) { - 0 - } else { - buffers.map(_.remaining).reduceLeft(_ + _) - } - } - - def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] = { - if (maxChunkSize <= 0) { - throw new Exception("Max chunk size is " + maxChunkSize) - } - - if (size == 0 && gotChunkForSendingOnce == false) { - val newChunk = new MessageChunk(new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null) - gotChunkForSendingOnce = true - return Some(newChunk) - } - - while(!buffers.isEmpty) { - val buffer = buffers(0) - if (buffer.remaining == 0) { - BlockManager.dispose(buffer) - buffers -= buffer - } else { - val newBuffer = if (buffer.remaining <= maxChunkSize) { - buffer.duplicate() - } else { - buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer] - } - buffer.position(buffer.position + newBuffer.remaining) - val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) - gotChunkForSendingOnce = true - return Some(newChunk) - } - } - None - } - - def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] = { - // STRONG ASSUMPTION: BufferMessage created when receiving data has ONLY ONE data buffer - if (buffers.size > 1) { - throw new Exception("Attempting to get chunk from message with multiple data buffers") - } - val buffer = buffers(0) - if (buffer.remaining > 0) { - if (buffer.remaining < chunkSize) { - throw new Exception("Not enough space in data buffer for receiving chunk") - } - val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer] - buffer.position(buffer.position + newBuffer.remaining) - val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) - return Some(newChunk) - } - None - } - - def flip() { - buffers.foreach(_.flip) - } - - def hasAckId() = (ackId != 0) - - def isCompletelyReceived() = !buffers(0).hasRemaining - - override def toString = { - if (hasAckId) { - "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")" - } else { - "BufferMessage(id = " + id + ", size = " + size + ")" - } - } -} - -private[spark] object MessageChunkHeader { - val HEADER_SIZE = 40 - - def create(buffer: ByteBuffer): MessageChunkHeader = { - if (buffer.remaining != HEADER_SIZE) { - throw new IllegalArgumentException("Cannot convert buffer data to Message") - } - val typ = buffer.getLong() - val id = buffer.getInt() - val totalSize = buffer.getInt() - val chunkSize = buffer.getInt() - val other = buffer.getInt() - val ipSize = buffer.getInt() - val ipBytes = new Array[Byte](ipSize) - buffer.get(ipBytes) - val ip = InetAddress.getByAddress(ipBytes) - val port = buffer.getInt() - new MessageChunkHeader(typ, id, totalSize, chunkSize, other, new InetSocketAddress(ip, port)) - } -} private[spark] object Message { val BUFFER_MESSAGE = 1111111111L @@ -181,14 +31,16 @@ private[spark] object Message { def getNewId() = synchronized { lastId += 1 - if (lastId == 0) lastId += 1 + if (lastId == 0) { + lastId += 1 + } lastId } def createBufferMessage(dataBuffers: Seq[ByteBuffer], ackId: Int): BufferMessage = { if (dataBuffers == null) { return new BufferMessage(getNewId(), new ArrayBuffer[ByteBuffer], ackId) - } + } if (dataBuffers.exists(_ == null)) { throw new Exception("Attempting to create buffer message with null buffer") } @@ -197,7 +49,7 @@ private[spark] object Message { def createBufferMessage(dataBuffers: Seq[ByteBuffer]): BufferMessage = createBufferMessage(dataBuffers, 0) - + def createBufferMessage(dataBuffer: ByteBuffer, ackId: Int): BufferMessage = { if (dataBuffer == null) { return createBufferMessage(Array(ByteBuffer.allocate(0)), ackId) @@ -205,15 +57,18 @@ private[spark] object Message { return createBufferMessage(Array(dataBuffer), ackId) } } - - def createBufferMessage(dataBuffer: ByteBuffer): BufferMessage = + + def createBufferMessage(dataBuffer: ByteBuffer): BufferMessage = createBufferMessage(dataBuffer, 0) - - def createBufferMessage(ackId: Int): BufferMessage = createBufferMessage(new Array[ByteBuffer](0), ackId) + + def createBufferMessage(ackId: Int): BufferMessage = { + createBufferMessage(new Array[ByteBuffer](0), ackId) + } def create(header: MessageChunkHeader): Message = { val newMessage: Message = header.typ match { - case BUFFER_MESSAGE => new BufferMessage(header.id, ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other) + case BUFFER_MESSAGE => new BufferMessage(header.id, + ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other) } newMessage.senderAddress = header.address newMessage diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/spark/network/MessageChunk.scala new file mode 100644 index 0000000000000..aaf9204d0e61a --- /dev/null +++ b/core/src/main/scala/spark/network/MessageChunk.scala @@ -0,0 +1,25 @@ +package spark.network + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + + +private[network] +class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { + + val size = if (buffer == null) 0 else buffer.remaining + + lazy val buffers = { + val ab = new ArrayBuffer[ByteBuffer]() + ab += header.buffer + if (buffer != null) { + ab += buffer + } + ab + } + + override def toString = { + "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")" + } +} diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/spark/network/MessageChunkHeader.scala new file mode 100644 index 0000000000000..3693d509d6859 --- /dev/null +++ b/core/src/main/scala/spark/network/MessageChunkHeader.scala @@ -0,0 +1,58 @@ +package spark.network + +import java.net.InetAddress +import java.net.InetSocketAddress +import java.nio.ByteBuffer + + +private[spark] class MessageChunkHeader( + val typ: Long, + val id: Int, + val totalSize: Int, + val chunkSize: Int, + val other: Int, + val address: InetSocketAddress) { + lazy val buffer = { + // No need to change this, at 'use' time, we do a reverse lookup of the hostname. + // Refer to network.Connection + val ip = address.getAddress.getAddress() + val port = address.getPort() + ByteBuffer. + allocate(MessageChunkHeader.HEADER_SIZE). + putLong(typ). + putInt(id). + putInt(totalSize). + putInt(chunkSize). + putInt(other). + putInt(ip.size). + put(ip). + putInt(port). + position(MessageChunkHeader.HEADER_SIZE). + flip.asInstanceOf[ByteBuffer] + } + + override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + + " and sizes " + totalSize + " / " + chunkSize + " bytes" +} + + +private[spark] object MessageChunkHeader { + val HEADER_SIZE = 40 + + def create(buffer: ByteBuffer): MessageChunkHeader = { + if (buffer.remaining != HEADER_SIZE) { + throw new IllegalArgumentException("Cannot convert buffer data to Message") + } + val typ = buffer.getLong() + val id = buffer.getInt() + val totalSize = buffer.getInt() + val chunkSize = buffer.getInt() + val other = buffer.getInt() + val ipSize = buffer.getInt() + val ipBytes = new Array[Byte](ipSize) + buffer.get(ipBytes) + val ip = InetAddress.getByAddress(ipBytes) + val port = buffer.getInt() + new MessageChunkHeader(typ, id, totalSize, chunkSize, other, new InetSocketAddress(ip, port)) + } +} From 9cafacf32ddb9a3f6c5cb774e4fe527225273f16 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 22:42:37 -0700 Subject: [PATCH 556/696] Added test for Netty suite. --- .../src/test/scala/spark/DistributedSuite.scala | 3 ++- .../test/scala/spark/ShuffleNettySuite.scala | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 core/src/test/scala/spark/ShuffleNettySuite.scala diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4df3bb5b6793b..488c70c414881 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -18,7 +18,8 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ import storage.{GetBlock, BlockManagerWorker, StorageLevel} -class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { +class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter + with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/spark/ShuffleNettySuite.scala new file mode 100644 index 0000000000000..bfaffa953e328 --- /dev/null +++ b/core/src/test/scala/spark/ShuffleNettySuite.scala @@ -0,0 +1,17 @@ +package spark + +import org.scalatest.BeforeAndAfterAll + + +class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { + + // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. + + override def beforeAll(configMap: Map[String, Any]) { + System.setProperty("spark.shuffle.use.netty", "true") + } + + override def afterAll(configMap: Map[String, Any]) { + System.setProperty("spark.shuffle.use.netty", "false") + } +} From 0ab818d50812f312596170b5e42aa76d2ff59d15 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 9 May 2013 00:38:59 -0700 Subject: [PATCH 557/696] fix linebreak --- .../spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 955ee5d806c57..170ede0f44226 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -31,8 +31,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) - val appDesc = - new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, + sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() From b05c9d22d70333924b988b2dfa359ce3e11f7c9d Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 9 May 2013 18:49:12 +0530 Subject: [PATCH 558/696] Remove explicit hardcoding of yarn-standalone as args(0) if it is missing. --- .../spark/deploy/yarn/ApplicationMaster.scala | 19 +++---------------- .../yarn/ApplicationMasterArguments.scala | 1 - .../spark/deploy/yarn/ClientArguments.scala | 1 - 3 files changed, 3 insertions(+), 18 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index ae719267e8062..aa72c1e5fef1c 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -148,22 +148,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { - var mainArgs: Array[String] = null - var startIndex = 0 - - // I am sure there is a better 'scala' way to do this .... but I am just trying to get things to work right now ! - if (args.userArgs.isEmpty || args.userArgs.get(0) != "yarn-standalone") { - // ensure that first param is ALWAYS "yarn-standalone" - mainArgs = new Array[String](args.userArgs.size() + 1) - mainArgs.update(0, "yarn-standalone") - startIndex = 1 - } - else { - mainArgs = new Array[String](args.userArgs.size()) - } - - args.userArgs.copyToArray(mainArgs, startIndex, args.userArgs.size()) - + // Copy + var mainArgs: Array[String] = new Array[String](args.userArgs.size()) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) } } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala index dc89125d8184a..1b00208511b16 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -69,7 +69,6 @@ class ApplicationMasterArguments(val args: Array[String]) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " Note that first argument will ALWAYS be yarn-standalone : will be added if missing.\n" + " --num-workers NUM Number of workers to start (Default: 2)\n" + " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 2e69fe3fb0566..24110558e7da8 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -92,7 +92,6 @@ class ClientArguments(val args: Array[String]) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " Note that first argument will ALWAYS be yarn-standalone : will be added if missing.\n" + " --num-workers NUM Number of workers to start (Default: 2)\n" + " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + From 012c9e5ab072239e07202abe4775b434be6e32b9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 9 May 2013 14:20:01 -0700 Subject: [PATCH 559/696] Revert "Merge pull request #596 from esjewett/master" because the dependency on hbase introduces netty-3.2.2 which conflicts with netty-3.5.3 already in Spark. This caused multiple test failures. This reverts commit 0f1b7a06e1f6782711170234f105f1b277e3b04c, reversing changes made to aacca1b8a85bd073ce185a06d6470b070761b2f4. --- .../main/scala/spark/examples/HBaseTest.scala | 35 ------------------- project/SparkBuild.scala | 6 +--- 2 files changed, 1 insertion(+), 40 deletions(-) delete mode 100644 examples/src/main/scala/spark/examples/HBaseTest.scala diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala deleted file mode 100644 index 9bad876860b43..0000000000000 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ /dev/null @@ -1,35 +0,0 @@ -package spark.examples - -import spark._ -import spark.rdd.NewHadoopRDD -import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} -import org.apache.hadoop.hbase.client.HBaseAdmin -import org.apache.hadoop.hbase.mapreduce.TableInputFormat - -object HBaseTest { - def main(args: Array[String]) { - val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - - val conf = HBaseConfiguration.create() - - // Other options for configuring scan behavior are available. More information available at - // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html - conf.set(TableInputFormat.INPUT_TABLE, args(1)) - - // Initialize hBase table if necessary - val admin = new HBaseAdmin(conf) - if(!admin.isTableAvailable(args(1))) { - val tableDesc = new HTableDescriptor(args(1)) - admin.createTable(tableDesc) - } - - val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], - classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], - classOf[org.apache.hadoop.hbase.client.Result]) - - hBaseRDD.count() - - System.exit(0) - } -} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6f5607d31c71e..190d72343587e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -200,11 +200,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), - libraryDependencies ++= Seq( - "com.twitter" % "algebird-core_2.9.2" % "0.1.11", - "org.apache.hbase" % "hbase" % "0.94.6" - ) + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From ee6f6aa6cd028e6a3938dcd5334661c27f493bc6 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Thu, 9 May 2013 18:33:38 -0500 Subject: [PATCH 560/696] Add hBase example --- .../main/scala/spark/examples/HBaseTest.scala | 35 +++++++++++++++++++ project/SparkBuild.scala | 6 +++- 2 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/spark/examples/HBaseTest.scala diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala new file mode 100644 index 0000000000000..6e910154d4241 --- /dev/null +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -0,0 +1,35 @@ +package spark.examples + +import spark._ +import spark.rdd.NewHadoopRDD +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} +import org.apache.hadoop.hbase.client.HBaseAdmin +import org.apache.hadoop.hbase.mapreduce.TableInputFormat + +object HBaseTest { + def main(args: Array[String]) { + val sc = new SparkContext(args(0), "HBaseTest", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + + val conf = HBaseConfiguration.create() + + // Other options for configuring scan behavior are available. More information available at + // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html + conf.set(TableInputFormat.INPUT_TABLE, args(1)) + + // Initialize hBase table if necessary + val admin = new HBaseAdmin(conf) + if(!admin.isTableAvailable(args(1))) { + val tableDesc = new HTableDescriptor(args(1)) + admin.createTable(tableDesc) + } + + val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], + classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], + classOf[org.apache.hadoop.hbase.client.Result]) + + hBaseRDD.count() + + System.exit(0) + } +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 190d72343587e..57fe04ea2d5b3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -200,7 +200,11 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") + resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), + libraryDependencies ++= Seq( + "com.twitter" % "algebird-core_2.9.2" % "0.1.11", + "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty) + ) ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From 6e6b3e0d7eadab97d45e975452c7e0c18246686e Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 10 May 2013 13:02:34 -0700 Subject: [PATCH 561/696] Actually use the cleaned closure in foreachPartition --- core/src/main/scala/spark/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index fd14ef17f1ce6..dde131696fdf2 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -489,7 +489,7 @@ abstract class RDD[T: ClassManifest]( */ def foreachPartition(f: Iterator[T] => Unit) { val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => f(iter)) + sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) } /** From ee37612bc95e8486fa328908005293585912db71 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sat, 11 May 2013 11:12:22 +0530 Subject: [PATCH 562/696] 1) Add support for HADOOP_CONF_DIR (and/or YARN_CONF_DIR - use either) : which is used to specify the client side configuration directory : which needs to be part of the CLASSPATH. 2) Move from var+=".." to var="$var.." : the former does not work on older bash shells unfortunately. --- docs/running-on-yarn.md | 3 ++ run | 65 ++++++++++++++++++++++++----------------- run2.cmd | 13 +++++++++ 3 files changed, 54 insertions(+), 27 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 26424bbe52a9b..c8cf8ffc35451 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -30,6 +30,9 @@ If you want to test out the YARN deployment mode, you can use the current Spark # Launching Spark on YARN +Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster. +This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager. + The command to launch the YARN Client is as follows: SPARK_JAR= ./run spark.deploy.yarn.Client \ diff --git a/run b/run index 0a58ac4a36eb7..c744bbd3dc86c 100755 --- a/run +++ b/run @@ -22,7 +22,7 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - SPARK_DAEMON_JAVA_OPTS+=" -Dspark.akka.logLifecycleEvents=true" + SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default fi @@ -30,19 +30,19 @@ fi # Add java opts for master, worker, executor. The opts maybe null case "$1" in 'spark.deploy.master.Master') - SPARK_JAVA_OPTS+=" $SPARK_MASTER_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_MASTER_OPTS" ;; 'spark.deploy.worker.Worker') - SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_WORKER_OPTS" ;; 'spark.executor.StandaloneExecutorBackend') - SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" ;; 'spark.executor.MesosExecutorBackend') - SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" ;; 'spark.repl.Main') - SPARK_JAVA_OPTS+=" $SPARK_REPL_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS" ;; esac @@ -85,11 +85,11 @@ export SPARK_MEM # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH" -JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM" +JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e $FWDIR/conf/java-opts ] ; then - JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" + JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS @@ -110,30 +110,30 @@ fi # Build up classpath CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH+=":$FWDIR/conf" -CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$FWDIR/conf" +CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" if [ -n "$SPARK_TESTING" ] ; then - CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" fi -CLASSPATH+=":$CORE_DIR/src/main/resources" -CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH+=":$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar +CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" +CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar if [ -e "$FWDIR/lib_managed" ]; then - CLASSPATH+=":$FWDIR/lib_managed/jars/*" - CLASSPATH+=":$FWDIR/lib_managed/bundles/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" fi -CLASSPATH+=":$REPL_DIR/lib/*" +CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" if [ -e $REPL_BIN_DIR/target ]; then for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do - CLASSPATH+=":$jar" + CLASSPATH="$CLASSPATH:$jar" done fi -CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH+=":$jar" + CLASSPATH="$CLASSPATH:$jar" done # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack @@ -147,6 +147,17 @@ if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` fi +# Add hadoop conf dir - else FileSystem.*, etc fail ! +# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +# the configurtion files. +if [ "x" != "x$HADOOP_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" +fi +if [ "x" != "x$YARN_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" +fi + + # Figure out whether to run our class with java or with the scala launcher. # In most cases, we'd prefer to execute our process with java because scala # creates a shell script as the parent of its Java process, which makes it @@ -156,9 +167,9 @@ fi if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS else - CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-library.jar" - CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-compiler.jar" - CLASSPATH+=":$SCALA_LIBRARY_PATH/jline.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" fi diff --git a/run2.cmd b/run2.cmd index d2d4807971d29..c6f43dde5b260 100644 --- a/run2.cmd +++ b/run2.cmd @@ -63,6 +63,19 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + + + rem Figure out the JAR file that our examples were packaged into. rem First search in the build path from SBT: for %%d in ("examples/target/scala-%SCALA_VERSION%/spark-examples*.jar") do ( From 0345954530a445b275595962c9f949cad55a01f6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 14:17:09 -0700 Subject: [PATCH 563/696] SPARK-738: Spark should detect and squash nonserializable exceptions --- .../main/scala/spark/executor/Executor.scala | 16 ++++++++++++-- .../test/scala/spark/DistributedSuite.scala | 21 +++++++++++++++++++ 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 344face5e6986..f9061b1c715dd 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,6 +1,6 @@ package spark.executor -import java.io.{File, FileOutputStream} +import java.io.{NotSerializableException, File, FileOutputStream} import java.net.{URI, URL, URLClassLoader} import java.util.concurrent._ @@ -123,7 +123,19 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert case t: Throwable => { val reason = ExceptionFailure(t) - context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) + val serReason = + try { + ser.serialize(reason) + } + catch { + case e: NotSerializableException => { + val message = "Spark caught unserializable exn: " + t.toString + val throwable = new Exception(message) + throwable.setStackTrace(t.getStackTrace) + ser.serialize(new ExceptionFailure(throwable)) + } + } + context.statusUpdate(taskId, TaskState.FAILED, serReason) // TODO: Should we exit the whole executor here? On the one hand, the failed task may // have left some weird state around depending on when the exception was thrown, but on diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4df3bb5b6793b..8ab0f2cfa2709 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -18,6 +18,9 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ import storage.{GetBlock, BlockManagerWorker, StorageLevel} +class NotSerializableClass +class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} + class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" @@ -27,6 +30,24 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter System.clearProperty("spark.storage.memoryFraction") } + test("task throws not serializable exception") { + // Ensures that executors do not crash when an exn is not serializable. If executors crash, + // this test will hang. Correct behavior is that executors don't crash but fail tasks + // and the scheduler throws a SparkException. + + // numSlaves must be less than numPartitions + val numSlaves = 3 + val numPartitions = 10 + + sc = new SparkContext("local-cluster[%s,1,512]".format(numSlaves), "test") + val data = sc.parallelize(1 to 100, numPartitions).map(x => (x, x)). + map(x => throw new NotSerializableExn(new NotSerializableClass)) + intercept[SparkException] { + data.count() + } + resetSparkContext() + } + test("local-cluster format") { sc = new SparkContext("local-cluster[2,1,512]", "test") assert(sc.parallelize(1 to 2, 2).count() == 2) From a5c28bb888f74d27893c198865f588ca0334a8a6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 14:20:39 -0700 Subject: [PATCH 564/696] Removing unnecessary map --- core/src/test/scala/spark/DistributedSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 8ab0f2cfa2709..33c99471c69d7 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -40,7 +40,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter val numPartitions = 10 sc = new SparkContext("local-cluster[%s,1,512]".format(numSlaves), "test") - val data = sc.parallelize(1 to 100, numPartitions).map(x => (x, x)). + val data = sc.parallelize(1 to 100, numPartitions). map(x => throw new NotSerializableExn(new NotSerializableClass)) intercept[SparkException] { data.count() From 440719109e10ea1cc6149a8f61d42ea7cc443352 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 11 May 2013 18:27:26 -0700 Subject: [PATCH 565/696] Throw exception if task result exceeds Akka frame size. This partially addresses SPARK-747. --- core/src/main/scala/spark/executor/Executor.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 344face5e6986..718f0ff5bcf8d 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -72,6 +72,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) SparkEnv.set(env) + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool val threadPool = new ThreadPoolExecutor( @@ -113,6 +114,9 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null)) val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) + if (serializedResult.limit >= (akkaFrameSize - 1024)) { + throw new SparkException("Result for " + taskId + " exceeded Akka frame size") + } context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) logInfo("Finished task ID " + taskId) } catch { From 3da2305ed0d4add7127953e5240632f86053b4aa Mon Sep 17 00:00:00 2001 From: Cody Koeninger Date: Sat, 11 May 2013 23:59:07 -0500 Subject: [PATCH 566/696] code cleanup per rxin comments --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 67 ++++++++++++--------- 1 file changed, 37 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index 4c3054465c384..b0f7054233f75 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -5,23 +5,27 @@ import java.sql.{Connection, ResultSet} import spark.{Logging, Partition, RDD, SparkContext, TaskContext} import spark.util.NextIterator +private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { + override def index = idx +} + /** - An RDD that executes an SQL query on a JDBC connection and reads results. - @param getConnection a function that returns an open Connection. - The RDD takes care of closing the connection. - @param sql the text of the query. - The query must contain two ? placeholders for parameters used to partition the results. - E.g. "select title, author from books where ? <= id and id <= ?" - @param lowerBound the minimum value of the first placeholder - @param upperBound the maximum value of the second placeholder - The lower and upper bounds are inclusive. - @param numPartitions the number of partitions. - Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, - the query would be executed twice, once with (1, 10) and once with (11, 20) - @param mapRow a function from a ResultSet to a single row of the desired result type(s). - This should only call getInt, getString, etc; the RDD takes care of calling next. - The default maps a ResultSet to an array of Object. -*/ + * An RDD that executes an SQL query on a JDBC connection and reads results. + * @param getConnection a function that returns an open Connection. + * The RDD takes care of closing the connection. + * @param sql the text of the query. + * The query must contain two ? placeholders for parameters used to partition the results. + * E.g. "select title, author from books where ? <= id and id <= ?" + * @param lowerBound the minimum value of the first placeholder + * @param upperBound the maximum value of the second placeholder + * The lower and upper bounds are inclusive. + * @param numPartitions the number of partitions. + * Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + * the query would be executed twice, once with (1, 10) and once with (11, 20) + * @param mapRow a function from a ResultSet to a single row of the desired result type(s). + * This should only call getInt, getString, etc; the RDD takes care of calling next. + * The default maps a ResultSet to an array of Object. + */ class JdbcRDD[T: ClassManifest]( sc: SparkContext, getConnection: () => Connection, @@ -29,26 +33,33 @@ class JdbcRDD[T: ClassManifest]( lowerBound: Long, upperBound: Long, numPartitions: Int, - mapRow: (ResultSet) => T = JdbcRDD.resultSetToObjectArray) + mapRow: (ResultSet) => T = JdbcRDD.resultSetToObjectArray _) extends RDD[T](sc, Nil) with Logging { - override def getPartitions: Array[Partition] = - ParallelCollectionRDD.slice(lowerBound to upperBound, numPartitions). - filter(! _.isEmpty). - zipWithIndex. - map(x => new JdbcPartition(x._2, x._1.head, x._1.last)). - toArray + override def getPartitions: Array[Partition] = { + // bounds are inclusive, hence the + 1 here and - 1 on end + val length = 1 + upperBound - lowerBound + (0 until numPartitions).map(i => { + val start = lowerBound + ((i * length) / numPartitions).toLong + val end = lowerBound + (((i + 1) * length) / numPartitions).toLong - 1 + new JdbcPartition(i, start, end) + }).toArray + } override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { context.addOnCompleteCallback{ () => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) - // force mysql driver to stream rather than pull entire resultset into memory + + // setFetchSize(Integer.MIN_VALUE) is a mysql driver specific way to force streaming results, + // rather than pulling entire resultset into memory. + // see http://dev.mysql.com/doc/refman/5.0/en/connector-j-reference-implementation-notes.html if (conn.getMetaData.getURL.matches("jdbc:mysql:.*")) { stmt.setFetchSize(Integer.MIN_VALUE) logInfo("statement fetch size set to: " + stmt.getFetchSize + " to force MySQL streaming ") } + stmt.setLong(1, part.lower) stmt.setLong(2, part.upper) val rs = stmt.executeQuery() @@ -81,14 +92,10 @@ class JdbcRDD[T: ClassManifest]( } } } - -} - -private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { - override def index = idx } object JdbcRDD { - val resultSetToObjectArray = (rs: ResultSet) => + def resultSetToObjectArray(rs: ResultSet) = { Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) + } } From 059ab8875463ab22fe329fb6a627cac0a7d8158c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 23:39:14 -0700 Subject: [PATCH 567/696] Changing technique to use same code path in all cases --- core/src/main/scala/spark/TaskEndReason.scala | 13 ++++++++++--- .../src/main/scala/spark/executor/Executor.scala | 16 ++-------------- .../spark/scheduler/cluster/TaskSetManager.scala | 8 ++++---- .../spark/scheduler/local/LocalScheduler.scala | 3 ++- 4 files changed, 18 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index 420c54bc9a0d4..ce9bb498974e7 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -14,9 +14,16 @@ private[spark] case object Success extends TaskEndReason private[spark] case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -private[spark] -case class FetchFailed(bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason +private[spark] case class FetchFailed( + bmAddress: BlockManagerId, + shuffleId: Int, + mapId: Int, + reduceId: Int) + extends TaskEndReason -private[spark] case class ExceptionFailure(exception: Throwable) extends TaskEndReason +private[spark] case class ExceptionFailure( + description: String, + stackTrace: Array[StackTraceElement]) + extends TaskEndReason private[spark] case class OtherFailure(message: String) extends TaskEndReason diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index f9061b1c715dd..9084def9b2629 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -122,20 +122,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } case t: Throwable => { - val reason = ExceptionFailure(t) - val serReason = - try { - ser.serialize(reason) - } - catch { - case e: NotSerializableException => { - val message = "Spark caught unserializable exn: " + t.toString - val throwable = new Exception(message) - throwable.setStackTrace(t.getStackTrace) - ser.serialize(new ExceptionFailure(throwable)) - } - } - context.statusUpdate(taskId, TaskState.FAILED, serReason) + val reason = ExceptionFailure(t.toString, t.getStackTrace) + context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // TODO: Should we exit the whole executor here? On the one hand, the failed task may // have left some weird state around depending on when the exception was thrown, but on diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 27e713e2c43dd..6d663de2f8571 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -493,7 +493,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return case ef: ExceptionFailure => - val key = ef.exception.toString + val key = ef.description val now = System.currentTimeMillis val (printFull, dupCount) = { if (recentExceptions.contains(key)) { @@ -511,10 +511,10 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } } if (printFull) { - val locs = ef.exception.getStackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s".format(ef.exception.toString, locs.mkString("\n"))) + val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s".format(ef.description, locs.mkString("\n"))) } else { - logInfo("Loss was due to %s [duplicate %d]".format(ef.exception.toString, dupCount)) + logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) } case _ => {} diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index f060a940a9269..42d5bc481306e 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -102,7 +102,8 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } else { // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, new ExceptionFailure(t), null, null, info, null) + listener.taskEnded( + task, new ExceptionFailure(t.getMessage, t.getStackTrace), null, null, info, null) } } } From 1c15b8505124c157449b6d41e1127f3eb4081a23 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 23:52:53 -0700 Subject: [PATCH 568/696] Removing import --- core/src/main/scala/spark/executor/Executor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 9084def9b2629..1d5516966d89d 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,6 +1,6 @@ package spark.executor -import java.io.{NotSerializableException, File, FileOutputStream} +import java.io.{File, FileOutputStream} import java.net.{URI, URL, URLClassLoader} import java.util.concurrent._ From 72b9c4cb6ec4080eb8751e5e040f180272ac82a6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 23:53:50 -0700 Subject: [PATCH 569/696] Small fix --- core/src/main/scala/spark/scheduler/local/LocalScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 42d5bc481306e..a357422466ccc 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -103,7 +103,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) listener.taskEnded( - task, new ExceptionFailure(t.getMessage, t.getStackTrace), null, null, info, null) + task, new ExceptionFailure(t.toString, t.getStackTrace), null, null, info, null) } } } From 7f0833647b784c4ec7cd2f2e8e4fcd5ed6f673cd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 12 May 2013 07:54:03 -0700 Subject: [PATCH 570/696] Capturing class name --- core/src/main/scala/spark/TaskEndReason.scala | 1 + core/src/main/scala/spark/executor/Executor.scala | 2 +- .../scala/spark/scheduler/cluster/TaskSetManager.scala | 3 ++- .../main/scala/spark/scheduler/local/LocalScheduler.scala | 7 ++++--- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index ce9bb498974e7..ca793eb4021a6 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -22,6 +22,7 @@ private[spark] case class FetchFailed( extends TaskEndReason private[spark] case class ExceptionFailure( + className: String, description: String, stackTrace: Array[StackTraceElement]) extends TaskEndReason diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 1d5516966d89d..da20b8454441c 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -122,7 +122,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } case t: Throwable => { - val reason = ExceptionFailure(t.toString, t.getStackTrace) + val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // TODO: Should we exit the whole executor here? On the one hand, the failed task may diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 6d663de2f8571..06de3c755eeee 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -512,7 +512,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } if (printFull) { val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s".format(ef.description, locs.mkString("\n"))) + logInfo("Loss was due to %s\n%s\n%s".format( + ef.className, ef.description, locs.mkString("\n"))) } else { logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index a357422466ccc..ebe42685ad016 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -101,9 +101,10 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon submitTask(task, idInJob) } else { // TODO: Do something nicer here to return all the way to the user - if (!Thread.currentThread().isInterrupted) - listener.taskEnded( - task, new ExceptionFailure(t.toString, t.getStackTrace), null, null, info, null) + if (!Thread.currentThread().isInterrupted) { + val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + listener.taskEnded(task, failure, null, null, info, null) + } } } } From b16c4896f617f352bb230908b7c08c7c5b028434 Mon Sep 17 00:00:00 2001 From: Cody Koeninger Date: Tue, 14 May 2013 23:44:04 -0500 Subject: [PATCH 571/696] add test for JdbcRDD using embedded derby, per rxin suggestion --- .gitignore | 1 + .../test/scala/spark/rdd/JdbcRDDSuite.scala | 56 +++++++++++++++++++ project/SparkBuild.scala | 1 + 3 files changed, 58 insertions(+) create mode 100644 core/src/test/scala/spark/rdd/JdbcRDDSuite.scala diff --git a/.gitignore b/.gitignore index 155e785b01beb..b87fc1ee7944c 100644 --- a/.gitignore +++ b/.gitignore @@ -36,3 +36,4 @@ streaming-tests.log dependency-reduced-pom.xml .ensime .ensime_lucene +derby.log diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala new file mode 100644 index 0000000000000..6afb0fa9bc77f --- /dev/null +++ b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala @@ -0,0 +1,56 @@ +package spark + +import org.scalatest.{ BeforeAndAfter, FunSuite } +import spark.SparkContext._ +import spark.rdd.JdbcRDD +import java.sql._ + +class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + before { + Class.forName("org.apache.derby.jdbc.EmbeddedDriver") + val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true") + try { + val create = conn.createStatement + create.execute(""" + CREATE TABLE FOO( + ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), + DATA INTEGER + )""") + create.close + val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") + (1 to 100).foreach { i => + insert.setInt(1, i * 2) + insert.executeUpdate + } + insert.close + } catch { + case e: SQLException if e.getSQLState == "X0Y32" => + // table exists + } finally { + conn.close + } + } + + test("basic functionality") { + sc = new SparkContext("local", "test") + val rdd = new JdbcRDD( + sc, + () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") }, + "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?", + 1, 100, 3, + (r: ResultSet) => { r.getInt(1) } ).cache + + assert(rdd.count === 100) + assert(rdd.reduce(_+_) === 10100) + } + + after { + try { + DriverManager.getConnection("jdbc:derby:;shutdown=true") + } catch { + case se: SQLException if se.getSQLState == "XJ015" => + // normal shutdown + } + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f0b371b2cf682..b11893590e762 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -147,6 +147,7 @@ object SparkBuild extends Build { "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", "cc.spray" % "spray-json_2.9.2" % "1.1.1", + "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.mesos" % "mesos" % "0.9.0-incubating" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } From d7d1da79d30961e461115a73bbfc9e4c4448e533 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 May 2013 16:37:49 -0700 Subject: [PATCH 572/696] when akka starts, use akkas default classloader (current thread) --- core/src/main/scala/spark/util/AkkaUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 9fb7e001badcb..cd79bd2bdad0c 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -52,7 +52,7 @@ private[spark] object AkkaUtils { """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, lifecycleEvents, akkaWriteTimeout)) - val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader) + val actorSystem = ActorSystem(name, akkaConf) // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. From 38d4b97c6d47df4e1f1a3279ff786509f60e0eaf Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 May 2013 16:50:43 -0700 Subject: [PATCH 573/696] use threads classloader when deserializing task results; classnotfoundexception includes classloader --- .../spark/scheduler/cluster/TaskSetManager.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index c69f3bdb7f564..b348092d89b50 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -503,9 +503,16 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + try { + val result = ser.deserialize[TaskResult[_]](serializedData) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + } catch { + case cnf: ClassNotFoundException => + val loader = Thread.currentThread().getContextClassLoader + throw new SparkException("ClassNotFound with classloader: " + loader, cnf) + case ex => throw ex + } // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { From 404f9ff617401a2f8d12845861ce8f02cfe6442c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 14 May 2013 23:28:34 -0700 Subject: [PATCH 574/696] Added derby dependency to Maven pom files for the JDBC Java test. --- core/pom.xml | 5 +++++ pom.xml | 8 +++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 9a019b5a42e53..57a95328c397b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -92,6 +92,11 @@ log4j + + org.apache.derby + derby + test + org.scalatest scalatest_${scala.version} diff --git a/pom.xml b/pom.xml index 3936165d784ed..d7cdc591cf487 100644 --- a/pom.xml +++ b/pom.xml @@ -256,6 +256,12 @@ mesos ${mesos.version} + + org.apache.derby + derby + 10.4.2.0 + test + org.scala-lang @@ -565,7 +571,7 @@ 2 - 2.0.2-alpha + 2.0.2-alpha From f9d40a5848a2e1eef31ac63cd9221d5b77c1c5a7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 14 May 2013 23:29:57 -0700 Subject: [PATCH 575/696] Added a comment in JdbcRDD for example usage. --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index b0f7054233f75..a50f40773738d 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -11,11 +11,13 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e /** * An RDD that executes an SQL query on a JDBC connection and reads results. + * For usage example, see test case JdbcRDDSuite. + * * @param getConnection a function that returns an open Connection. * The RDD takes care of closing the connection. * @param sql the text of the query. * The query must contain two ? placeholders for parameters used to partition the results. - * E.g. "select title, author from books where ? <= id and id <= ?" + * E.g. "select title, author from books where ? <= id and id <= ?" * @param lowerBound the minimum value of the first placeholder * @param upperBound the maximum value of the second placeholder * The lower and upper bounds are inclusive. From afcad7b3aa8736231a526417ede47ce6d353a70c Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 15 May 2013 14:45:14 -0300 Subject: [PATCH 576/696] Docs: Mention spark shell's default for MASTER --- docs/scala-programming-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2315aadbdf17c..b0da130fcb320 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -67,6 +67,8 @@ The master URL passed to Spark can be in one of the following formats:
    spark.ui.port(random) + Port for your application's dashboard, which shows memory usage of each RDD. +
    spark.shuffle.compress true
    groupByKey([numTasks]) When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together all the values of each key in the RDDs of the source DStream.
    - Note: By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional numTasks argument to set a different number of tasks. + Note: By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluster) to do the grouping. You can pass an optional numTasks argument to set a different number of tasks.
    groupByKeyAndWindow(windowDuration, slideDuration, [numTasks]) When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together values of each key over batches in a sliding window.
    -Note: By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional numTasks argument to set a different number of tasks.
    reduceByKeyAndWindow(func, windowDuration, slideDuration, [numTasks])
    @(status.blockManagerId.ip + ":" + status.blockManagerId.port)@(status.blockManagerId.host + ":" + status.blockManagerId.port) @(Utils.memoryBytesToString(status.memUsed(prefix))) (@(Utils.memoryBytesToString(status.memRemaining)) Total Available) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4104b33c8b681..c9b4707def82b 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -153,7 +153,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter val blockManager = SparkEnv.get.blockManager blockManager.master.getLocations(blockId).foreach(id => { val bytes = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(id.ip, id.port)) + GetBlock(blockId), ConnectionManagerId(id.host, id.port)) val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList assert(deserialized === (1 to 100).toList) }) diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index 91b48c745659b..a3840905f4c0b 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -18,6 +18,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsTextFile(outputDir) + println("outputDir = " + outputDir) // Read the plain text file and check it's OK val outputFile = new File(outputDir, "part-00000") val content = Source.fromFile(outputFile).mkString diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 6da58a0f6e5fe..c0f8986de8dd5 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -271,7 +271,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont // have the 2nd attempt pass complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) // we can see both result blocks now - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.ip) === Array("hostA", "hostB")) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) } From 54b3d45b816f26a9d3509c1f8bea70c6d99d3de0 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Mon, 15 Apr 2013 18:26:50 +0530 Subject: [PATCH 450/696] Checkpoint commit - compiles and passes a lot of tests - not all though, looking into FileSuite issues --- core/src/main/scala/spark/HadoopWriter.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala index afcf9f6db4a2b..80421b6328a53 100644 --- a/core/src/main/scala/spark/HadoopWriter.scala +++ b/core/src/main/scala/spark/HadoopWriter.scala @@ -24,6 +24,8 @@ import spark.SerializableWritable * a filename to write to, etc, exactly like in a Hadoop MapReduce job. */ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRedUtil with Serializable { + + println("Created HadoopWriter") private val now = new Date() private val conf = new SerializableWritable(jobConf) @@ -41,6 +43,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe @transient private var taskContext: TaskAttemptContext = null def preSetup() { + println("preSetup") setIDs(0, 0, 0) setConfParams() @@ -50,17 +53,20 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe def setup(jobid: Int, splitid: Int, attemptid: Int) { + println("setup") setIDs(jobid, splitid, attemptid) setConfParams() } def open() { + println("open") val numfmt = NumberFormat.getInstance() numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) val outputName = "part-" + numfmt.format(splitID) val path = FileOutputFormat.getOutputPath(conf.value) + println("open outputName = " + outputName + ", fs for " + conf.value) val fs: FileSystem = { if (path != null) { path.getFileSystem(conf.value) @@ -75,6 +81,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe } def write(key: AnyRef, value: AnyRef) { + println("write " + key + " = " + value) if (writer!=null) { //println (">>> Writing ("+key.toString+": " + key.getClass.toString + ", " + value.toString + ": " + value.getClass.toString + ")") writer.write(key, value) @@ -84,16 +91,19 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe } def close() { + println("close") writer.close(Reporter.NULL) } def commit() { + println("commit") val taCtxt = getTaskContext() val cmtr = getOutputCommitter() if (cmtr.needsTaskCommit(taCtxt)) { try { cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") + println("Committed = " + taID) } catch { case e: IOException => { logError("Error committing the output of task: " + taID.value, e) @@ -102,11 +112,13 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe } } } else { + println("No need to commit") logWarning ("No need to commit output of task: " + taID.value) } } def cleanup() { + println("cleanup") getOutputCommitter().cleanupJob(getJobContext()) } From 19652a44be81f3b8fbbb9ecc4987dcd933d2eca9 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Mon, 15 Apr 2013 19:16:36 +0530 Subject: [PATCH 451/696] Fix issue with FileSuite failing --- core/src/main/scala/spark/HadoopWriter.scala | 22 +++++-------------- .../main/scala/spark/PairRDDFunctions.scala | 1 + core/src/test/scala/spark/FileSuite.scala | 1 - 3 files changed, 7 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala index 80421b6328a53..5e8396edb9223 100644 --- a/core/src/main/scala/spark/HadoopWriter.scala +++ b/core/src/main/scala/spark/HadoopWriter.scala @@ -2,14 +2,10 @@ package org.apache.hadoop.mapred import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.apache.hadoop.util.ReflectionUtils -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text import java.text.SimpleDateFormat import java.text.NumberFormat import java.io.IOException -import java.net.URI import java.util.Date import spark.Logging @@ -25,8 +21,6 @@ import spark.SerializableWritable */ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRedUtil with Serializable { - println("Created HadoopWriter") - private val now = new Date() private val conf = new SerializableWritable(jobConf) @@ -43,7 +37,6 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe @transient private var taskContext: TaskAttemptContext = null def preSetup() { - println("preSetup") setIDs(0, 0, 0) setConfParams() @@ -53,20 +46,17 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe def setup(jobid: Int, splitid: Int, attemptid: Int) { - println("setup") setIDs(jobid, splitid, attemptid) setConfParams() } def open() { - println("open") val numfmt = NumberFormat.getInstance() numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) val outputName = "part-" + numfmt.format(splitID) val path = FileOutputFormat.getOutputPath(conf.value) - println("open outputName = " + outputName + ", fs for " + conf.value) val fs: FileSystem = { if (path != null) { path.getFileSystem(conf.value) @@ -81,7 +71,6 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe } def write(key: AnyRef, value: AnyRef) { - println("write " + key + " = " + value) if (writer!=null) { //println (">>> Writing ("+key.toString+": " + key.getClass.toString + ", " + value.toString + ": " + value.getClass.toString + ")") writer.write(key, value) @@ -91,19 +80,16 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe } def close() { - println("close") writer.close(Reporter.NULL) } def commit() { - println("commit") val taCtxt = getTaskContext() val cmtr = getOutputCommitter() if (cmtr.needsTaskCommit(taCtxt)) { try { cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") - println("Committed = " + taID) } catch { case e: IOException => { logError("Error committing the output of task: " + taID.value, e) @@ -112,13 +98,17 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe } } } else { - println("No need to commit") logWarning ("No need to commit output of task: " + taID.value) } } + def commitJob() { + // always ? Or if cmtr.needsTaskCommit ? + val cmtr = getOutputCommitter() + cmtr.commitJob(getJobContext()) + } + def cleanup() { - println("cleanup") getOutputCommitter().cleanupJob(getJobContext()) } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 39469fa3c87e8..9a6966b3f1524 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -636,6 +636,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } self.context.runJob(self, writeToFile _) + writer.commitJob() writer.cleanup() } diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index a3840905f4c0b..91b48c745659b 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -18,7 +18,6 @@ class FileSuite extends FunSuite with LocalSparkContext { val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsTextFile(outputDir) - println("outputDir = " + outputDir) // Read the plain text file and check it's OK val outputFile = new File(outputDir, "part-00000") val content = Source.fromFile(outputFile).mkString From a64c10744988f55776e61c305734f9f5a42f9ea5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Apr 2013 16:41:51 -0400 Subject: [PATCH 452/696] Make ShuffledRDD.prev transient --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/ShuffledRDD.scala | 2 +- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 9213513e80914..a6235491cab96 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -29,7 +29,7 @@ private[spark] case class NarrowCoGroupSplitDep( private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep private[spark] -class CoGroupPartition(idx: Int, val deps: Seq[CoGroupSplitDep]) +class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) extends Partition with Serializable { override val index: Int = idx override def hashCode(): Int = idx @@ -88,7 +88,7 @@ class CoGroupedRDD[K]( case _ => new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } - }.toList) + }.toArray) } array } diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 51f02409b6a75..4e33b7dd5ca84 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -16,7 +16,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @tparam V the value class. */ class ShuffledRDD[K, V]( - prev: RDD[(K, V)], + @transient prev: RDD[(K, V)], part: Partitioner) extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part))) { diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 0a02561062677..481e03b349af0 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -56,7 +56,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM case _ => new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } - }.toList) + }.toArray) } array } From eb7e95e833376904bea4a9e6d1cc67c00fcfb06c Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 16 Apr 2013 02:56:36 +0530 Subject: [PATCH 453/696] Commit job to persist files --- core/src/main/scala/spark/PairRDDFunctions.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 9a6966b3f1524..67fd1c1a8f68d 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -569,6 +569,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) jobCommitter.setupJob(jobTaskContext) val count = self.context.runJob(self, writeShard _).sum + jobCommitter.commitJob(jobTaskContext) jobCommitter.cleanupJob(jobTaskContext) } From 5540ab8243a8488e30a21e1d4bb1720f1a9a555f Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 16 Apr 2013 02:57:43 +0530 Subject: [PATCH 454/696] Use hostname instead of hostport for executor, fix creation of workdir --- .../scala/spark/deploy/worker/ExecutorRunner.scala | 2 +- core/src/main/scala/spark/deploy/worker/Worker.scala | 3 ++- .../spark/executor/StandaloneExecutorBackend.scala | 11 ++++++----- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index dfcb9f0d05d5f..04a774658e426 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -70,7 +70,7 @@ private[spark] class ExecutorRunner( /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */ def substituteVariables(argument: String): String = argument match { case "{{EXECUTOR_ID}}" => execId.toString - case "{{HOSTPORT}}" => hostPort + case "{{HOSTNAME}}" => Utils.parseHostPort(hostPort)._1 case "{{CORES}}" => cores.toString case other => other } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index cf4babc892703..1a7da0f7bf911 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -54,10 +54,11 @@ private[spark] class Worker( def createWorkDir() { workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) try { - if (!workDir.exists() && !workDir.mkdirs()) { + if ( (workDir.exists() && !workDir.isDirectory) || (!workDir.exists() && !workDir.mkdirs()) ) { logError("Failed to create work directory " + workDir) System.exit(1) } + assert (workDir.isDirectory) } catch { case e: Exception => logError("Failed to create work directory " + workDir, e) diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 49e1f3f07a995..ebe2ac68d8b25 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -75,17 +75,18 @@ private[spark] object StandaloneExecutorBackend { def run0(args: Product) { assert(4 == args.productArity) runImpl(args.productElement(0).asInstanceOf[String], - args.productElement(0).asInstanceOf[String], - args.productElement(0).asInstanceOf[String], - args.productElement(0).asInstanceOf[Int]) + args.productElement(1).asInstanceOf[String], + args.productElement(2).asInstanceOf[String], + args.productElement(3).asInstanceOf[Int]) } private def runImpl(driverUrl: String, executorId: String, hostname: String, cores: Int) { + // Debug code + Utils.checkHost(hostname) + // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0) - // Debug code - Utils.checkHost(hostname) // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 6b61152ed0374..0b8922d139490 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -27,7 +27,7 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTPORT}}", "{{CORES}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) From dd2b64ec97ad241b6f171cac0dbb1841b185675a Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 16 Apr 2013 03:19:24 +0530 Subject: [PATCH 455/696] Fix bug with atomic update --- .../scala/spark/storage/BlockManager.scala | 44 +++++++++++++------ 1 file changed, 31 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 10e70723db549..483b6de34b4ab 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -527,13 +527,22 @@ class BlockManager( // Remember the block's storage level so that we can correctly drop it to disk if it needs // to be dropped right after it got put into memory. Note, however, that other threads will // not be able to get() this block until we call markReady on its BlockInfo. - val myInfo = new BlockInfo(level, tellMaster) - // Do atomically ! - val oldBlockOpt = blockInfo.putIfAbsent(blockId, myInfo) + val myInfo = { + val tinfo = new BlockInfo(level, tellMaster) + // Do atomically ! + val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) + + if (oldBlockOpt.isDefined) { + if (oldBlockOpt.get.waitForReady()) { + logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + return oldBlockOpt.get.size + } - if (oldBlockOpt.isDefined && oldBlockOpt.get.waitForReady()) { - logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - return oldBlockOpt.get.size + // TODO: So the block info exists - but previous attempt to load it (?) failed. What do we do now ? Retry on it ? + oldBlockOpt.get + } else { + tinfo + } } val startTimeMs = System.currentTimeMillis @@ -638,13 +647,22 @@ class BlockManager( // Remember the block's storage level so that we can correctly drop it to disk if it needs // to be dropped right after it got put into memory. Note, however, that other threads will // not be able to get() this block until we call markReady on its BlockInfo. - val myInfo = new BlockInfo(level, tellMaster) - // Do atomically ! - val prevInfo = blockInfo.putIfAbsent(blockId, myInfo) - if (prevInfo != null) { - // Should we check for prevInfo.waitForReady() here ? - logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - return + val myInfo = { + val tinfo = new BlockInfo(level, tellMaster) + // Do atomically ! + val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) + + if (oldBlockOpt.isDefined) { + if (oldBlockOpt.get.waitForReady()) { + logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + return + } + + // TODO: So the block info exists - but previous attempt to load it (?) failed. What do we do now ? Retry on it ? + oldBlockOpt.get + } else { + tinfo + } } val startTimeMs = System.currentTimeMillis From 59c380d69a3831f0239b434a0fa1cf26a481d222 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 16 Apr 2013 03:29:38 +0530 Subject: [PATCH 456/696] Fix npe --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 483b6de34b4ab..c98ee5a0e722a 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -794,7 +794,7 @@ class BlockManager( diskStore.putBytes(blockId, bytes, level) } } - val droppedMemorySize = memoryStore.getSize(blockId) + val droppedMemorySize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val blockWasRemoved = memoryStore.remove(blockId) if (!blockWasRemoved) { logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") From b493f55a4fe43c83061a361eef029edbac50c006 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Tue, 16 Apr 2013 10:00:33 +0800 Subject: [PATCH 457/696] fix a bug in netty Block Fetcher Signed-off-by: shane-huang --- .../java/spark/network/netty/FileServer.java | 1 - .../scala/spark/storage/BlockManager.scala | 69 ++++++++++--------- .../main/scala/spark/storage/DiskStore.scala | 4 +- 3 files changed, 37 insertions(+), 37 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index 729e45f0a13a8..38af3050967eb 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -51,7 +51,6 @@ public void stop(){ } if (bootstrap != null){ bootstrap.shutdown(); - bootstrap = null; } } } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index b8b68d4283a61..5a001809223e6 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -469,21 +469,6 @@ class BlockManager( getLocal(blockId).orElse(getRemote(blockId)) } - /** - * A request to fetch one or more blocks, complete with their sizes - */ - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - - /** - * A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - * the block (since we want all deserializaton to happen in the calling thread); can also - * represent a fetch failure if size == -1. - */ - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } /** * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined @@ -494,9 +479,9 @@ class BlockManager( : BlockFetcherIterator = { if(System.getProperty("spark.shuffle.use.netty", "false").toBoolean){ - return new NettyBlockFetcherIterator(this, blocksByAddress) + return BlockFetcherIterator("netty",this, blocksByAddress) } else { - return new BlockFetcherIterator(this, blocksByAddress) + return BlockFetcherIterator("", this, blocksByAddress) } } @@ -916,10 +901,29 @@ object BlockManager extends Logging { } } -class BlockFetcherIterator( + +trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { + def initialize +} + +object BlockFetcherIterator { + + // A request to fetch one or more blocks, complete with their sizes + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + // the block (since we want all deserializaton to happen in the calling thread); can also + // represent a fetch failure if size == -1. + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } + +class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] -) extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { +) extends BlockFetcherIterator { import blockManager._ @@ -936,21 +940,9 @@ class BlockFetcherIterator( val localBlockIds = new ArrayBuffer[String]() val remoteBlockIds = new HashSet[String]() - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - // A queue to hold our results. val results = new LinkedBlockingQueue[FetchResult] - // A request to fetch one or more blocks, complete with their sizes - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight val fetchRequests = new Queue[FetchRequest] @@ -1072,7 +1064,6 @@ class BlockFetcherIterator( } - initialize() //an iterator that will read fetched blocks off the queue as they arrive. var resultsGotten = 0 @@ -1107,7 +1098,7 @@ class BlockFetcherIterator( class NettyBlockFetcherIterator( blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] -) extends BlockFetcherIterator(blockManager,blocksByAddress) { +) extends BasicBlockFetcherIterator(blockManager,blocksByAddress) { import blockManager._ @@ -1129,7 +1120,7 @@ class NettyBlockFetcherIterator( } } catch { case x: InterruptedException => logInfo("Copier Interrupted") - case _ => throw new SparkException("Exception Throw in Shuffle Copier") + //case _ => throw new SparkException("Exception Throw in Shuffle Copier") } } } @@ -1232,3 +1223,13 @@ class NettyBlockFetcherIterator( } } + def apply(t: String, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]): BlockFetcherIterator = { + val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress) } + else { new BasicBlockFetcherIterator(blockManager,blocksByAddress) } + iter.initialize + iter + } + +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index d702bb23e0138..cc5bf29a326a7 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -39,7 +39,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() if(useNetty){ - startShuffleBlockSender() + startShuffleBlockSender() } override def getSize(blockId: String): Long = { @@ -229,7 +229,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) case e: Exception => { logError("Error running ShuffleBlockSender ", e) if (shuffleSender != null) { - shuffleSender.stop + shuffleSender.stop shuffleSender = null } } From 323ab8ff3b822af28276e1460db0f9c73d6d6409 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 16 Apr 2013 17:05:10 +0530 Subject: [PATCH 458/696] Scala does not prevent variable shadowing ! Sick error due to it ... --- core/src/main/scala/spark/MapOutputTracker.scala | 1 - core/src/main/scala/spark/storage/BlockManager.scala | 4 ++-- pom.xml | 3 ++- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 6e9da02893c24..fde597ffd1a30 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -148,7 +148,6 @@ private[spark] class MapOutputTracker extends Logging { logInfo("Doing the fetch; tracker actor = " + trackerActor) val hostPort = Utils.localHostPort() // This try-finally prevents hangs due to timeouts: - var fetchedStatuses: Array[MapStatus] = null try { val fetchedBytes = askTracker(GetMapOutputStatuses(shuffleId, hostPort)).asInstanceOf[Array[Byte]] diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index c98ee5a0e722a..6e861ac734604 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -1092,7 +1092,7 @@ class BlockFetcherIterator( logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") //an iterator that will read fetched blocks off the queue as they arrive. - var resultsGotten = 0 + @volatile private var resultsGotten = 0 def hasNext: Boolean = resultsGotten < totalBlocks @@ -1102,7 +1102,7 @@ class BlockFetcherIterator( val result = results.take() val stopFetchWait = System.currentTimeMillis() _fetchWaitTime += (stopFetchWait - startFetchWait) - bytesInFlight -= result.size + if (! result.failed) bytesInFlight -= result.size while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { sendRequest(fetchRequests.dequeue()) diff --git a/pom.xml b/pom.xml index b3134a957d1a3..c0ba6b9b0ebbc 100644 --- a/pom.xml +++ b/pom.xml @@ -417,8 +417,9 @@ ${project.build.directory}/surefire-reports . - WDF TestSuite.txt + ${project.build.directory}/SparkTestSuite.txt -Xms64m -Xmx1024m + From f7969f72eeaba0dc127efd13e983791f6f7930c5 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 16 Apr 2013 21:51:38 +0530 Subject: [PATCH 459/696] Fix exception when checkpoint path does not exist (no data in rdd which is being checkpointed for example) --- .../main/scala/spark/rdd/CheckpointRDD.scala | 26 ++++++++++++++----- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 9e37bdf659201..1026dc54e0d9b 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -21,13 +21,20 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) override def getPartitions: Array[Partition] = { - val dirContents = fs.listStatus(new Path(checkpointPath)) - val partitionFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted - val numPartitions = partitionFiles.size - if (numPartitions > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || - ! partitionFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1)))) { - throw new SparkException("Invalid checkpoint directory: " + checkpointPath) - } + val cpath = new Path(checkpointPath) + val numPartitions = + // listStatus can throw exception if path does not exist. + if (fs.exists(cpath)) { + val dirContents = fs.listStatus(cpath) + val partitionFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted + val numPart = partitionFiles.size + if (numPart > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || + ! partitionFiles(numPart-1).endsWith(CheckpointRDD.splitIdToFile(numPart-1)))) { + throw new SparkException("Invalid checkpoint directory: " + checkpointPath) + } + numPart + } else 0 + Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i)) } @@ -64,6 +71,8 @@ private[spark] object CheckpointRDD extends Logging { val finalOutputPath = new Path(outputDir, finalOutputName) val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId) + println("writeToFile. path = " + path + ", tempOutputPath = " + tempOutputPath + ", finalOutputPath = " + finalOutputPath) + if (fs.exists(tempOutputPath)) { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") @@ -81,8 +90,11 @@ private[spark] object CheckpointRDD extends Logging { serializeStream.writeAll(iterator) serializeStream.close() + println("writeToFile. serializeStream.close ... renaming from " + tempOutputPath + " to " + finalOutputPath) + if (!fs.rename(tempOutputPath, finalOutputPath)) { if (!fs.exists(finalOutputPath)) { + logInfo("Deleting tempOutputPath " + tempOutputPath) fs.delete(tempOutputPath, false) throw new IOException("Checkpoint failed: failed to save output of task: " + ctx.attemptId + " and final output path does not exist") From ad80f68eb5d153d7f666447966755efce186d021 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 16 Apr 2013 22:15:34 +0530 Subject: [PATCH 460/696] remove spurious debug statements --- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 1026dc54e0d9b..24d527f38fa02 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -71,8 +71,6 @@ private[spark] object CheckpointRDD extends Logging { val finalOutputPath = new Path(outputDir, finalOutputName) val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId) - println("writeToFile. path = " + path + ", tempOutputPath = " + tempOutputPath + ", finalOutputPath = " + finalOutputPath) - if (fs.exists(tempOutputPath)) { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") @@ -90,8 +88,6 @@ private[spark] object CheckpointRDD extends Logging { serializeStream.writeAll(iterator) serializeStream.close() - println("writeToFile. serializeStream.close ... renaming from " + tempOutputPath + " to " + finalOutputPath) - if (!fs.rename(tempOutputPath, finalOutputPath)) { if (!fs.exists(finalOutputPath)) { logInfo("Deleting tempOutputPath " + tempOutputPath) From 6bfe4bf3eb9421b4e4143f384a0012c3694df8c9 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 15 Apr 2013 23:32:25 -0700 Subject: [PATCH 461/696] Increase ReservedCodeCacheSize for sbt --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 8f426d18e892f..850c58e1e9745 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then fi export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath -java -Xmx1200M -XX:MaxPermSize=250m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=250m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From bcdde331c3ed68af27bc5d6067c78f68dbd6b032 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 17 Apr 2013 04:12:18 +0530 Subject: [PATCH 462/696] Move from master to driver --- .../spark/deploy/yarn/ApplicationMaster.scala | 20 +++++++++---------- .../deploy/yarn/YarnAllocationHandler.scala | 6 +++--- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 65361e0ed9952..ae719267e8062 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -76,7 +76,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Start the user's JAR userThread = startUserClass() - // This a bit hacky, but we need to wait until the spark.master.port property has + // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. waitForSparkMaster() @@ -124,19 +124,19 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } private def waitForSparkMaster() { - logInfo("Waiting for spark master to be reachable.") - var masterUp = false - while(!masterUp) { - val masterHost = System.getProperty("spark.master.host") - val masterPort = System.getProperty("spark.master.port") + logInfo("Waiting for spark driver to be reachable.") + var driverUp = false + while(!driverUp) { + val driverHost = System.getProperty("spark.driver.host") + val driverPort = System.getProperty("spark.driver.port") try { - val socket = new Socket(masterHost, masterPort.toInt) + val socket = new Socket(driverHost, driverPort.toInt) socket.close() - logInfo("Master now available: " + masterHost + ":" + masterPort) - masterUp = true + logInfo("Master now available: " + driverHost + ":" + driverPort) + driverUp = true } catch { case e: Exception => - logError("Failed to connect to master at " + masterHost + ":" + masterPort) + logError("Failed to connect to driver at " + driverHost + ":" + driverPort) Thread.sleep(100) } } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala index cac9dab401aa7..61dd72a6518ca 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala @@ -191,8 +191,8 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM else { // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString - val masterUrl = "akka://spark@%s:%s/user/%s".format( - System.getProperty("spark.master.host"), System.getProperty("spark.master.port"), + val driverUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + workerHostname) @@ -209,7 +209,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM } new Thread( - new WorkerRunnable(container, conf, masterUrl, workerId, + new WorkerRunnable(container, conf, driverUrl, workerId, workerHostname, workerMemory, workerCores) ).start() } From a402b23bcd9a9470c5fa38bf46f150b51d43eac9 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 17 Apr 2013 05:52:00 +0530 Subject: [PATCH 463/696] Fudge order of classpath - so that our jars take precedence over what is in CLASSPATH variable. Sounds logical, hope there is no issue cos of it --- core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala | 2 +- .../hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index 7fa67405798cd..c007dae98c0d1 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -165,8 +165,8 @@ class Client(conf: Configuration, args: ClientArguments) extends Logging { // If log4j present, ensure ours overrides all others if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") - Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*") + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") Client.populateHadoopClasspath(yarnConf, env) SparkHadoopUtil.setYarnMode(env) env("SPARK_YARN_JAR_PATH") = diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index 5688f1ab66603..a2bf0af762417 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -153,8 +153,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*") + Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") Client.populateHadoopClasspath(yarnConf, env) System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } From 02dffd2eb0f5961a0e0ad93a136a086c36670b76 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 17 Apr 2013 05:52:57 +0530 Subject: [PATCH 464/696] Ensure all ask/await block for spark.akka.askTimeout - so that it is controllable : instead of arbitrary timeouts spread across codebase. In our tests, we use 30 seconds, though default of 10 is maintained --- core/src/main/scala/spark/deploy/client/Client.scala | 3 ++- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala | 4 ++-- .../spark/scheduler/cluster/StandaloneSchedulerBackend.scala | 1 - core/src/main/scala/spark/storage/BlockManagerMaster.scala | 2 +- core/src/main/scala/spark/storage/BlockManagerUI.scala | 4 ++-- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 072232e33adbb..4af44f9c164c5 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -3,6 +3,7 @@ package spark.deploy.client import spark.deploy._ import akka.actor._ import akka.pattern.ask +import akka.util.Duration import akka.util.duration._ import akka.pattern.AskTimeoutException import spark.{SparkException, Logging} @@ -112,7 +113,7 @@ private[spark] class Client( def stop() { if (actor != null) { try { - val timeout = 5.seconds + val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 54faa375fbd46..a4e21c81308e5 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -3,7 +3,7 @@ package spark.deploy.master import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.Await import akka.pattern.ask -import akka.util.Timeout +import akka.util.{Duration, Timeout} import akka.util.duration._ import cc.spray.Directives import cc.spray.directives._ @@ -22,7 +22,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct val RESOURCE_DIR = "spark/deploy/master/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Timeout(10 seconds) + implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) val handler = { get { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index c834f87d50111..3235c50d1bd31 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -3,7 +3,7 @@ package spark.deploy.worker import akka.actor.{ActorRef, ActorSystem} import akka.dispatch.Await import akka.pattern.ask -import akka.util.Timeout +import akka.util.{Duration, Timeout} import akka.util.duration._ import cc.spray.Directives import cc.spray.typeconversion.TwirlSupport._ @@ -22,7 +22,7 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) val RESOURCE_DIR = "spark/deploy/worker/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Timeout(10 seconds) + implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) val handler = { get { diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index c20276a605796..004592a540438 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -162,7 +162,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor // Called by subclasses when notified of a lost worker def removeExecutor(executorId: String, reason: String) { try { - val timeout = 5.seconds val future = driverActor.ask(RemoveExecutor(executorId, reason))(timeout) Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 036fdc3480119..6fae62d373ba9 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -22,7 +22,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" - val timeout = 10.seconds + val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 9e6721ec17169..07da5720440cd 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -1,7 +1,7 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} -import akka.util.Timeout +import akka.util.Duration import akka.util.duration._ import cc.spray.typeconversion.TwirlSupport._ import cc.spray.Directives @@ -19,7 +19,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Timeout(10 seconds) + implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") /** Start a HTTP server to run the Web interface */ def start() { From 46779b4745dcd9cbfa6f48cd906d4a9c32fa83e2 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 17 Apr 2013 05:53:28 +0530 Subject: [PATCH 465/696] Move back to 2.0.2-alpha, since 2.0.3-alpha is not available in cloudera yet --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c0ba6b9b0ebbc..ecbfaf9b4743e 100644 --- a/pom.xml +++ b/pom.xml @@ -564,7 +564,7 @@ hadoop2-yarn 2 - 2.0.3-alpha + 2.0.2-alpha From 5d891534fd5ca268f6ba7c9a47680846eb3a15ae Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 17 Apr 2013 05:54:43 +0530 Subject: [PATCH 466/696] Move back to 2.0.2-alpha, since 2.0.3-alpha is not available in cloudera yet. Also, add netty dependency explicitly to prevent resolving to older 2.3x version. Additionally, comment out retrievePattern to ensure correct netty is picked up --- project/SparkBuild.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f041930b4ec5d..91e3123bc5eb6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -20,7 +20,7 @@ object SparkBuild extends Build { //val HADOOP_YARN = false // For Hadoop 2 YARN support - val HADOOP_VERSION = "2.0.3-alpha" + val HADOOP_VERSION = "2.0.2-alpha" val HADOOP_MAJOR_VERSION = "2" val HADOOP_YARN = true @@ -47,9 +47,10 @@ object SparkBuild extends Build { scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, - retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", + // retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), - testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), + // For some reason this fails on some nodes and works on others - not yet debugged why + // testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), // shared between both core and streaming. resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), @@ -99,6 +100,7 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( + "io.netty" % "netty" % "3.5.3.Final", "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", "org.scalatest" %% "scalatest" % "1.8" % "test", "org.scalacheck" %% "scalacheck" % "1.9" % "test", @@ -131,11 +133,13 @@ object SparkBuild extends Build { ), libraryDependencies ++= Seq( + "io.netty" % "netty" % "3.5.3.Final", "com.google.guava" % "guava" % "11.0.1", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "com.ning" % "compress-lzf" % "0.8.4", + "commons-daemon" % "commons-daemon" % "1.0.10", "asm" % "asm-all" % "3.3.1", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", From f07961060d8d9dd85ab2a581adc45f886bb0e629 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 17 Apr 2013 23:13:02 +0530 Subject: [PATCH 467/696] Add a small note on spark.tasks.schedule.aggression --- .../scala/spark/scheduler/cluster/ClusterScheduler.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 2e18d46edce4f..a9d9c5e44c72b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -47,6 +47,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) - ANY Note that this property makes more sense when used in conjugation with spark.tasks.revive.interval > 0 : else it is not very effective. + + Additional Note: For non trivial clusters, there is a 4x - 5x reduction in running time (in some of our experiments) based on whether + it is left at default HOST_LOCAL, RACK_LOCAL (if cluster is configured to be rack aware) or ANY. + If cluster is rack aware, then setting it to RACK_LOCAL gives best tradeoff and a 3x - 4x performance improvement while minimizing IO impact. + Also, it brings down the variance in running time drastically. */ val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "HOST_LOCAL")) @@ -68,7 +73,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val activeExecutorIds = new HashSet[String] // TODO: We might want to remove this and merge it with execId datastructures - but later. - // Which hosts in the cluster are alive (contains hostPort's) + // Which hosts in the cluster are alive (contains hostPort's) - used for hyper local and local task locality. private val hostPortsAlive = new HashSet[String] private val hostToAliveHostPorts = new HashMap[String, HashSet[String]] From 5ee2f5c4837f0098282d93c85e606e1a3af40dd6 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 17 Apr 2013 23:13:34 +0530 Subject: [PATCH 468/696] Cache pattern, add (commented out) alternatives for check* apis --- core/src/main/scala/spark/Utils.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 14bb153d54ab3..3e54fa7a7e79a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -13,6 +13,7 @@ import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import spark.serializer.SerializerInstance import spark.deploy.SparkHadoopUtil +import java.util.regex.Pattern /** * Various utility methods used by Spark. @@ -337,9 +338,11 @@ private object Utils extends Logging { } // Used by DEBUG code : remove when all testing done + private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$") def checkHost(host: String, message: String = "") { // Currently catches only ipv4 pattern, this is just a debugging tool - not rigourous ! - if (host.matches("^[0-9]+(\\.[0-9]+)*$")) { + // if (host.matches("^[0-9]+(\\.[0-9]+)*$")) { + if (ipPattern.matcher(host).matches()) { Utils.logErrorWithStack("Unexpected to have host " + host + " which matches IP pattern. Message " + message) } if (Utils.parseHostPort(host)._2 != 0){ @@ -356,6 +359,12 @@ private object Utils extends Logging { } } + // Once testing is complete in various modes, replace with this ? + /* + def checkHost(host: String, message: String = "") {} + def checkHostPort(hostPort: String, message: String = "") {} + */ + def getUserNameFromEnvironment(): String = { SparkHadoopUtil.getUserNameFromEnvironment } From e0603d7e8bfa991dfd5dc43b303c23a47aa70bca Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 18 Apr 2013 13:13:54 +0800 Subject: [PATCH 469/696] refactor the Schedulable interface and add unit test for SchedulingAlgorithm --- core/src/main/scala/spark/SparkContext.scala | 12 +- .../scheduler/cluster/ClusterScheduler.scala | 77 ++++--- .../cluster/FIFOTaskSetQueuesManager.scala | 49 ----- .../cluster/FairTaskSetQueuesManager.scala | 157 ------------- .../scala/spark/scheduler/cluster/Pool.scala | 96 +++++--- .../spark/scheduler/cluster/Schedulable.scala | 23 +- .../cluster/SchedulableBuilder.scala | 115 ++++++++++ .../cluster/SchedulingAlgorithm.scala | 33 +-- .../scheduler/cluster/TaskSetManager.scala | 58 +++-- core/src/test/resources/fairscheduler.xml | 14 ++ .../scheduler/ClusterSchedulerSuite.scala | 207 ++++++++++++++++++ 11 files changed, 525 insertions(+), 316 deletions(-) delete mode 100644 core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala delete mode 100644 core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala create mode 100644 core/src/test/resources/fairscheduler.xml create mode 100644 core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 7c96ae637b3ef..5d9a0357ad480 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -146,9 +146,7 @@ class SparkContext( case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). - newInstance().asInstanceOf[TaskSetQueuesManager] - scheduler.initialize(backend, taskSetQueuesManager) + scheduler.initialize(backend) scheduler case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => @@ -167,9 +165,7 @@ class SparkContext( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). - newInstance().asInstanceOf[TaskSetQueuesManager] - scheduler.initialize(backend, taskSetQueuesManager) + scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() } @@ -188,9 +184,7 @@ class SparkContext( } else { new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName) } - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). - newInstance().asInstanceOf[TaskSetQueuesManager] - scheduler.initialize(backend, taskSetQueuesManager) + scheduler.initialize(backend) scheduler } } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 2ddac0ff30f75..1a300c9e8cbe7 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -61,17 +61,31 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val mapOutputTracker = SparkEnv.get.mapOutputTracker - var taskSetQueuesManager: TaskSetQueuesManager = null + var schedulableBuilder: SchedulableBuilder = null + var rootPool: Pool = null override def setListener(listener: TaskSchedulerListener) { this.listener = listener } - def initialize(context: SchedulerBackend, taskSetQueuesManager: TaskSetQueuesManager) { + def initialize(context: SchedulerBackend) { backend = context - this.taskSetQueuesManager = taskSetQueuesManager + //default scheduler is FIFO + val schedulingMode = System.getProperty("spark.cluster.schedulingmode", "FIFO") + //temporarily set rootPool name to empty + rootPool = new Pool("", SchedulingMode.withName(schedulingMode), 0, 0) + schedulableBuilder = { + schedulingMode match { + case "FIFO" => + new FIFOSchedulableBuilder(rootPool) + case "FAIR" => + new FairSchedulableBuilder(rootPool) + } + } + schedulableBuilder.buildPools() } + def newTaskId(): Long = nextTaskId.getAndIncrement() override def start() { @@ -101,7 +115,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) this.synchronized { val manager = new TaskSetManager(this, taskSet) activeTaskSets(taskSet.id) = manager - taskSetQueuesManager.addTaskSetManager(manager) + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) taskSetTaskIds(taskSet.id) = new HashSet[Long]() if (hasReceivedTask == false) { @@ -124,26 +138,21 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def taskSetFinished(manager: TaskSetManager) { this.synchronized { activeTaskSets -= manager.taskSet.id - taskSetQueuesManager.removeTaskSetManager(manager) + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) taskSetTaskIds.remove(manager.taskSet.id) } } - def taskFinished(manager: TaskSetManager) { - this.synchronized { - taskSetQueuesManager.taskFinished(manager) - } - } - /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task * 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 { + synchronized { SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname for (o <- offers) { @@ -155,27 +164,27 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray - for (i <- 0 until offers.size){ - var launchedTask = true - val execId = offers(i).executorId - val host = offers(i).hostname - while (availableCpus(i) > 0 && launchedTask){ - launchedTask = false - taskSetQueuesManager.receiveOffer(execId,host,availableCpus(i)) match { - case Some(task) => - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = task.taskSetId - taskSetTaskIds(task.taskSetId) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - - case None => {} - } + for (i <- 0 until offers.size) { + var launchedTask = true + val execId = offers(i).executorId + val host = offers(i).hostname + while (availableCpus(i) > 0 && launchedTask) { + launchedTask = false + rootPool.receiveOffer(execId,host,availableCpus(i)) match { + case Some(task) => + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = task.taskSetId + taskSetTaskIds(task.taskSetId) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + + case None => {} } + } } if (tasks.size > 0) { hasLaunchedTask = true @@ -271,7 +280,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def checkSpeculatableTasks() { var shouldRevive = false synchronized { - shouldRevive = taskSetQueuesManager.checkSpeculatableTasks() + shouldRevive = rootPool.checkSpeculatableTasks() } if (shouldRevive) { backend.reviveOffers() @@ -314,6 +323,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHost -= host } executorIdToHost -= executorId - taskSetQueuesManager.removeExecutor(executorId, host) + rootPool.executorLost(executorId, host) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala deleted file mode 100644 index 62d31303414de..0000000000000 --- a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala +++ /dev/null @@ -1,49 +0,0 @@ -package spark.scheduler.cluster - -import scala.collection.mutable.ArrayBuffer - -import spark.Logging - -/** - * A FIFO Implementation of the TaskSetQueuesManager - */ -private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - - var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] - val tasksetSchedulingAlgorithm = new FIFOSchedulingAlgorithm() - - override def addTaskSetManager(manager: TaskSetManager) { - activeTaskSetsQueue += manager - } - - override def removeTaskSetManager(manager: TaskSetManager) { - activeTaskSetsQueue -= manager - } - - override def taskFinished(manager: TaskSetManager) { - //do nothing - } - - override def removeExecutor(executorId: String, host: String) { - activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) - } - - override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = { - - for (manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) { - val task = manager.slaveOffer(execId,host,avaiableCpus) - if (task != None) { - return task - } - } - return None - } - - override def checkSpeculatableTasks(): Boolean = { - var shouldRevive = false - for (ts <- activeTaskSetsQueue) { - shouldRevive |= ts.checkSpeculatableTasks() - } - return shouldRevive - } -} diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala deleted file mode 100644 index 89b74fbb4731d..0000000000000 --- a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala +++ /dev/null @@ -1,157 +0,0 @@ -package spark.scheduler.cluster - -import java.io.{File, FileInputStream, FileOutputStream} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.util.control.Breaks._ -import scala.xml._ - -import spark.Logging -import spark.scheduler.cluster.SchedulingMode.SchedulingMode - -/** - * A Fair Implementation of the TaskSetQueuesManager - * - * Currently we support minShare,weight for fair scheduler between pools - * Within a pool, it supports FIFO or FS - * Also, currently we could allocate pools dynamically - */ -private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - - val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") - val poolNameToPool= new HashMap[String, Pool] - var pools = new ArrayBuffer[Pool] - val poolScheduleAlgorithm = new FairSchedulingAlgorithm() - val POOL_FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" - val POOL_DEFAULT_POOL_NAME = "default" - val POOL_MINIMUM_SHARES_PROPERTY = "minShares" - val POOL_SCHEDULING_MODE_PROPERTY = "schedulingMode" - val POOL_WEIGHT_PROPERTY = "weight" - val POOL_POOL_NAME_PROPERTY = "@name" - val POOL_POOLS_PROPERTY = "pool" - val POOL_DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO - val POOL_DEFAULT_MINIMUM_SHARES = 2 - val POOL_DEFAULT_WEIGHT = 1 - - loadPoolProperties() - - override def addTaskSetManager(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if (manager.taskSet.properties != null) { - poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - if (!poolNameToPool.contains(poolName)) { - //we will create a new pool that user has configured in app instead of being defined in xml file - val pool = new Pool(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) - pools += pool - poolNameToPool(poolName) = pool - logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( - poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } - } - poolNameToPool(poolName).addTaskSetManager(manager) - logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) - } - - override def removeTaskSetManager(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if (manager.taskSet.properties != null) { - poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - } - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id,poolName)) - val pool = poolNameToPool(poolName) - pool.removeTaskSetManager(manager) - pool.runningTasks -= manager.runningTasks - } - - override def taskFinished(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if (manager.taskSet.properties != null) { - poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - } - val pool = poolNameToPool(poolName) - pool.runningTasks -= 1 - manager.runningTasks -=1 - } - - override def removeExecutor(executorId: String, host: String) { - for (pool <- pools) { - pool.removeExecutor(executorId,host) - } - } - - override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = { - val sortedPools = pools.sortWith(poolScheduleAlgorithm.comparator) - for (pool <- sortedPools) { - logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format( - pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) - } - for (pool <- sortedPools) { - val task = pool.receiveOffer(execId,host,avaiableCpus) - if(task != None) { - pool.runningTasks += 1 - return task - } - } - return None - } - - override def checkSpeculatableTasks(): Boolean = { - var shouldRevive = false - for (pool <- pools) { - shouldRevive |= pool.checkSpeculatableTasks() - } - return shouldRevive - } - - def loadPoolProperties() { - //first check if the file exists - val file = new File(schedulerAllocFile) - if (file.exists()) { - val xml = XML.loadFile(file) - for (poolNode <- (xml \\ POOL_POOLS_PROPERTY)) { - - val poolName = (poolNode \ POOL_POOL_NAME_PROPERTY).text - var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE - var minShares = POOL_DEFAULT_MINIMUM_SHARES - var weight = POOL_DEFAULT_WEIGHT - - val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text - if (xmlSchedulingMode != "") { - try{ - schedulingMode = SchedulingMode.withName(xmlSchedulingMode) - } - catch{ - case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") - } - } - - val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text - if (xmlMinShares != "") { - minShares = xmlMinShares.toInt - } - - val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text - if (xmlWeight != "") { - weight = xmlWeight.toInt - } - - val pool = new Pool(poolName,schedulingMode,minShares,weight) - pools += pool - poolNameToPool(poolName) = pool - logInfo("Create new pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( - poolName,schedulingMode,minShares,weight)) - } - } - - if (!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) { - val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) - pools += pool - poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool - logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( - POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } - } - } diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index e0917ca1ca21b..d5482f71add04 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -1,74 +1,106 @@ package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import spark.Logging import spark.scheduler.cluster.SchedulingMode.SchedulingMode + /** - * An Schedulable entity that represent collection of TaskSetManager + * An Schedulable entity that represent collection of Pools or TaskSetManagers */ + private[spark] class Pool( val poolName: String, val schedulingMode: SchedulingMode, - initMinShare:Int, - initWeight:Int) + initMinShare: Int, + initWeight: Int) extends Schedulable with Logging { - var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + var schedulableQueue = new ArrayBuffer[Schedulable] + var schedulableNameToSchedulable = new HashMap[String, Schedulable] var weight = initWeight var minShare = initMinShare var runningTasks = 0 - val priority = 0 - val stageId = 0 + var priority = 0 + var stageId = 0 + var name = poolName + var parent:Schedulable = null var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { - schedulingMode match { + schedulingMode match { case SchedulingMode.FAIR => - val schedule = new FairSchedulingAlgorithm() - schedule + new FairSchedulingAlgorithm() case SchedulingMode.FIFO => - val schedule = new FIFOSchedulingAlgorithm() - schedule + new FIFOSchedulingAlgorithm() } } - def addTaskSetManager(manager:TaskSetManager) { - activeTaskSetsQueue += manager + override def addSchedulable(schedulable: Schedulable) { + schedulableQueue += schedulable + schedulableNameToSchedulable(schedulable.name) = schedulable + schedulable.parent= this } - def removeTaskSetManager(manager:TaskSetManager) { - activeTaskSetsQueue -= manager + override def removeSchedulable(schedulable: Schedulable) { + schedulableQueue -= schedulable + schedulableNameToSchedulable -= schedulable.name } - def removeExecutor(executorId: String, host: String) { - activeTaskSetsQueue.foreach(_.executorLost(executorId,host)) + override def getSchedulableByName(schedulableName: String): Schedulable = { + if (schedulableNameToSchedulable.contains(schedulableName)) { + return schedulableNameToSchedulable(schedulableName) + } + for (schedulable <- schedulableQueue) { + var sched = schedulable.getSchedulableByName(schedulableName) + if (sched != null) { + return sched + } + } + return null } - def checkSpeculatableTasks(): Boolean = { + override def executorLost(executorId: String, host: String) { + schedulableQueue.foreach(_.executorLost(executorId, host)) + } + + override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for (ts <- activeTaskSetsQueue) { - shouldRevive |= ts.checkSpeculatableTasks() + for (schedulable <- schedulableQueue) { + shouldRevive |= schedulable.checkSpeculatableTasks() } return shouldRevive } - def receiveOffer(execId:String,host:String,availableCpus:Double):Option[TaskDescription] = { - val sortedActiveTasksSetQueue = activeTaskSetsQueue.sortWith(taskSetSchedulingAlgorithm.comparator) - for (manager <- sortedActiveTasksSetQueue) { - logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format( - poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) + override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) + for (manager <- sortedSchedulableQueue) { + logInfo("parentName:%s,schedulableName:%s,minShares:%d,weight:%d,runningTasks:%d".format( + manager.parent.name, manager.name, manager.minShare, manager.weight, manager.runningTasks)) } - - for (manager <- sortedActiveTasksSetQueue) { - val task = manager.slaveOffer(execId,host,availableCpus) - if (task != None) { - manager.runningTasks += 1 - return task - } + for (manager <- sortedSchedulableQueue) { + val task = manager.receiveOffer(execId, host, availableCpus) + if (task != None) { + return task + } } return None } + + override def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + override def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 8dfc369c035be..54e8ae95f9f37 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,13 +1,26 @@ package spark.scheduler.cluster +import scala.collection.mutable.ArrayBuffer + /** * An interface for schedulable entities. * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { - def weight:Int - def minShare:Int - def runningTasks:Int - def priority:Int - def stageId:Int + var parent: Schedulable + def weight: Int + def minShare: Int + def runningTasks: Int + def priority: Int + def stageId: Int + def name: String + + def increaseRunningTasks(taskNum: Int): Unit + def decreaseRunningTasks(taskNum: Int): Unit + def addSchedulable(schedulable: Schedulable): Unit + def removeSchedulable(schedulable: Schedulable): Unit + def getSchedulableByName(name: String): Schedulable + def executorLost(executorId: String, host: String): Unit + def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] + def checkSpeculatableTasks(): Boolean } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala new file mode 100644 index 0000000000000..47a426a45b8bc --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -0,0 +1,115 @@ +package spark.scheduler.cluster + +import java.io.{File, FileInputStream, FileOutputStream} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.util.control.Breaks._ +import scala.xml._ + +import spark.Logging +import spark.scheduler.cluster.SchedulingMode.SchedulingMode + +import java.util.Properties + +/** + * An interface to build Schedulable tree + * buildPools: build the tree nodes(pools) + * addTaskSetManager: build the leaf nodes(TaskSetManagers) + */ +private[spark] trait SchedulableBuilder { + def buildPools() + def addTaskSetManager(manager: Schedulable, properties: Properties) +} + +private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { + + override def buildPools() { + //nothing + } + + override def addTaskSetManager(manager: Schedulable, properties: Properties) { + rootPool.addSchedulable(manager) + } +} + +private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { + + val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") + val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" + val DEFAULT_POOL_NAME = "default" + val MINIMUM_SHARES_PROPERTY = "minShare" + val SCHEDULING_MODE_PROPERTY = "schedulingMode" + val WEIGHT_PROPERTY = "weight" + val POOL_NAME_PROPERTY = "@name" + val POOLS_PROPERTY = "pool" + val DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO + val DEFAULT_MINIMUM_SHARE = 2 + val DEFAULT_WEIGHT = 1 + + override def buildPools() { + val file = new File(schedulerAllocFile) + if (file.exists()) { + val xml = XML.loadFile(file) + for (poolNode <- (xml \\ POOLS_PROPERTY)) { + + val poolName = (poolNode \ POOL_NAME_PROPERTY).text + var schedulingMode = DEFAULT_SCHEDULING_MODE + var minShare = DEFAULT_MINIMUM_SHARE + var weight = DEFAULT_WEIGHT + + val xmlSchedulingMode = (poolNode \ SCHEDULING_MODE_PROPERTY).text + if (xmlSchedulingMode != "") { + try { + schedulingMode = SchedulingMode.withName(xmlSchedulingMode) + } catch { + case e: Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + } + } + + val xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text + if (xmlMinShare != "") { + minShare = xmlMinShare.toInt + } + + val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text + if (xmlWeight != "") { + weight = xmlWeight.toInt + } + + val pool = new Pool(poolName, schedulingMode, minShare, weight) + rootPool.addSchedulable(pool) + logInfo("Create new pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + poolName, schedulingMode, minShare, weight)) + } + } + + //finally create "default" pool + if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { + val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + rootPool.addSchedulable(pool) + logInfo("Create default pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + } +} + + override def addTaskSetManager(manager: Schedulable, properties: Properties) { + var poolName = DEFAULT_POOL_NAME + var parentPool = rootPool.getSchedulableByName(poolName) + if (properties != null) { + poolName = properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) + parentPool = rootPool.getSchedulableByName(poolName) + if (parentPool == null) { + //we will create a new pool that user has configured in app instead of being defined in xml file + parentPool = new Pool(poolName,DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + rootPool.addSchedulable(parentPool) + logInfo("Create pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + poolName, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + } + } + parentPool.addSchedulable(manager) + logInfo("Added task set " + manager.name + " tasks to pool "+poolName) + } +} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index ac2237a7ef1ff..a5d6285c993c4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -2,11 +2,11 @@ package spark.scheduler.cluster /** * An interface for sort algorithm - * FIFO: FIFO algorithm for TaskSetManagers - * FS: FS algorithm for Pools, and FIFO or FS for TaskSetManagers + * FIFO: FIFO algorithm between TaskSetManagers + * FS: FS algorithm between Pools, and FIFO or FS within Pools */ private[spark] trait SchedulingAlgorithm { - def comparator(s1: Schedulable,s2: Schedulable): Boolean + def comparator(s1: Schedulable, s2: Schedulable): Boolean } private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { @@ -15,40 +15,41 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { val priority2 = s2.priority var res = Math.signum(priority1 - priority2) if (res == 0) { - val stageId1 = s1.stageId - val stageId2 = s2.stageId - res = Math.signum(stageId1 - stageId2) + val stageId1 = s1.stageId + val stageId2 = s2.stageId + res = Math.signum(stageId1 - stageId2) } - if (res < 0) + if (res < 0) { return true - else + } else { return false + } } } private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { - def comparator(s1: Schedulable, s2:Schedulable): Boolean = { + override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { val minShare1 = s1.minShare val minShare2 = s2.minShare val runningTasks1 = s1.runningTasks val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1,1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2,1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1, 1.0).toDouble + val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2, 1.0).toDouble val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true - if (s1Needy && !s2Needy) + if (s1Needy && !s2Needy) { res = true - else if(!s1Needy && s2Needy) + } else if (!s1Needy && s2Needy) { res = false - else if (s1Needy && s2Needy) + } else if (s1Needy && s2Needy) { res = minShareRatio1 <= minShareRatio2 - else + } else { res = taskToWeightRatio1 <= taskToWeightRatio2 - + } return res } } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 7ec2f69da559a..baaaa41a37f6b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -32,8 +32,6 @@ private[spark] class TaskSetManager( // Maximum times a task is allowed to fail before failing the job val MAX_TASK_FAILURES = 4 - val TASKSET_MINIMUM_SHARES = 1 - val TASKSET_WEIGHT = 1 // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble @@ -41,12 +39,6 @@ private[spark] class TaskSetManager( // Serializer for closures and tasks. val ser = SparkEnv.get.closureSerializer.newInstance() - var weight = TASKSET_WEIGHT - var minShare = TASKSET_MINIMUM_SHARES - var runningTasks = 0 - val priority = taskSet.priority - val stageId = taskSet.stageId - val tasks = taskSet.tasks val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) @@ -55,6 +47,14 @@ private[spark] class TaskSetManager( val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 + var weight = 1 + var minShare = 0 + var runningTasks = 0 + var priority = taskSet.priority + var stageId = taskSet.stageId + var name = "TaskSet_"+taskSet.stageId.toString + var parent:Schedulable = null + // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -198,7 +198,7 @@ private[spark] class TaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -230,10 +230,11 @@ private[spark] class TaskSetManager( val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) val timeTaken = System.currentTimeMillis - startTime + increaseRunningTasks(1) logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId,taskSet.id,execId, taskName, serializedTask)) + return Some(new TaskDescription(taskId, taskSet.id, execId, taskName, serializedTask)) } case _ => } @@ -264,7 +265,7 @@ private[spark] class TaskSetManager( } val index = info.index info.markSuccessful() - sched.taskFinished(this) + decreaseRunningTasks(1) if (!finished(index)) { tasksFinished += 1 logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( @@ -293,7 +294,7 @@ private[spark] class TaskSetManager( } val index = info.index info.markFailed() - sched.taskFinished(this) + decreaseRunningTasks(1) if (!finished(index)) { logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) copiesRunning(index) -= 1 @@ -308,6 +309,7 @@ private[spark] class TaskSetManager( finished(index) = true tasksFinished += 1 sched.taskSetFinished(this) + decreaseRunningTasks(runningTasks) return case ef: ExceptionFailure => @@ -365,10 +367,38 @@ private[spark] class TaskSetManager( causeOfFailure = message // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.listener.taskSetFailed(taskSet, message) + decreaseRunningTasks(runningTasks) sched.taskSetFinished(this) } - def executorLost(execId: String, hostname: String) { + override def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + override def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + override def getSchedulableByName(name: String): Schedulable = { + return null + } + + override def addSchedulable(schedulable:Schedulable) { + //nothing + } + + override def removeSchedulable(schedulable:Schedulable) { + //nothing + } + + override def executorLost(execId: String, hostname: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) val newHostsAlive = sched.hostsAlive // If some task has preferred locations only on hostname, and there are no more executors there, @@ -409,7 +439,7 @@ private[spark] class TaskSetManager( * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that * we don't scan the whole task set. It might also help to make this sorted by launch time. */ - def checkSpeculatableTasks(): Boolean = { + override def checkSpeculatableTasks(): Boolean = { // Can't speculate if we only have one task, or if all tasks have finished. if (numTasks == 1 || tasksFinished == numTasks) { return false diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml new file mode 100644 index 0000000000000..5a688b0ebb973 --- /dev/null +++ b/core/src/test/resources/fairscheduler.xml @@ -0,0 +1,14 @@ + + + 2 + 1 + FIFO + + + 3 + 1 + FIFO + + + + diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala new file mode 100644 index 0000000000000..2eda48196be74 --- /dev/null +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -0,0 +1,207 @@ +package spark.scheduler + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter + +import spark._ +import spark.scheduler._ +import spark.scheduler.cluster._ + +import java.util.Properties + +class DummyTaskSetManager( + initPriority: Int, + initStageId: Int, + initNumTasks: Int) + extends Schedulable { + + var parent: Schedulable = null + var weight = 1 + var minShare = 2 + var runningTasks = 0 + var priority = initPriority + var stageId = initStageId + var name = "TaskSet_"+stageId + var numTasks = initNumTasks + var tasksFinished = 0 + + def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + def addSchedulable(schedulable: Schedulable) { + } + + def removeSchedulable(schedulable: Schedulable) { + } + + def getSchedulableByName(name: String): Schedulable = { + return null + } + + def executorLost(executorId: String, host: String): Unit = { + } + + def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] = { + if (tasksFinished + runningTasks < numTasks) { + increaseRunningTasks(1) + return Some(new TaskDescription(0, stageId.toString, execId, "task 0:0", null)) + } + return None + } + + def checkSpeculatableTasks(): Boolean = { + return true + } + + def taskFinished() { + decreaseRunningTasks(1) + tasksFinished +=1 + if (tasksFinished == numTasks) { + parent.removeSchedulable(this) + } + } + + def abort() { + decreaseRunningTasks(runningTasks) + parent.removeSchedulable(this) + } +} + +class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { + + def receiveOffer(rootPool: Pool) : Option[TaskDescription] = { + rootPool.receiveOffer("execId_1", "hostname_1", 1) + } + + def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { + receiveOffer(rootPool) match { + case Some(task) => + assert(task.taskSetId.toInt === expectedTaskSetId) + case _ => + } + } + + test("FIFO Scheduler Test") { + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) + val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + val taskSetManager0 = new DummyTaskSetManager(0, 0, 2) + val taskSetManager1 = new DummyTaskSetManager(0, 1, 2) + val taskSetManager2 = new DummyTaskSetManager(0, 2, 2) + schedulableBuilder.addTaskSetManager(taskSetManager0, null) + schedulableBuilder.addTaskSetManager(taskSetManager1, null) + schedulableBuilder.addTaskSetManager(taskSetManager2, null) + + checkTaskSetId(rootPool, 0) + receiveOffer(rootPool) + checkTaskSetId(rootPool, 1) + receiveOffer(rootPool) + taskSetManager1.abort() + checkTaskSetId(rootPool, 2) + } + + test("Fair Scheduler Test") { + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + System.setProperty("spark.fairscheduler.allocation.file", xmlPath) + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + assert(rootPool.getSchedulableByName("default") != null) + assert(rootPool.getSchedulableByName("1") != null) + assert(rootPool.getSchedulableByName("2") != null) + assert(rootPool.getSchedulableByName("3") != null) + assert(rootPool.getSchedulableByName("1").minShare === 2) + assert(rootPool.getSchedulableByName("1").weight === 1) + assert(rootPool.getSchedulableByName("2").minShare === 3) + assert(rootPool.getSchedulableByName("2").weight === 1) + assert(rootPool.getSchedulableByName("3").minShare === 2) + assert(rootPool.getSchedulableByName("3").weight === 1) + + val properties1 = new Properties() + properties1.setProperty("spark.scheduler.cluster.fair.pool","1") + val properties2 = new Properties() + properties2.setProperty("spark.scheduler.cluster.fair.pool","2") + + val taskSetManager10 = new DummyTaskSetManager(1, 0, 1) + val taskSetManager11 = new DummyTaskSetManager(1, 1, 1) + val taskSetManager12 = new DummyTaskSetManager(1, 2, 2) + schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) + + val taskSetManager23 = new DummyTaskSetManager(2, 3, 2) + val taskSetManager24 = new DummyTaskSetManager(2, 4, 2) + schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) + schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 1) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 4) + + taskSetManager12.taskFinished() + assert(rootPool.getSchedulableByName("1").runningTasks === 3) + taskSetManager24.abort() + assert(rootPool.getSchedulableByName("2").runningTasks === 2) + } + + test("Nested Pool Test") { + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) + val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) + rootPool.addSchedulable(pool0) + rootPool.addSchedulable(pool1) + + val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) + val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) + pool0.addSchedulable(pool00) + pool0.addSchedulable(pool01) + + val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) + val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) + pool1.addSchedulable(pool10) + pool1.addSchedulable(pool11) + + val taskSetManager000 = new DummyTaskSetManager(0, 0, 5) + val taskSetManager001 = new DummyTaskSetManager(0, 1, 5) + pool00.addSchedulable(taskSetManager000) + pool00.addSchedulable(taskSetManager001) + + val taskSetManager010 = new DummyTaskSetManager(1, 2, 5) + val taskSetManager011 = new DummyTaskSetManager(1, 3, 5) + pool01.addSchedulable(taskSetManager010) + pool01.addSchedulable(taskSetManager011) + + val taskSetManager100 = new DummyTaskSetManager(2, 4, 5) + val taskSetManager101 = new DummyTaskSetManager(2, 5, 5) + pool10.addSchedulable(taskSetManager100) + pool10.addSchedulable(taskSetManager101) + + val taskSetManager110 = new DummyTaskSetManager(3, 6, 5) + val taskSetManager111 = new DummyTaskSetManager(3, 7, 5) + pool11.addSchedulable(taskSetManager110) + pool11.addSchedulable(taskSetManager111) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 6) + checkTaskSetId(rootPool, 2) + } +} From 8436bd5d4a96480ac1871330a28d9d712e64959d Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 19 Apr 2013 02:17:22 +0800 Subject: [PATCH 470/696] remove TaskSetQueueManager and update code style --- core/src/main/scala/spark/SparkContext.scala | 10 ++-------- .../spark/scheduler/DAGSchedulerEvent.scala | 1 - .../scheduler/cluster/SchedulableBuilder.scala | 14 +++++++------- .../scheduler/cluster/TaskSetQueuesManager.scala | 16 ---------------- 4 files changed, 9 insertions(+), 32 deletions(-) delete mode 100644 core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 5d9a0357ad480..eef25ef588386 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -39,7 +39,7 @@ import spark.partial.PartialResult import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import spark.scheduler._ import spark.scheduler.local.LocalScheduler -import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler, TaskSetQueuesManager} +import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.BlockManagerUI import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -75,11 +75,6 @@ class SparkContext( System.setProperty("spark.driver.port", "0") } - //Set the default task scheduler - if (System.getProperty("spark.cluster.taskscheduler") == null) { - System.setProperty("spark.cluster.taskscheduler", "spark.scheduler.cluster.FIFOTaskSetQueuesManager") - } - private val isLocal = (master == "local" || master.startsWith("local[")) // Create the Spark execution environment (cache, map output tracker, etc) @@ -599,8 +594,7 @@ class SparkContext( val callSite = Utils.getSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler - ,localProperties.value) + val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler, localProperties.value) logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() result diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 11fec568c6077..303c211e2a172 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,6 +1,5 @@ package spark.scheduler - import java.util.Properties import spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index 47a426a45b8bc..18cc15c2a5ee9 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -86,14 +86,14 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula } } - //finally create "default" pool - if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { - val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) - rootPool.addSchedulable(pool) - logInfo("Create default pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( - DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + //finally create "default" pool + if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { + val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + rootPool.addSchedulable(pool) + logInfo("Create default pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + } } -} override def addTaskSetManager(manager: Schedulable, properties: Properties) { var poolName = DEFAULT_POOL_NAME diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala deleted file mode 100644 index 86971d47e6147..0000000000000 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala +++ /dev/null @@ -1,16 +0,0 @@ -package spark.scheduler.cluster - -import scala.collection.mutable.ArrayBuffer - -/** - * An interface for managing TaskSet queue/s that allows plugging different policy for - * offering tasks to resources - */ -private[spark] trait TaskSetQueuesManager { - def addTaskSetManager(manager: TaskSetManager): Unit - def removeTaskSetManager(manager: TaskSetManager): Unit - def taskFinished(manager: TaskSetManager): Unit - def removeExecutor(executorId: String, host: String): Unit - def receiveOffer(execId: String, host:String, avaiableCpus:Double):Option[TaskDescription] - def checkSpeculatableTasks(): Boolean -} From ac2e8e8720f10efd640a67ad85270719ab2d43e9 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Fri, 19 Apr 2013 00:13:19 +0530 Subject: [PATCH 471/696] Add some basic documentation --- .../spark/deploy/yarn/ClientArguments.scala | 6 ++-- docs/running-on-yarn.md | 31 +++++++++++++------ 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 53b305f7df0e7..2e69fe3fb0566 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -94,9 +94,11 @@ class ClientArguments(val args: Array[String]) { " Mutliple invocations are possible, each will be passed in order.\n" + " Note that first argument will ALWAYS be yarn-standalone : will be added if missing.\n" + " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + - " --user USERNAME Run the ApplicationMaster as a different user\n" + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + + " --user USERNAME Run the ApplicationMaster (and slaves) as a different user\n" ) System.exit(exitCode) } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c2957e6cb42eb..26424bbe52a9b 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -5,18 +5,25 @@ title: Launching Spark on YARN Experimental support for running over a [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html) -cluster was added to Spark in version 0.6.0. Because YARN depends on version -2.0 of the Hadoop libraries, this currently requires checking out a separate -branch of Spark, called `yarn`, which you can do as follows: +cluster was added to Spark in version 0.6.0. This was merged into master as part of 0.7 effort. +To build spark core with YARN support, please use the hadoop2-yarn profile. +Ex: mvn -Phadoop2-yarn clean install - git clone git://github.com/mesos/spark - cd spark - git checkout -b yarn --track origin/yarn +# Building spark core consolidated jar. + +Currently, only sbt can buid a consolidated jar which contains the entire spark code - which is required for launching jars on yarn. +To do this via sbt - though (right now) is a manual process of enabling it in project/SparkBuild.scala. +Please comment out the + HADOOP_VERSION, HADOOP_MAJOR_VERSION and HADOOP_YARN +variables before the line 'For Hadoop 2 YARN support' +Next, uncomment the subsequent 3 variable declaration lines (for these three variables) which enable hadoop yarn support. + +Currnetly, it is a TODO to add support for maven assembly. # Preparations -- In order to distribute Spark within the cluster, it must be packaged into a single JAR file. This can be done by running `sbt/sbt assembly` +- Building spark core assembled jar (see above). - Your application code must be packaged into a separate JAR file. If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt package`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. @@ -30,8 +37,11 @@ The command to launch the YARN Client is as follows: --class \ --args \ --num-workers \ + --master-memory \ --worker-memory \ - --worker-cores + --worker-cores \ + --user \ + --queue For example: @@ -40,8 +50,9 @@ For example: --class spark.examples.SparkPi \ --args standalone \ --num-workers 3 \ + --master-memory 4g \ --worker-memory 2g \ - --worker-cores 2 + --worker-cores 1 The above starts a YARN Client programs which periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running. @@ -49,3 +60,5 @@ The above starts a YARN Client programs which periodically polls the Application - When your application instantiates a Spark context it must use a special "standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "standalone" as an argument to your program, as shown in the example above. - YARN does not support requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. +- Currently, we have not yet integrated with hadoop security. If --user is present, the hadoop_user specified will be used to run the tasks on the cluster. If unspecified, current user will be used (which should be valid in cluster). + Once hadoop security support is added, and if hadoop cluster is enabled with security, additional restrictions would apply via delegation tokens passed. From 17e076de800ea0d4c55f2bd657348641f6f9c55b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 18 Apr 2013 22:25:21 -0700 Subject: [PATCH 472/696] Turn on forking in test JVMs to reduce the pressure on perm gen and code cache sizes due to having 2 instances of the Scala compiler and a bunch of classloaders. --- project/SparkBuild.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 91541a34f9605..f0b371b2cf682 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,6 +44,10 @@ object SparkBuild extends Build { transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), + // Fork new JVMs for tests and set Java options for those + fork := true, + javaOptions += "-Xmx1g", + // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), From b2a3f24dde7a69587a5fea50d3e1e4e8f02a2dc3 Mon Sep 17 00:00:00 2001 From: koeninger Date: Sun, 21 Apr 2013 00:29:37 -0500 Subject: [PATCH 473/696] first attempt at an RDD to pull data from JDBC sources --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 79 +++++++++++++++++++++ 1 file changed, 79 insertions(+) create mode 100644 core/src/main/scala/spark/rdd/JdbcRDD.scala diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala new file mode 100644 index 0000000000000..c8a5d760127a3 --- /dev/null +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -0,0 +1,79 @@ +package spark.rdd + +import java.sql.{Connection, ResultSet} + +import spark.{Logging, Partition, RDD, SparkContext, TaskContext} +import spark.util.NextIterator + +/** + An RDD that executes an SQL query on a JDBC connection and reads results. + @param getConnection a function that returns an open Connection. + The RDD takes care of closing the connection. + @param sql the text of the query. + The query must contain two ? placeholders for parameters used to partition the results. + E.g. "select title, author from books where ? <= id and id <= ?" + @param lowerBound the minimum value of the first placeholder + @param upperBound the maximum value of the second placeholder + The lower and upper bounds are inclusive. + @param numPartitions the amount of parallelism. + Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + the query would be executed twice, once with (1, 10) and once with (11, 20) + @param mapRow a function from a ResultSet to a single row of the desired result type(s). + This should only call getInt, getString, etc; the RDD takes care of calling next. + The default maps a ResultSet to an array of Object. +*/ +class JdbcRDD[T: ClassManifest]( + sc: SparkContext, + getConnection: () => Connection, + sql: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int, + mapRow: (ResultSet) => T = JdbcRDD.resultSetToObjectArray) + extends RDD[T](sc, Nil) with Logging { + + override def getPartitions: Array[Partition] = + ParallelCollectionRDD.slice(lowerBound to upperBound, numPartitions). + filter(! _.isEmpty). + zipWithIndex. + map(x => new JdbcPartition(x._2, x._1.head, x._1.last)). + toArray + + override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { + val part = thePart.asInstanceOf[JdbcPartition] + val conn = getConnection() + context.addOnCompleteCallback{ () => closeIfNeeded() } + val stmt = conn.prepareStatement(sql) + stmt.setLong(1, part.lower) + stmt.setLong(2, part.upper) + val rs = stmt.executeQuery() + + override def getNext: T = { + if (rs.next()) { + mapRow(rs) + } else { + finished = true + null.asInstanceOf[T] + } + } + + override def close() { + try { + logInfo("closing connection") + conn.close() + } catch { + case e: Exception => logWarning("Exception closing connection", e) + } + } + } + +} + +private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { + override def index = idx +} + +object JdbcRDD { + val resultSetToObjectArray = (rs: ResultSet) => + Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) +} From 7acab3ab45df421601ee9a076a61de00561a0308 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Mon, 22 Apr 2013 08:01:13 +0530 Subject: [PATCH 474/696] Fix review comments, add a new api to SparkHadoopUtil to create appropriate Configuration. Modify an example to show how to use SplitInfo --- .../scala/spark/deploy/SparkHadoopUtil.scala | 5 +++++ .../scala/spark/deploy/SparkHadoopUtil.scala | 6 +++++- .../scala/spark/deploy/SparkHadoopUtil.scala | 5 +++++ core/src/main/scala/spark/SparkContext.scala | 14 +++++++++----- core/src/main/scala/spark/Utils.scala | 8 +++----- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 7 ++++--- .../main/scala/spark/examples/SparkHdfsLR.scala | 10 ++++++++-- project/SparkBuild.scala | 10 ++++++---- 8 files changed, 45 insertions(+), 20 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index d4badbc5c4254..a0fb4fe25d188 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,4 +1,6 @@ package spark.deploy +import org.apache.hadoop.conf.Configuration + /** * Contains util methods to interact with Hadoop from spark. @@ -15,4 +17,7 @@ object SparkHadoopUtil { // Add support, if exists - for now, simply run func ! func(args) } + + // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems + def newConfiguration(): Configuration = new Configuration() } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index 66e5ad8491064..ab1ab9d8a7a8e 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -12,7 +12,7 @@ import java.security.PrivilegedExceptionAction */ object SparkHadoopUtil { - val yarnConf = new YarnConfiguration(new Configuration()) + val yarnConf = newConfiguration() def getUserNameFromEnvironment(): String = { // defaulting to env if -D is not present ... @@ -56,4 +56,8 @@ object SparkHadoopUtil { def setYarnMode(env: HashMap[String, String]) { env("SPARK_YARN_MODE") = "true" } + + // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems + // Always create a new config, dont reuse yarnConf. + def newConfiguration(): Configuration = new YarnConfiguration(new Configuration()) } diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index d4badbc5c4254..a0fb4fe25d188 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,4 +1,6 @@ package spark.deploy +import org.apache.hadoop.conf.Configuration + /** * Contains util methods to interact with Hadoop from spark. @@ -15,4 +17,7 @@ object SparkHadoopUtil { // Add support, if exists - for now, simply run func ! func(args) } + + // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems + def newConfiguration(): Configuration = new Configuration() } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index e853bce2c4254..5f5ec0b0f4d29 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} import org.apache.mesos.MesosNativeLibrary -import spark.deploy.LocalSparkCluster +import spark.deploy.{SparkHadoopUtil, LocalSparkCluster} import spark.partial.ApproximateEvaluator import spark.partial.PartialResult import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} @@ -102,7 +102,9 @@ class SparkContext( // Add each JAR given through the constructor - if (jars != null) jars.foreach { addJar(_) } + if (jars != null) { + jars.foreach { addJar(_) } + } // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() @@ -114,7 +116,9 @@ class SparkContext( executorEnvs(key) = value } } - if (environment != null) executorEnvs ++= environment + if (environment != null) { + executorEnvs ++= environment + } // Create and start the scheduler private var taskScheduler: TaskScheduler = { @@ -207,7 +211,7 @@ class SparkContext( /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { - val conf = new Configuration() + val conf = SparkHadoopUtil.newConfiguration() // Explicitly check for S3 environment variables if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) { conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) @@ -711,7 +715,7 @@ class SparkContext( */ def setCheckpointDir(dir: String, useExisting: Boolean = false) { val path = new Path(dir) - val fs = path.getFileSystem(new Configuration()) + val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration()) if (!useExisting) { if (fs.exists(path)) { throw new Exception("Checkpoint directory '" + path + "' already exists.") diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 3e54fa7a7e79a..9f48cbe490a27 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -4,7 +4,6 @@ import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ @@ -208,7 +207,7 @@ private object Utils extends Logging { case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val uri = new URI(url) - val conf = new Configuration() + val conf = SparkHadoopUtil.newConfiguration() val fs = FileSystem.get(uri, conf) val in = fs.open(new Path(uri)) val out = new FileOutputStream(tempFile) @@ -317,7 +316,6 @@ private object Utils extends Logging { * Get the local machine's hostname. */ def localHostName(): String = { - // customHostname.getOrElse(InetAddress.getLocalHost.getHostName) customHostname.getOrElse(localIpAddressHostname) } @@ -337,6 +335,7 @@ private object Utils extends Logging { retval } + /* // Used by DEBUG code : remove when all testing done private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$") def checkHost(host: String, message: String = "") { @@ -358,12 +357,11 @@ private object Utils extends Logging { Utils.logErrorWithStack("Unexpected to have port " + port + " which is not valid in " + hostPort + ". Message " + message) } } + */ // Once testing is complete in various modes, replace with this ? - /* def checkHost(host: String, message: String = "") {} def checkHostPort(hostPort: String, message: String = "") {} - */ def getUserNameFromEnvironment(): String = { SparkHadoopUtil.getUserNameFromEnvironment diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 24d527f38fa02..79d00edee78ec 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -8,6 +8,7 @@ import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.fs.Path import java.io.{File, IOException, EOFException} import java.text.NumberFormat +import spark.deploy.SparkHadoopUtil private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} @@ -65,7 +66,7 @@ private[spark] object CheckpointRDD extends Logging { def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) { val outputDir = new Path(path) - val fs = outputDir.getFileSystem(new Configuration()) + val fs = outputDir.getFileSystem(SparkHadoopUtil.newConfiguration()) val finalOutputName = splitIdToFile(ctx.splitId) val finalOutputPath = new Path(outputDir, finalOutputName) @@ -103,7 +104,7 @@ private[spark] object CheckpointRDD extends Logging { } def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { - val fs = path.getFileSystem(new Configuration()) + val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration()) val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = SparkEnv.get.serializer.newInstance() @@ -125,7 +126,7 @@ private[spark] object CheckpointRDD extends Logging { val sc = new SparkContext(cluster, "CheckpointRDD Test") val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000) val path = new Path(hdfsPath, "temp") - val fs = path.getFileSystem(new Configuration()) + val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration()) sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala index 0f42f405a058c..3d080a02577a3 100644 --- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala @@ -4,6 +4,8 @@ import java.util.Random import scala.math.exp import spark.util.Vector import spark._ +import spark.deploy.SparkHadoopUtil +import spark.scheduler.InputFormatInfo /** * Logistic regression based classification. @@ -32,9 +34,13 @@ object SparkHdfsLR { System.err.println("Usage: SparkHdfsLR ") System.exit(1) } + val inputPath = args(1) + val conf = SparkHadoopUtil.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val lines = sc.textFile(args(1)) + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), + InputFormatInfo.computePreferredLocations( + Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)))) + val lines = sc.textFile(inputPath) val points = lines.map(parsePoint _).cache() val ITERATIONS = args(2).toInt diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 91e3123bc5eb6..0a5b89d92704f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -47,10 +47,8 @@ object SparkBuild extends Build { scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, - // retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), - // For some reason this fails on some nodes and works on others - not yet debugged why - // testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), + testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), // shared between both core and streaming. resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), @@ -170,7 +168,11 @@ object SparkBuild extends Build { Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION) }), unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / - ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") "src/hadoop2-yarn/scala" else "src/hadoop" + HADOOP_MAJOR_VERSION + "/scala" ) + ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") { + "src/hadoop2-yarn/scala" + } else { + "src/hadoop" + HADOOP_MAJOR_VERSION + "/scala" + } ) } ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings From dfac0aa5c2e5f46955b008b1e8d9ee5d8069efa5 Mon Sep 17 00:00:00 2001 From: koeninger Date: Mon, 22 Apr 2013 21:12:52 -0500 Subject: [PATCH 475/696] prevent mysql driver from pulling entire resultset into memory. explicitly close resultset and statement. --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 25 ++++++++++++++++----- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index c8a5d760127a3..4c3054465c384 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -15,7 +15,7 @@ import spark.util.NextIterator @param lowerBound the minimum value of the first placeholder @param upperBound the maximum value of the second placeholder The lower and upper bounds are inclusive. - @param numPartitions the amount of parallelism. + @param numPartitions the number of partitions. Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, the query would be executed twice, once with (1, 10) and once with (11, 20) @param mapRow a function from a ResultSet to a single row of the desired result type(s). @@ -40,10 +40,15 @@ class JdbcRDD[T: ClassManifest]( toArray override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { + context.addOnCompleteCallback{ () => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() - context.addOnCompleteCallback{ () => closeIfNeeded() } - val stmt = conn.prepareStatement(sql) + val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) + // force mysql driver to stream rather than pull entire resultset into memory + if (conn.getMetaData.getURL.matches("jdbc:mysql:.*")) { + stmt.setFetchSize(Integer.MIN_VALUE) + logInfo("statement fetch size set to: " + stmt.getFetchSize + " to force MySQL streaming ") + } stmt.setLong(1, part.lower) stmt.setLong(2, part.upper) val rs = stmt.executeQuery() @@ -59,8 +64,18 @@ class JdbcRDD[T: ClassManifest]( override def close() { try { - logInfo("closing connection") - conn.close() + if (null != rs && ! rs.isClosed()) rs.close() + } catch { + case e: Exception => logWarning("Exception closing resultset", e) + } + try { + if (null != stmt && ! stmt.isClosed()) stmt.close() + } catch { + case e: Exception => logWarning("Exception closing statement", e) + } + try { + if (null != conn && ! stmt.isClosed()) conn.close() + logInfo("closed connection") } catch { case e: Exception => logWarning("Exception closing connection", e) } From b11058f42c1c9c66ea94d3732c2efbdb57cb42b6 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 23 Apr 2013 22:48:32 +0530 Subject: [PATCH 476/696] Ensure that maven package adds yarn jars as part of shaded jar for hadoop2-yarn profile --- repl-bin/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index f9d84fd3c4ff2..b66d193b5df05 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -189,17 +189,17 @@ org.apache.hadoop hadoop-client - provided + runtime org.apache.hadoop hadoop-yarn-api - provided + runtime org.apache.hadoop hadoop-yarn-common - provided + runtime From 8faf5c51c3ea0b3ad83418552b50db596fefc558 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 24 Apr 2013 02:31:57 +0530 Subject: [PATCH 477/696] Patch from Thomas Graves to improve the YARN Client, and move to more production ready hadoop yarn branch --- core/pom.xml | 5 ++ .../scala/spark/deploy/yarn/Client.scala | 72 +++---------------- pom.xml | 9 ++- project/SparkBuild.scala | 5 +- repl-bin/pom.xml | 5 ++ 5 files changed, 30 insertions(+), 66 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 9baa4476621b3..7f65ce5c004af 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -297,6 +297,11 @@ hadoop-yarn-common provided + + org.apache.hadoop + hadoop-yarn-client + provided + diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index c007dae98c0d1..7a881e26dfffc 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -7,6 +7,7 @@ import org.apache.hadoop.net.NetUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.client.YarnClientImpl import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import scala.collection.mutable.HashMap @@ -16,19 +17,19 @@ import org.apache.hadoop.yarn.util.{Apps, Records, ConverterUtils} import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import spark.deploy.SparkHadoopUtil -class Client(conf: Configuration, args: ClientArguments) extends Logging { +class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { def this(args: ClientArguments) = this(new Configuration(), args) - var applicationsManager: ClientRMProtocol = null var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run() { - connectToASM() + init(yarnConf) + start() logClusterResourceDetails() - val newApp = getNewApplication() + val newApp = super.getNewApplication() val appId = newApp.getApplicationId() verifyClusterResources(newApp) @@ -47,64 +48,17 @@ class Client(conf: Configuration, args: ClientArguments) extends Logging { System.exit(0) } - - def connectToASM() { - val rmAddress: InetSocketAddress = NetUtils.createSocketAddr( - yarnConf.get(YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS) - ) - logInfo("Connecting to ResourceManager at" + rmAddress) - applicationsManager = rpc.getProxy(classOf[ClientRMProtocol], rmAddress, conf) - .asInstanceOf[ClientRMProtocol] - } def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = getYarnClusterMetrics + val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics logInfo("Got Cluster metric info from ASM, numNodeManagers=" + clusterMetrics.getNumNodeManagers) -/* - val clusterNodeReports: List[NodeReport] = getNodeReports - logDebug("Got Cluster node info from ASM") - for (node <- clusterNodeReports) { - logDebug("Got node report from ASM for, nodeId=" + node.getNodeId + ", nodeAddress=" + node.getHttpAddress + - ", nodeRackName=" + node.getRackName + ", nodeNumContainers=" + node.getNumContainers + ", nodeHealthStatus=" + node.getNodeHealthStatus) - } -*/ - - val queueInfo: QueueInfo = getQueueInfo(args.amQueue) + val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) logInfo("Queue info .. queueName=" + queueInfo.getQueueName + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity + ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size + ", queueChildQueueCount=" + queueInfo.getChildQueues.size) } - def getYarnClusterMetrics: YarnClusterMetrics = { - val request: GetClusterMetricsRequest = Records.newRecord(classOf[GetClusterMetricsRequest]) - val response: GetClusterMetricsResponse = applicationsManager.getClusterMetrics(request) - return response.getClusterMetrics - } - - def getNodeReports: List[NodeReport] = { - val request: GetClusterNodesRequest = Records.newRecord(classOf[GetClusterNodesRequest]) - val response: GetClusterNodesResponse = applicationsManager.getClusterNodes(request) - return response.getNodeReports.toList - } - - def getQueueInfo(queueName: String): QueueInfo = { - val request: GetQueueInfoRequest = Records.newRecord(classOf[GetQueueInfoRequest]) - request.setQueueName(queueName) - request.setIncludeApplications(true) - request.setIncludeChildQueues(false) - request.setRecursive(false) - Records.newRecord(classOf[GetQueueInfoRequest]) - return applicationsManager.getQueueInfo(request).getQueueInfo - } - - def getNewApplication(): GetNewApplicationResponse = { - logInfo("Requesting new Application") - val request = Records.newRecord(classOf[GetNewApplicationRequest]) - val response = applicationsManager.getNewApplication(request) - logInfo("Got new ApplicationId: " + response.getApplicationId()) - return response - } def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() @@ -265,23 +219,15 @@ class Client(conf: Configuration, args: ClientArguments) extends Logging { } def submitApp(appContext: ApplicationSubmissionContext) = { - // Create the request to send to the applications manager - val appRequest = Records.newRecord(classOf[SubmitApplicationRequest]) - .asInstanceOf[SubmitApplicationRequest] - appRequest.setApplicationSubmissionContext(appContext) // Submit the application to the applications manager logInfo("Submitting application to ASM") - applicationsManager.submitApplication(appRequest) + super.submitApplication(appContext) } def monitorApplication(appId: ApplicationId): Boolean = { while(true) { Thread.sleep(1000) - val reportRequest = Records.newRecord(classOf[GetApplicationReportRequest]) - .asInstanceOf[GetApplicationReportRequest] - reportRequest.setApplicationId(appId) - val reportResponse = applicationsManager.getApplicationReport(reportRequest) - val report = reportResponse.getApplicationReport() + val report = super.getApplicationReport(appId) logInfo("Application report from ASM: \n" + "\t application identifier: " + appId.toString() + "\n" + diff --git a/pom.xml b/pom.xml index ecbfaf9b4743e..0e95520d5063c 100644 --- a/pom.xml +++ b/pom.xml @@ -564,7 +564,9 @@ hadoop2-yarn 2 - 2.0.2-alpha + + 0.23.7 + @@ -599,6 +601,11 @@ hadoop-yarn-common ${yarn.version} + + org.apache.hadoop + hadoop-yarn-client + ${yarn.version} + org.apache.avro diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0a5b89d92704f..819e940403dd8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -20,7 +20,7 @@ object SparkBuild extends Build { //val HADOOP_YARN = false // For Hadoop 2 YARN support - val HADOOP_VERSION = "2.0.2-alpha" + val HADOOP_VERSION = "0.23.7" val HADOOP_MAJOR_VERSION = "2" val HADOOP_YARN = true @@ -156,7 +156,8 @@ object SparkBuild extends Build { Seq( "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION, "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION, - "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION + "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION, + "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION ) } else { Seq( diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index b66d193b5df05..46f38c277231f 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -201,6 +201,11 @@ hadoop-yarn-common runtime + + org.apache.hadoop + hadoop-yarn-client + runtime + From 5b85c715c8e4241f5e07237ed62729f1a7a800a8 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 24 Apr 2013 02:57:51 +0530 Subject: [PATCH 478/696] Revert back to 2.0.2-alpha : 0.23.7 has protocol changes which break against cloudera --- pom.xml | 4 ++-- project/SparkBuild.scala | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 0e95520d5063c..12e310a03838f 100644 --- a/pom.xml +++ b/pom.xml @@ -565,8 +565,8 @@ 2 - 0.23.7 - + + 2.0.2-alpha diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 819e940403dd8..0a761f1c13980 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -20,7 +20,8 @@ object SparkBuild extends Build { //val HADOOP_YARN = false // For Hadoop 2 YARN support - val HADOOP_VERSION = "0.23.7" + // val HADOOP_VERSION = "0.23.7" + val HADOOP_VERSION = "2.0.2-alpha" val HADOOP_MAJOR_VERSION = "2" val HADOOP_YARN = true From 31ce6c66d6f29302d0f0f2c70e494fad0ba71e4d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Apr 2013 17:48:59 -0700 Subject: [PATCH 479/696] Added a BlockObjectWriter interface in block manager so ShuffleMapTask doesn't need to build up an array buffer for each shuffle bucket. --- core/src/main/scala/spark/SparkEnv.scala | 18 +++++--- .../spark/scheduler/ShuffleMapTask.scala | 30 +++++++------ .../scala/spark/storage/BlockException.scala | 5 +++ .../scala/spark/storage/BlockManager.scala | 36 ++++++++++++---- .../spark/storage/BlockObjectWriter.scala | 27 ++++++++++++ .../main/scala/spark/storage/DiskStore.scala | 43 ++++++++++++++++--- .../scala/spark/storage/ThreadingTest.scala | 2 +- 7 files changed, 129 insertions(+), 32 deletions(-) create mode 100644 core/src/main/scala/spark/storage/BlockException.scala create mode 100644 core/src/main/scala/spark/storage/BlockObjectWriter.scala diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7157fd26883d3..c10bedb8f6f53 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -22,6 +22,7 @@ class SparkEnv ( val actorSystem: ActorSystem, val serializer: Serializer, val closureSerializer: Serializer, + val shuffleSerializer: Serializer, val cacheManager: CacheManager, val mapOutputTracker: MapOutputTracker, val shuffleFetcher: ShuffleFetcher, @@ -82,7 +83,7 @@ object SparkEnv extends Logging { } val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") - + def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { logInfo("Registering " + name) @@ -96,18 +97,22 @@ object SparkEnv extends Logging { } } + val closureSerializer = instantiateClass[Serializer]( + "spark.closure.serializer", "spark.JavaSerializer") + + val shuffleSerializer = instantiateClass[Serializer]( + "spark.shuffle.serializer", "spark.JavaSerializer") + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new spark.storage.BlockManagerMasterActor(isLocal))) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) + val blockManager = new BlockManager( + executorId, actorSystem, blockManagerMaster, serializer, shuffleSerializer) val connectionManager = blockManager.connectionManager val broadcastManager = new BroadcastManager(isDriver) - val closureSerializer = instantiateClass[Serializer]( - "spark.closure.serializer", "spark.JavaSerializer") - val cacheManager = new CacheManager(blockManager) // Have to assign trackerActor after initialization as MapOutputTrackerActor @@ -144,6 +149,7 @@ object SparkEnv extends Logging { actorSystem, serializer, closureSerializer, + shuffleSerializer, cacheManager, mapOutputTracker, shuffleFetcher, @@ -153,5 +159,5 @@ object SparkEnv extends Logging { httpFileServer, sparkFilesDir) } - + } diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 36d087a4d009c..97b668cd58c05 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -122,27 +122,33 @@ private[spark] class ShuffleMapTask( val taskContext = new TaskContext(stageId, partition, attemptId) metrics = Some(taskContext.taskMetrics) try { - // Partition the map output. - val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) + // Obtain all the block writers for shuffle blocks. + val blockManager = SparkEnv.get.blockManager + val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => + val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId + blockManager.getBlockWriter(blockId) + } + + // Write the map output to its associated buckets. for (elem <- rdd.iterator(split, taskContext)) { val pair = elem.asInstanceOf[(Any, Any)] val bucketId = dep.partitioner.getPartition(pair._1) - buckets(bucketId) += pair + buckets(bucketId).write(pair) } + // Close the bucket writers and get the sizes of each block. val compressedSizes = new Array[Byte](numOutputSplits) - - var totalBytes = 0l - - val blockManager = SparkEnv.get.blockManager - for (i <- 0 until numOutputSplits) { - val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i - // Get a Scala iterator from Java map - val iter: Iterator[(Any, Any)] = buckets(i).iterator - val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) + var i = 0 + var totalBytes = 0L + while (i < numOutputSplits) { + buckets(i).close() + val size = buckets(i).size() totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) + i += 1 } + + // Update shuffle metrics. val shuffleMetrics = new ShuffleWriteMetrics shuffleMetrics.shuffleBytesWritten = totalBytes metrics.get.shuffleWriteMetrics = Some(shuffleMetrics) diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/spark/storage/BlockException.scala new file mode 100644 index 0000000000000..f275d476df9f1 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockException.scala @@ -0,0 +1,5 @@ +package spark.storage + +private[spark] +case class BlockException(blockId: String, message: String) extends Exception(message) + diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 210061e9726b4..2f97bad916401 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -24,16 +24,13 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer -private[spark] -case class BlockException(blockId: String, message: String, ex: Exception = null) -extends Exception(message) - private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, val serializer: Serializer, + val shuffleSerializer: Serializer, maxMemory: Long) extends Logging { @@ -78,7 +75,7 @@ class BlockManager( private val blockInfo = new TimeStampedHashMap[String, BlockInfo] private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) - private[storage] val diskStore: BlockStore = + private[storage] val diskStore: DiskStore = new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) val connectionManager = new ConnectionManager(0) @@ -126,8 +123,17 @@ class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) + serializer: Serializer, shuffleSerializer: Serializer) = { + this(execId, actorSystem, master, serializer, shuffleSerializer, + BlockManager.getMaxMemoryFromSystemProperties) + } + + /** + * Construct a BlockManager with a memory limit set based on system properties. + */ + def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, + serializer: Serializer, maxMemory: Long) = { + this(execId, actorSystem, master, serializer, serializer, maxMemory) } /** @@ -485,6 +491,21 @@ class BlockManager( put(blockId, elements, level, tellMaster) } + /** + * A short circuited method to get a block writer that can write data directly to disk. + * This is currently used for writing shuffle files out. + */ + def getBlockWriter(blockId: String): BlockObjectWriter = { + val writer = diskStore.getBlockWriter(blockId) + writer.registerCloseEventHandler(() => { + // TODO(rxin): This doesn't handle error cases. + val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) + blockInfo.put(blockId, myInfo) + myInfo.markReady(writer.size()) + }) + writer + } + /** * Put a new block of values to the block manager. Returns its (estimated) size in bytes. */ @@ -574,7 +595,6 @@ class BlockManager( } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) - // Replicate block if required if (level.replication > 1) { val remoteStartTime = System.currentTimeMillis diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala new file mode 100644 index 0000000000000..657a7e9143ab7 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -0,0 +1,27 @@ +package spark.storage + +import java.nio.ByteBuffer + + +abstract class BlockObjectWriter(val blockId: String) { + + // TODO(rxin): What if there is an exception when the block is being written out? + + var closeEventHandler: () => Unit = _ + + def registerCloseEventHandler(handler: () => Unit) { + closeEventHandler = handler + } + + def write(value: Any) + + def writeAll(value: Iterator[Any]) { + value.foreach(write) + } + + def close() { + closeEventHandler() + } + + def size(): Long +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index ddbf8821ad15a..493936fdbe19e 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -1,7 +1,7 @@ package spark.storage import java.nio.ByteBuffer -import java.io.{File, FileOutputStream, RandomAccessFile} +import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} import java.nio.channels.FileChannel.MapMode import java.util.{Random, Date} import java.text.SimpleDateFormat @@ -10,9 +10,9 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import scala.collection.mutable.ArrayBuffer +import spark.Utils import spark.executor.ExecutorExitCode -import spark.Utils /** * Stores BlockManager blocks on disk. @@ -20,6 +20,33 @@ import spark.Utils private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { + class DiskBlockObjectWriter(blockId: String) extends BlockObjectWriter(blockId) { + + private val f: File = createFile(blockId /*, allowAppendExisting */) + private val bs: OutputStream = blockManager.wrapForCompression(blockId, + new FastBufferedOutputStream(new FileOutputStream(f))) + private val objOut = blockManager.shuffleSerializer.newInstance().serializeStream(bs) + + private var _size: Long = -1L + + override def write(value: Any) { + objOut.writeObject(value) + } + + override def close() { + objOut.close() + bs.close() + super.close() + } + + override def size(): Long = { + if (_size < 0) { + _size = f.length() + } + _size + } + } + val MAX_DIR_CREATION_ATTEMPTS: Int = 10 val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt @@ -31,6 +58,10 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() + def getBlockWriter(blockId: String): BlockObjectWriter = { + new DiskBlockObjectWriter(blockId) + } + override def getSize(blockId: String): Long = { getFile(blockId).length() } @@ -65,8 +96,10 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) objOut.writeAll(values.iterator) objOut.close() val length = file.length() + + val timeTaken = System.currentTimeMillis - startTime logDebug("Block %s stored as %s file on disk in %d ms".format( - blockId, Utils.memoryBytesToString(length), (System.currentTimeMillis - startTime))) + blockId, Utils.memoryBytesToString(length), timeTaken)) if (returnValues) { // Return a byte buffer for the contents of the file @@ -106,9 +139,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) getFile(blockId).exists() } - private def createFile(blockId: String): File = { + private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = { val file = getFile(blockId) - if (file.exists()) { + if (!allowAppendExisting && file.exists()) { throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 5c406e68cb2ac..3875e7459e6d1 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -78,7 +78,7 @@ private[spark] object ThreadingTest { val blockManagerMaster = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) + "", actorSystem, blockManagerMaster, serializer, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) From adcda84f9646f12e6d5fb4f1e5e3a1b0a98b7c9f Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 24 Apr 2013 08:57:25 +0530 Subject: [PATCH 480/696] Pull latest SparkBuild.scala from master and merge conflicts --- project/SparkBuild.scala | 43 +++++++++++++++++++++++----------------- 1 file changed, 25 insertions(+), 18 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0a761f1c13980..0c2598ab35e84 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -43,15 +43,22 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", - version := "0.7.1-SNAPSHOT", - scalaVersion := "2.9.2", + version := "0.8.0-SNAPSHOT", + scalaVersion := "2.9.3", scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), - // shared between both core and streaming. + // Fork new JVMs for tests and set Java options for those + fork := true, + javaOptions += "-Xmx1g", + + // Only allow one test at a time, even across projects, since they run in the same JVM + concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), + + // Shared between both core and streaming. resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), // For Sonatype publishing @@ -100,13 +107,12 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty" % "3.5.3.Final", - "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", - "org.scalatest" %% "scalatest" % "1.8" % "test", - "org.scalacheck" %% "scalacheck" % "1.9" % "test", - "com.novocode" % "junit-interface" % "0.8" % "test", + "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", + "com.novocode" % "junit-interface" % "0.9" % "test", "org.easymock" % "easymock" % "3.1" % "test" ), - parallelExecution := false, /* Workaround for issue #206 (fixed after SBT 0.11.0) */ watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task, const(std.TaskExtra.constant(Nil)), aggregate = true, includeRoot = true) apply { _.join.map(_.flatten) }, @@ -137,8 +143,8 @@ object SparkBuild extends Build { "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "com.ning" % "compress-lzf" % "0.8.4", "commons-daemon" % "commons-daemon" % "1.0.10", + "com.ning" % "compress-lzf" % "0.8.4", "asm" % "asm-all" % "3.3.1", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", @@ -149,25 +155,26 @@ object SparkBuild extends Build { "colt" % "colt" % "1.2.0", "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", - "cc.spray" %% "spray-json" % "1.1.1", + "cc.spray" % "spray-json_2.9.2" % "1.1.1", "org.apache.mesos" % "mesos" % "0.9.0-incubating" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { Seq( - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION, - "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION, - "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION, - "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION + // Exclude rule required for all ? + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), + "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), + "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), + "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ) ) } else { Seq( - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ) ) } } else { - Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION) + Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ) ) }), unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") { @@ -189,7 +196,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8") + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From d09db1c051d255157f38f400fe9301fa438c5f41 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 24 Apr 2013 09:15:29 +0530 Subject: [PATCH 481/696] concurrentRestrictions fails for this PR - but works for master, probably some version change --- project/SparkBuild.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0c2598ab35e84..947ac47f6bb62 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -55,9 +55,6 @@ object SparkBuild extends Build { fork := true, javaOptions += "-Xmx1g", - // Only allow one test at a time, even across projects, since they run in the same JVM - concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), - // Shared between both core and streaming. resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), From 3b594a4e3b94de49a09dc679a30d857e3f41df69 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 24 Apr 2013 10:18:25 +0530 Subject: [PATCH 482/696] Do not add signature files - results in validation errors when using assembled file --- project/SparkBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0c2598ab35e84..b3f410bfa63fc 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -214,6 +214,7 @@ object SparkBuild extends Build { def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq( mergeStrategy in assembly := { case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard + case m if m.toLowerCase.matches("meta-inf/.*\\.sf$") => MergeStrategy.discard case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first } From aa618ed2a2df209da3f93a025928366959c37d04 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 24 Apr 2013 14:52:49 -0700 Subject: [PATCH 483/696] Allow changing the serializer on a per shuffle basis. --- .../spark/BlockStoreShuffleFetcher.scala | 13 +++-- core/src/main/scala/spark/Dependency.scala | 4 +- .../main/scala/spark/PairRDDFunctions.scala | 11 ++-- .../src/main/scala/spark/ShuffleFetcher.scala | 7 ++- core/src/main/scala/spark/SparkEnv.scala | 20 +++----- .../main/scala/spark/rdd/CoGroupedRDD.scala | 13 +++-- .../main/scala/spark/rdd/ShuffledRDD.scala | 11 ++-- .../main/scala/spark/rdd/SubtractedRDD.scala | 18 +++++-- .../spark/scheduler/ShuffleMapTask.scala | 8 +-- .../scala/spark/serializer/Serializer.scala | 50 ++++++++++++++++++- .../scala/spark/storage/BlockManager.scala | 42 ++++++++-------- .../spark/storage/BlockManagerWorker.scala | 18 +++---- .../main/scala/spark/storage/DiskStore.scala | 18 ++++--- .../scala/spark/storage/ThreadingTest.scala | 2 +- 14 files changed, 153 insertions(+), 82 deletions(-) diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index c27ed36406119..2156efbd458fe 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,14 +1,19 @@ package spark -import executor.{ShuffleReadMetrics, TaskMetrics} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import spark.executor.{ShuffleReadMetrics, TaskMetrics} +import spark.serializer.Serializer import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} -import util.{CompletionIterator, TimedIterator} +import spark.util.{CompletionIterator, TimedIterator} + private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { - override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { + + override def fetch[K, V]( + shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer) = { + logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager @@ -48,7 +53,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress) + val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker itr.setDelegate(blockFetcherItr) CompletionIterator[(K,V), Iterator[(K,V)]](itr, { diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 5eea9073220f1..2af44aa383444 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -25,10 +25,12 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * @param shuffleId the shuffle id * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output + * @param serializerClass class name of the serializer to use */ class ShuffleDependency[K, V]( @transient rdd: RDD[(K, V)], - val partitioner: Partitioner) + val partitioner: Partitioner, + val serializerClass: String = null) extends Dependency(rdd) { val shuffleId: Int = rdd.context.newShuffleId() diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 07efba9e8d26e..1b9b9d21d8ecb 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -52,7 +52,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, - mapSideCombine: Boolean = true): RDD[(K, C)] = { + mapSideCombine: Boolean = true, + serializerClass: String = null): RDD[(K, C)] = { if (getKeyClass().isArray) { if (mapSideCombine) { throw new SparkException("Cannot use map-side combining with array keys.") @@ -67,13 +68,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( self.mapPartitions(aggregator.combineValuesByKey(_), true) } else if (mapSideCombine) { val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true) - val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner) + val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner, serializerClass) partitioned.mapPartitions(aggregator.combineCombinersByKey(_), true) } else { // Don't apply map-side combiner. // A sanity check to make sure mergeCombiners is not defined. assert(mergeCombiners == null) - val values = new ShuffledRDD[K, V](self, partitioner) + val values = new ShuffledRDD[K, V](self, partitioner, serializerClass) values.mapPartitions(aggregator.combineValuesByKey(_), true) } } @@ -469,7 +470,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Return an RDD with the pairs from `this` whose keys are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ @@ -644,7 +645,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Return an RDD with the keys of each tuple. */ def keys: RDD[K] = self.map(_._1) - + /** * Return an RDD with the values of each tuple. */ diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 442e9f0269dc4..49addc0c1075f 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,13 +1,16 @@ package spark -import executor.TaskMetrics +import spark.executor.TaskMetrics +import spark.serializer.Serializer + private[spark] abstract class ShuffleFetcher { /** * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) : Iterator[(K,V)] + def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, + serializer: Serializer = Serializer.default): Iterator[(K,V)] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index c10bedb8f6f53..8a751fbd6ebf9 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -3,13 +3,14 @@ package spark import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider -import serializer.Serializer import spark.broadcast.BroadcastManager import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager +import spark.serializer.Serializer import spark.util.AkkaUtils + /** * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently @@ -22,7 +23,6 @@ class SparkEnv ( val actorSystem: ActorSystem, val serializer: Serializer, val closureSerializer: Serializer, - val shuffleSerializer: Serializer, val cacheManager: CacheManager, val mapOutputTracker: MapOutputTracker, val shuffleFetcher: ShuffleFetcher, @@ -82,7 +82,11 @@ object SparkEnv extends Logging { Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } - val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") + val serializer = Serializer.setDefault( + System.getProperty("spark.serializer", "spark.JavaSerializer")) + + val closureSerializer = Serializer.get( + System.getProperty("spark.closure.serializer", "spark.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { @@ -97,17 +101,10 @@ object SparkEnv extends Logging { } } - val closureSerializer = instantiateClass[Serializer]( - "spark.closure.serializer", "spark.JavaSerializer") - - val shuffleSerializer = instantiateClass[Serializer]( - "spark.shuffle.serializer", "spark.JavaSerializer") - val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new spark.storage.BlockManagerMasterActor(isLocal))) - val blockManager = new BlockManager( - executorId, actorSystem, blockManagerMaster, serializer, shuffleSerializer) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) val connectionManager = blockManager.connectionManager @@ -149,7 +146,6 @@ object SparkEnv extends Logging { actorSystem, serializer, closureSerializer, - shuffleSerializer, cacheManager, mapOutputTracker, shuffleFetcher, diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index a6235491cab96..9e996e9958b0e 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -8,6 +8,7 @@ import scala.collection.mutable.ArrayBuffer import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} +import spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -54,7 +55,8 @@ private[spark] class CoGroupAggregator class CoGroupedRDD[K]( @transient var rdds: Seq[RDD[(K, _)]], part: Partitioner, - val mapSideCombine: Boolean = true) + val mapSideCombine: Boolean = true, + val serializerClass: String = null) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { private val aggr = new CoGroupAggregator @@ -68,9 +70,9 @@ class CoGroupedRDD[K]( logInfo("Adding shuffle dependency with " + rdd) if (mapSideCombine) { val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true) - new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part) + new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part, serializerClass) } else { - new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part) + new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part, serializerClass) } } } @@ -112,6 +114,7 @@ class CoGroupedRDD[K]( } } + val ser = Serializer.get(serializerClass) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent @@ -124,12 +127,12 @@ class CoGroupedRDD[K]( val fetcher = SparkEnv.get.shuffleFetcher if (mapSideCombine) { // With map side combine on, for each key, the shuffle fetcher returns a list of values. - fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics).foreach { + fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics, ser).foreach { case (key, values) => getSeq(key)(depNum) ++= values } } else { // With map side combine off, for each key the shuffle fetcher returns a single value. - fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics).foreach { + fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics, ser).foreach { case (key, value) => getSeq(key)(depNum) += value } } diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 4e33b7dd5ca84..8175e23eff264 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -2,6 +2,8 @@ package spark.rdd import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} import spark.SparkContext._ +import spark.serializer.Serializer + private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx @@ -12,13 +14,15 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD + * @param serializerClass class name of the serializer to use. * @tparam K the key class. * @tparam V the value class. */ class ShuffledRDD[K, V]( @transient prev: RDD[(K, V)], - part: Partitioner) - extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part))) { + part: Partitioner, + serializerClass: String = null) + extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part, serializerClass))) { override val partitioner = Some(part) @@ -28,6 +32,7 @@ class ShuffledRDD[K, V]( override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics) + SparkEnv.get.shuffleFetcher.fetch[K, V]( + shuffledId, split.index, context.taskMetrics, Serializer.get(serializerClass)) } } diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 481e03b349af0..f60c35c38ec31 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -11,6 +11,7 @@ import spark.Partition import spark.SparkEnv import spark.ShuffleDependency import spark.OneToOneDependency +import spark.serializer.Serializer /** * An optimized version of cogroup for set difference/subtraction. @@ -31,7 +32,9 @@ import spark.OneToOneDependency private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest]( @transient var rdd1: RDD[(K, V)], @transient var rdd2: RDD[(K, W)], - part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { + part: Partitioner, + val serializerClass: String = null) + extends RDD[(K, V)](rdd1.context, Nil) { override def getDependencies: Seq[Dependency[_]] = { Seq(rdd1, rdd2).map { rdd => @@ -40,7 +43,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM new OneToOneDependency(rdd) } else { logInfo("Adding shuffle dependency with " + rdd) - new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part) + new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part, serializerClass) } } } @@ -65,6 +68,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] + val serializer = Serializer.get(serializerClass) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) @@ -77,12 +81,16 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM } } def integrate(dep: CoGroupSplitDep, op: ((K, V)) => Unit) = dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { for (t <- rdd.iterator(itsSplit, context)) op(t.asInstanceOf[(K, V)]) - case ShuffleCoGroupSplitDep(shuffleId) => - for (t <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, context.taskMetrics)) + } + case ShuffleCoGroupSplitDep(shuffleId) => { + val iter = SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, + context.taskMetrics, serializer) + for (t <- iter) op(t.asInstanceOf[(K, V)]) + } } // the first dep is rdd1; add all values to the map integrate(partition.deps(0), t => getSeq(t._1) += t._2) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 97b668cd58c05..d9b26c9db9187 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -13,9 +13,11 @@ import com.ning.compress.lzf.LZFInputStream import com.ning.compress.lzf.LZFOutputStream import spark._ -import executor.ShuffleWriteMetrics +import spark.executor.ShuffleWriteMetrics +import spark.serializer.Serializer import spark.storage._ -import util.{TimeStampedHashMap, MetadataCleaner} +import spark.util.{TimeStampedHashMap, MetadataCleaner} + private[spark] object ShuffleMapTask { @@ -126,7 +128,7 @@ private[spark] class ShuffleMapTask( val blockManager = SparkEnv.get.blockManager val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId - blockManager.getBlockWriter(blockId) + blockManager.getBlockWriter(blockId, Serializer.get(dep.serializerClass)) } // Write the map output to its associated buckets. diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index aca86ab6f0f1d..77b1a1a434cb3 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -1,10 +1,14 @@ package spark.serializer -import java.nio.ByteBuffer import java.io.{EOFException, InputStream, OutputStream} +import java.nio.ByteBuffer +import java.util.concurrent.ConcurrentHashMap + import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream + import spark.util.ByteBufferInputStream + /** * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[spark.serializer.SerializerInstance]] objects that do the actual serialization and are @@ -14,6 +18,48 @@ trait Serializer { def newInstance(): SerializerInstance } + +/** + * A singleton object that can be used to fetch serializer objects based on the serializer + * class name. If a previous instance of the serializer object has been created, the get + * method returns that instead of creating a new one. + */ +object Serializer { + + private val serializers = new ConcurrentHashMap[String, Serializer] + private var _default: Serializer = _ + + def default = _default + + def setDefault(clsName: String): Serializer = { + _default = get(clsName) + _default + } + + def get(clsName: String): Serializer = { + if (clsName == null) { + default + } else { + var serializer = serializers.get(clsName) + if (serializer != null) { + // If the serializer has been created previously, reuse that. + serializer + } else this.synchronized { + // Otherwise, create a new one. But make sure no other thread has attempted + // to create another new one at the same time. + serializer = serializers.get(clsName) + if (serializer == null) { + val clsLoader = Thread.currentThread.getContextClassLoader + serializer = Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] + serializers.put(clsName, serializer) + } + serializer + } + } + } +} + + /** * An instance of a serializer, for use by one thread at a time. */ @@ -45,6 +91,7 @@ trait SerializerInstance { } } + /** * A stream for writing serialized objects. */ @@ -61,6 +108,7 @@ trait SerializationStream { } } + /** * A stream for reading serialized objects. */ diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 2f97bad916401..9f7985e2e8db3 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -29,8 +29,7 @@ class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, - val serializer: Serializer, - val shuffleSerializer: Serializer, + val defaultSerializer: Serializer, maxMemory: Long) extends Logging { @@ -123,17 +122,8 @@ class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, shuffleSerializer: Serializer) = { - this(execId, actorSystem, master, serializer, shuffleSerializer, - BlockManager.getMaxMemoryFromSystemProperties) - } - - /** - * Construct a BlockManager with a memory limit set based on system properties. - */ - def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, maxMemory: Long) = { - this(execId, actorSystem, master, serializer, serializer, maxMemory) + serializer: Serializer) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) } /** @@ -479,9 +469,10 @@ class BlockManager( * fashion as they're received. Expects a size in bytes to be provided for each block fetched, * so that we can control the maxMegabytesInFlight for the fetch. */ - def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]) + def getMultiple( + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], serializer: Serializer) : BlockFetcherIterator = { - return new BlockFetcherIterator(this, blocksByAddress) + return new BlockFetcherIterator(this, blocksByAddress, serializer) } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -495,8 +486,8 @@ class BlockManager( * A short circuited method to get a block writer that can write data directly to disk. * This is currently used for writing shuffle files out. */ - def getBlockWriter(blockId: String): BlockObjectWriter = { - val writer = diskStore.getBlockWriter(blockId) + def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { + val writer = diskStore.getBlockWriter(blockId, serializer) writer.registerCloseEventHandler(() => { // TODO(rxin): This doesn't handle error cases. val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) @@ -850,7 +841,10 @@ class BlockManager( if (shouldCompress(blockId)) new LZFInputStream(s) else s } - def dataSerialize(blockId: String, values: Iterator[Any]): ByteBuffer = { + def dataSerialize( + blockId: String, + values: Iterator[Any], + serializer: Serializer = defaultSerializer): ByteBuffer = { val byteStream = new FastByteArrayOutputStream(4096) val ser = serializer.newInstance() ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() @@ -862,7 +856,10 @@ class BlockManager( * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of * the iterator is reached. */ - def dataDeserialize(blockId: String, bytes: ByteBuffer): Iterator[Any] = { + def dataDeserialize( + blockId: String, + bytes: ByteBuffer, + serializer: Serializer = defaultSerializer): Iterator[Any] = { bytes.rewind() val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) serializer.newInstance().deserializeStream(stream).asIterator @@ -916,7 +913,8 @@ object BlockManager extends Logging { class BlockFetcherIterator( private val blockManager: BlockManager, - val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] + val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer ) extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { import blockManager._ @@ -979,8 +977,8 @@ class BlockFetcherIterator( "Unexpected message " + blockMessage.getType + " received from " + cmId) } val blockId = blockMessage.getId - results.put(new FetchResult( - blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) + results.put(new FetchResult(blockId, sizeMap(blockId), + () => dataDeserialize(blockId, blockMessage.getData, serializer))) _remoteBytesRead += req.size logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index d2985559c13f8..15225f93a626d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -19,7 +19,7 @@ import spark.network._ */ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging { initLogging() - + blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive) def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { @@ -51,7 +51,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends logDebug("Received [" + pB + "]") putBlock(pB.id, pB.data, pB.level) return None - } + } case BlockMessage.TYPE_GET_BLOCK => { val gB = new GetBlock(blockMessage.getId) logDebug("Received [" + gB + "]") @@ -90,28 +90,26 @@ private[spark] object BlockManagerWorker extends Logging { private var blockManagerWorker: BlockManagerWorker = null private val DATA_TRANSFER_TIME_OUT_MS: Long = 500 private val REQUEST_RETRY_INTERVAL_MS: Long = 1000 - + initLogging() - + def startBlockManagerWorker(manager: BlockManager) { blockManagerWorker = new BlockManagerWorker(manager) } - + def syncPutBlock(msg: PutBlock, toConnManagerId: ConnectionManagerId): Boolean = { val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val serializer = blockManager.serializer + val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromPutBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) val resultMessage = connectionManager.sendMessageReliablySync( toConnManagerId, blockMessageArray.toBufferMessage) return (resultMessage != None) } - + def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = { val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val serializer = blockManager.serializer + val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromGetBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) val responseMessage = connectionManager.sendMessageReliablySync( diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 493936fdbe19e..70ad887c3ba2f 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -1,17 +1,18 @@ package spark.storage -import java.nio.ByteBuffer import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} +import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode import java.util.{Random, Date} import java.text.SimpleDateFormat -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - import scala.collection.mutable.ArrayBuffer +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream + import spark.Utils import spark.executor.ExecutorExitCode +import spark.serializer.Serializer /** @@ -20,12 +21,13 @@ import spark.executor.ExecutorExitCode private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { - class DiskBlockObjectWriter(blockId: String) extends BlockObjectWriter(blockId) { + class DiskBlockObjectWriter(blockId: String, serializer: Serializer) + extends BlockObjectWriter(blockId) { private val f: File = createFile(blockId /*, allowAppendExisting */) private val bs: OutputStream = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(new FileOutputStream(f))) - private val objOut = blockManager.shuffleSerializer.newInstance().serializeStream(bs) + private val objOut = serializer.newInstance().serializeStream(bs) private var _size: Long = -1L @@ -58,8 +60,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() - def getBlockWriter(blockId: String): BlockObjectWriter = { - new DiskBlockObjectWriter(blockId) + def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { + new DiskBlockObjectWriter(blockId, serializer) } override def getSize(blockId: String): Long = { @@ -92,7 +94,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val file = createFile(blockId) val fileOut = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(new FileOutputStream(file))) - val objOut = blockManager.serializer.newInstance().serializeStream(fileOut) + val objOut = blockManager.defaultSerializer.newInstance().serializeStream(fileOut) objOut.writeAll(values.iterator) objOut.close() val length = file.length() diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 3875e7459e6d1..5c406e68cb2ac 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -78,7 +78,7 @@ private[spark] object ThreadingTest { val blockManagerMaster = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, serializer, 1024 * 1024) + "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) From ba6ffa6a5f39765e1652735d1b16b54c2fc78674 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 24 Apr 2013 17:38:07 -0700 Subject: [PATCH 484/696] Allow the specification of a shuffle serializer in the read path (for local block reads). --- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../scala/spark/storage/BlockManager.scala | 29 ++++++++++--------- .../main/scala/spark/storage/DiskStore.scala | 8 +++++ 3 files changed, 24 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index d9b26c9db9187..826f14d658a5e 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -128,7 +128,7 @@ private[spark] class ShuffleMapTask( val blockManager = SparkEnv.get.blockManager val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId - blockManager.getBlockWriter(blockId, Serializer.get(dep.serializerClass)) + blockManager.getDiskBlockWriter(blockId, Serializer.get(dep.serializerClass)) } // Write the map output to its associated buckets. diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9f7985e2e8db3..fa02dd54b8760 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -268,23 +268,24 @@ class BlockManager( return locations } + /** + * A short-circuited method to get blocks directly from disk. This is used for getting + * shuffle blocks. It is safe to do so without a lock on block info since disk store + * never deletes (recent) items. + */ + def getLocalFromDisk(blockId: String, serializer: Serializer): Option[Iterator[Any]] = { + diskStore.getValues(blockId, serializer) match { + case Some(iterator) => Some(iterator) + case None => + throw new Exception("Block " + blockId + " not found on disk, though it should be") + } + } + /** * Get block from local block manager. */ def getLocal(blockId: String): Option[Iterator[Any]] = { logDebug("Getting local block " + blockId) - - // As an optimization for map output fetches, if the block is for a shuffle, return it - // without acquiring a lock; the disk store never deletes (recent) items so this should work - if (blockId.startsWith("shuffle_")) { - return diskStore.getValues(blockId) match { - case Some(iterator) => - Some(iterator) - case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") - } - } - val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { @@ -486,7 +487,7 @@ class BlockManager( * A short circuited method to get a block writer that can write data directly to disk. * This is currently used for writing shuffle files out. */ - def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { + def getDiskBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { val writer = diskStore.getBlockWriter(blockId, serializer) writer.registerCloseEventHandler(() => { // TODO(rxin): This doesn't handle error cases. @@ -1042,7 +1043,7 @@ class BlockFetcherIterator( // any memory that might exceed our maxBytesInFlight startTime = System.currentTimeMillis for (id <- localBlockIds) { - getLocal(id) match { + getLocalFromDisk(id, serializer) match { case Some(iter) => { results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight logDebug("Got local block " + id) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 70ad887c3ba2f..7f512b162a71a 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -127,6 +127,14 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes)) } + /** + * A version of getValues that allows a custom serializer. This is used as part of the + * shuffle short-circuit code. + */ + def getValues(blockId: String, serializer: Serializer): Option[Iterator[Any]] = { + getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer)) + } + override def remove(blockId: String): Boolean = { val file = getFile(blockId) if (file.exists()) { From 01d9ba503878d4191eaa8080e86c631d3c705cce Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Apr 2013 00:11:27 -0700 Subject: [PATCH 485/696] Add back line removed during YARN merge --- project/SparkBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b3f410bfa63fc..44758ad87e13e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -48,6 +48,7 @@ object SparkBuild extends Build { scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, + retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), From eef9ea1993270d5f07e52e807e8d149e54079aad Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 24 Apr 2013 15:08:20 -0700 Subject: [PATCH 486/696] Update unit test memory to 2 GB --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 44758ad87e13e..f32c47e71f4db 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -54,7 +54,7 @@ object SparkBuild extends Build { // Fork new JVMs for tests and set Java options for those fork := true, - javaOptions += "-Xmx1g", + javaOptions += "-Xmx2g", // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), From 6e6b5204ea015fc7cc2c3e16e0032be3074413be Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 24 Apr 2013 18:53:12 -0700 Subject: [PATCH 487/696] Create an empty directory when checkpointing a 0-partition RDD (fixes a test failure on Hadoop 2.0) --- core/src/main/scala/spark/RDDCheckpointData.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index d00092e9845e2..57e0405fb4185 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -1,6 +1,7 @@ package spark import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration import rdd.{CheckpointRDD, CoalescedRDD} import scheduler.{ResultTask, ShuffleMapTask} @@ -62,14 +63,20 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) } } + // Create the output path for the checkpoint + val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) + val fs = path.getFileSystem(new Configuration()) + if (!fs.mkdirs(path)) { + throw new SparkException("Failed to create checkpoint path " + path) + } + // Save to file, and reload it as an RDD - val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id).toString - rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path) _) - val newRDD = new CheckpointRDD[T](rdd.context, path) + rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString) _) + val newRDD = new CheckpointRDD[T](rdd.context, path.toString) // Change the dependencies and partitions of the RDD RDDCheckpointData.synchronized { - cpFile = Some(path) + cpFile = Some(path.toString) cpRDD = Some(newRDD) rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions cpState = Checkpointed From a72134a6ac04e2e49679bbd5ba1266daf909bec8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 25 Apr 2013 10:39:28 -0700 Subject: [PATCH 488/696] SPARK-739 Have quickstart standlone job use README --- docs/quick-start.md | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index 2d961b29cb197..335643536aac9 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -113,8 +113,8 @@ import SparkContext._ object SimpleJob { def main(args: Array[String]) { - val logFile = "/var/log/syslog" // Should be some file on your system - val sc = new SparkContext("local", "Simple Job", "$YOUR_SPARK_HOME", + val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system + val sc = new SparkContext("local", "Simple Job", "YOUR_SPARK_HOME", List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar")) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() @@ -124,7 +124,7 @@ object SimpleJob { } {% endhighlight %} -This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. +This job simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds two repositories which host Spark dependencies: @@ -156,7 +156,7 @@ $ find . $ sbt package $ sbt run ... -Lines with a: 8422, Lines with b: 1836 +Lines with a: 46, Lines with b: 23 {% endhighlight %} This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. @@ -173,7 +173,7 @@ import spark.api.java.function.Function; public class SimpleJob { public static void main(String[] args) { - String logFile = "/var/log/syslog"; // Should be some file on your system + String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system JavaSparkContext sc = new JavaSparkContext("local", "Simple Job", "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"}); JavaRDD logData = sc.textFile(logFile).cache(); @@ -191,7 +191,7 @@ public class SimpleJob { } {% endhighlight %} -This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that like in the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. +This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. To build the job, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version. @@ -239,7 +239,7 @@ Now, we can execute the job using Maven: $ mvn package $ mvn exec:java -Dexec.mainClass="SimpleJob" ... -Lines with a: 8422, Lines with b: 1836 +Lines with a: 46, Lines with b: 23 {% endhighlight %} This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. @@ -253,7 +253,7 @@ As an example, we'll create a simple Spark job, `SimpleJob.py`: """SimpleJob.py""" from pyspark import SparkContext -logFile = "/var/log/syslog" # Should be some file on your system +logFile = "$YOUR_SPARK_HOME/README.md" # Should be some file on your system sc = SparkContext("local", "Simple job") logData = sc.textFile(logFile).cache() @@ -265,7 +265,8 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs) This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. -Like in the Scala and Java examples, we use a SparkContext to create RDDs. +Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. +As with the Scala and Java examples, we use a SparkContext to create RDDs. We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. For jobs that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html). `SimpleJob` is simple enough that we do not need to specify any code dependencies. @@ -276,7 +277,7 @@ We can run this job using the `pyspark` script: $ cd $SPARK_HOME $ ./pyspark SimpleJob.py ... -Lines with a: 8422, Lines with b: 1836 +Lines with a: 46, Lines with b: 23 {% endhighlight python %} This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. From 1b169f190c5c5210d088faced86dee1007295ac8 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Apr 2013 19:52:12 -0700 Subject: [PATCH 489/696] Exclude old versions of Netty, which had a different Maven organization --- project/SparkBuild.scala | 35 +++++++++++++++++++---------------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f32c47e71f4db..7bd6c4c2355f3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -129,6 +129,9 @@ object SparkBuild extends Build { val slf4jVersion = "1.6.1" + val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") + val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( @@ -149,33 +152,33 @@ object SparkBuild extends Build { "asm" % "asm-all" % "3.3.1", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", - "com.typesafe.akka" % "akka-actor" % "2.0.3", - "com.typesafe.akka" % "akka-remote" % "2.0.3", - "com.typesafe.akka" % "akka-slf4j" % "2.0.3", + "com.typesafe.akka" % "akka-actor" % "2.0.3" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-remote" % "2.0.3" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-slf4j" % "2.0.3" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "cc.spray" % "spray-can" % "1.0-M2.1", - "cc.spray" % "spray-server" % "1.0-M2.1", - "cc.spray" % "spray-json_2.9.2" % "1.1.1", + "cc.spray" % "spray-can" % "1.0-M2.1" excludeAll(excludeNetty), + "cc.spray" % "spray-server" % "1.0-M2.1" excludeAll(excludeNetty), + "cc.spray" % "spray-json_2.9.2" % "1.1.1" excludeAll(excludeNetty), "org.apache.mesos" % "mesos" % "0.9.0-incubating" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), - "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), - "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), - "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ) + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) ) } else { Seq( - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ), - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ) + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) ) } } else { - Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll( ExclusionRule(organization = "org.codehaus.jackson") ) ) + Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) ) }), unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") { @@ -205,10 +208,10 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), "com.github.sgroschupf" % "zkclient" % "0.1", - "org.twitter4j" % "twitter4j-stream" % "3.0.3", - "com.typesafe.akka" % "akka-zeromq" % "2.0.3" + "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-zeromq" % "2.0.3" excludeAll(excludeNetty) ) ) ++ assemblySettings ++ extraAssemblySettings From c9c4954d994c5ba824e71c1c5cd8d5de531caf78 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 26 Apr 2013 16:57:46 -0700 Subject: [PATCH 490/696] Add an interface to zip iterators of multiple RDDs The current code supports 2, 3 or 4 arguments but can be extended to more arguments if required. --- core/src/main/scala/spark/RDD.scala | 22 ++++ .../spark/rdd/MapZippedPartitionsRDD.scala | 118 ++++++++++++++++++ .../spark/MapZippedPartitionsSuite.scala | 34 +++++ 3 files changed, 174 insertions(+) create mode 100644 core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala create mode 100644 core/src/test/scala/spark/MapZippedPartitionsSuite.scala diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index ccd9d0364ad95..8e7e1457c14a3 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -35,6 +35,9 @@ import spark.rdd.ShuffledRDD import spark.rdd.SubtractedRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD +import spark.rdd.MapZippedPartitionsRDD2 +import spark.rdd.MapZippedPartitionsRDD3 +import spark.rdd.MapZippedPartitionsRDD4 import spark.storage.StorageLevel import SparkContext._ @@ -436,6 +439,25 @@ abstract class RDD[T: ClassManifest]( */ def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) + def zipAndMapPartitions[B: ClassManifest, V: ClassManifest]( + f: (Iterator[T], Iterator[B]) => Iterator[V], + rdd2: RDD[B]) = + new MapZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) + + def zipAndMapPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]( + f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V], + rdd2: RDD[B], + rdd3: RDD[C]) = + new MapZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) + + def zipAndMapPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]( + f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], + rdd2: RDD[B], + rdd3: RDD[C], + rdd4: RDD[D]) = + new MapZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) + + // Actions (launch a job to return a value to the user program) /** diff --git a/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala new file mode 100644 index 0000000000000..6653b3b44451e --- /dev/null +++ b/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala @@ -0,0 +1,118 @@ +package spark.rdd + +import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import java.io.{ObjectOutputStream, IOException} + +private[spark] class MapZippedPartition( + idx: Int, + @transient rdds: Seq[RDD[_]] + ) extends Partition { + + override val index: Int = idx + var partitionValues = rdds.map(rdd => rdd.partitions(idx)) + def partitions = partitionValues + + @throws(classOf[IOException]) + private def writeObject(oos: ObjectOutputStream) { + // Update the reference to parent split at the time of task serialization + partitionValues = rdds.map(rdd => rdd.partitions(idx)) + oos.defaultWriteObject() + } +} + +abstract class MapZippedPartitionsBaseRDD[V: ClassManifest]( + sc: SparkContext, + var rdds: Seq[RDD[_]]) + extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) { + + override def getPartitions: Array[Partition] = { + val sizes = rdds.map(x => x.partitions.size) + if (!sizes.forall(x => x == sizes(0))) { + throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") + } + val array = new Array[Partition](sizes(0)) + for (i <- 0 until sizes(0)) { + array(i) = new MapZippedPartition(i, rdds) + } + array + } + + override def getPreferredLocations(s: Partition): Seq[String] = { + val splits = s.asInstanceOf[MapZippedPartition].partitions + val preferredLocations = rdds.zip(splits).map(x => x._1.preferredLocations(x._2)) + preferredLocations.reduce((x, y) => x.intersect(y)) + } + + override def clearDependencies() { + super.clearDependencies() + rdds = null + } +} + +class MapZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest]( + sc: SparkContext, + f: (Iterator[A], Iterator[B]) => Iterator[V], + var rdd1: RDD[A], + var rdd2: RDD[B]) + extends MapZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) { + + override def compute(s: Partition, context: TaskContext): Iterator[V] = { + val partitions = s.asInstanceOf[MapZippedPartition].partitions + f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context)) + } + + override def clearDependencies() { + super.clearDependencies() + rdd1 = null + rdd2 = null + } +} + +class MapZippedPartitionsRDD3[A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest]( + sc: SparkContext, + f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], + var rdd1: RDD[A], + var rdd2: RDD[B], + var rdd3: RDD[C]) + extends MapZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3)) { + + override def compute(s: Partition, context: TaskContext): Iterator[V] = { + val partitions = s.asInstanceOf[MapZippedPartition].partitions + f(rdd1.iterator(partitions(0), context), + rdd2.iterator(partitions(1), context), + rdd3.iterator(partitions(2), context)) + } + + override def clearDependencies() { + super.clearDependencies() + rdd1 = null + rdd2 = null + rdd3 = null + } +} + +class MapZippedPartitionsRDD4[A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest]( + sc: SparkContext, + f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], + var rdd1: RDD[A], + var rdd2: RDD[B], + var rdd3: RDD[C], + var rdd4: RDD[D]) + extends MapZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4)) { + + override def compute(s: Partition, context: TaskContext): Iterator[V] = { + val partitions = s.asInstanceOf[MapZippedPartition].partitions + f(rdd1.iterator(partitions(0), context), + rdd2.iterator(partitions(1), context), + rdd3.iterator(partitions(2), context), + rdd4.iterator(partitions(3), context)) + } + + override def clearDependencies() { + super.clearDependencies() + rdd1 = null + rdd2 = null + rdd3 = null + rdd4 = null + } +} diff --git a/core/src/test/scala/spark/MapZippedPartitionsSuite.scala b/core/src/test/scala/spark/MapZippedPartitionsSuite.scala new file mode 100644 index 0000000000000..f65a6464169e2 --- /dev/null +++ b/core/src/test/scala/spark/MapZippedPartitionsSuite.scala @@ -0,0 +1,34 @@ +package spark + +import scala.collection.immutable.NumericRange + +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import SparkContext._ + + +object MapZippedPartitionsSuite { + def procZippedData(i: Iterator[Int], s: Iterator[String], d: Iterator[Double]) : Iterator[Int] = { + Iterator(i.toArray.size, s.toArray.size, d.toArray.size) + } +} + +class MapZippedPartitionsSuite extends FunSuite with LocalSparkContext { + test("print sizes") { + sc = new SparkContext("local", "test") + val data1 = sc.makeRDD(Array(1, 2, 3, 4), 2) + val data2 = sc.makeRDD(Array("1", "2", "3", "4", "5", "6"), 2) + val data3 = sc.makeRDD(Array(1.0, 2.0), 2) + + val zippedRDD = data1.zipAndMapPartitions(MapZippedPartitionsSuite.procZippedData, data2, data3) + + val obtainedSizes = zippedRDD.collect() + val expectedSizes = Array(2, 3, 1, 2, 3, 1) + assert(obtainedSizes.size == 6) + assert(obtainedSizes.zip(expectedSizes).forall(x => x._1 == x._2)) + } +} From 0cc6642b7c6fbb4167956b668603f2ea6fb5ac8e Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 28 Apr 2013 05:11:03 -0700 Subject: [PATCH 491/696] Rename to zipPartitions and style changes --- core/src/main/scala/spark/RDD.scala | 24 +++++++++---------- .../spark/rdd/MapZippedPartitionsRDD.scala | 22 +++++++++-------- .../spark/MapZippedPartitionsSuite.scala | 2 +- 3 files changed, 25 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8e7e1457c14a3..bded55238f8ce 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -439,22 +439,22 @@ abstract class RDD[T: ClassManifest]( */ def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) - def zipAndMapPartitions[B: ClassManifest, V: ClassManifest]( - f: (Iterator[T], Iterator[B]) => Iterator[V], - rdd2: RDD[B]) = + def zipPartitions[B: ClassManifest, V: ClassManifest]( + f: (Iterator[T], Iterator[B]) => Iterator[V], + rdd2: RDD[B]) = new MapZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) - def zipAndMapPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]( - f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V], - rdd2: RDD[B], - rdd3: RDD[C]) = + def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]( + f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V], + rdd2: RDD[B], + rdd3: RDD[C]) = new MapZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) - def zipAndMapPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]( - f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], - rdd2: RDD[B], - rdd3: RDD[C], - rdd4: RDD[D]) = + def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]( + f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], + rdd2: RDD[B], + rdd3: RDD[C], + rdd4: RDD[D]) = new MapZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) diff --git a/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala index 6653b3b44451e..3520fd24b0fd6 100644 --- a/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala @@ -4,13 +4,13 @@ import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class MapZippedPartition( - idx: Int, - @transient rdds: Seq[RDD[_]] - ) extends Partition { + idx: Int, + @transient rdds: Seq[RDD[_]]) + extends Partition { override val index: Int = idx var partitionValues = rdds.map(rdd => rdd.partitions(idx)) - def partitions = partitionValues + def partitions = partitionValues @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { @@ -68,7 +68,8 @@ class MapZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManife } } -class MapZippedPartitionsRDD3[A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest]( +class MapZippedPartitionsRDD3 + [A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], var rdd1: RDD[A], @@ -78,8 +79,8 @@ class MapZippedPartitionsRDD3[A: ClassManifest, B: ClassManifest, C: ClassManife override def compute(s: Partition, context: TaskContext): Iterator[V] = { val partitions = s.asInstanceOf[MapZippedPartition].partitions - f(rdd1.iterator(partitions(0), context), - rdd2.iterator(partitions(1), context), + f(rdd1.iterator(partitions(0), context), + rdd2.iterator(partitions(1), context), rdd3.iterator(partitions(2), context)) } @@ -91,7 +92,8 @@ class MapZippedPartitionsRDD3[A: ClassManifest, B: ClassManifest, C: ClassManife } } -class MapZippedPartitionsRDD4[A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest]( +class MapZippedPartitionsRDD4 + [A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], var rdd1: RDD[A], @@ -102,8 +104,8 @@ class MapZippedPartitionsRDD4[A: ClassManifest, B: ClassManifest, C: ClassManife override def compute(s: Partition, context: TaskContext): Iterator[V] = { val partitions = s.asInstanceOf[MapZippedPartition].partitions - f(rdd1.iterator(partitions(0), context), - rdd2.iterator(partitions(1), context), + f(rdd1.iterator(partitions(0), context), + rdd2.iterator(partitions(1), context), rdd3.iterator(partitions(2), context), rdd4.iterator(partitions(3), context)) } diff --git a/core/src/test/scala/spark/MapZippedPartitionsSuite.scala b/core/src/test/scala/spark/MapZippedPartitionsSuite.scala index f65a6464169e2..834b517cbc000 100644 --- a/core/src/test/scala/spark/MapZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/MapZippedPartitionsSuite.scala @@ -24,7 +24,7 @@ class MapZippedPartitionsSuite extends FunSuite with LocalSparkContext { val data2 = sc.makeRDD(Array("1", "2", "3", "4", "5", "6"), 2) val data3 = sc.makeRDD(Array(1.0, 2.0), 2) - val zippedRDD = data1.zipAndMapPartitions(MapZippedPartitionsSuite.procZippedData, data2, data3) + val zippedRDD = data1.zipPartitions(MapZippedPartitionsSuite.procZippedData, data2, data3) val obtainedSizes = zippedRDD.collect() val expectedSizes = Array(2, 3, 1, 2, 3, 1) From afee9024430ef79cc0840a5e5788b60c8c53f9d2 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sun, 28 Apr 2013 22:26:45 +0530 Subject: [PATCH 492/696] Attempt to fix streaming test failures after yarn branch merge --- bagel/src/test/scala/bagel/BagelSuite.scala | 1 + .../test/scala/spark/LocalSparkContext.scala | 3 +- .../src/test/scala/spark/repl/ReplSuite.scala | 1 + .../scala/spark/streaming/Checkpoint.scala | 30 ++++++++++++++----- .../streaming/util/MasterFailureTest.scala | 8 ++++- .../streaming/BasicOperationsSuite.scala | 1 + .../spark/streaming/CheckpointSuite.scala | 4 ++- .../scala/spark/streaming/FailureSuite.scala | 2 ++ .../spark/streaming/InputStreamsSuite.scala | 1 + .../streaming/WindowOperationsSuite.scala | 1 + 10 files changed, 42 insertions(+), 10 deletions(-) diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala index 25db395c22128..a09c97806869e 100644 --- a/bagel/src/test/scala/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/bagel/BagelSuite.scala @@ -23,6 +23,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") } test("halting by voting") { diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index ff00dd05dd787..76d5258b02b97 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -27,6 +27,7 @@ object LocalSparkContext { sc.stop() // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") } /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ @@ -38,4 +39,4 @@ object LocalSparkContext { } } -} \ No newline at end of file +} diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index 43559b96d3a80..1c64f9b98d099 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -32,6 +32,7 @@ class ReplSuite extends FunSuite { interp.sparkContext.stop() // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") return out.toString } diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index e303e33e5e401..7bd104b8d5b94 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -38,28 +38,43 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) private[streaming] class CheckpointWriter(checkpointDir: String) extends Logging { val file = new Path(checkpointDir, "graph") + // The file to which we actually write - and then "move" to file. + private val writeFile = new Path(file.getParent, file.getName + ".next") + private val bakFile = new Path(file.getParent, file.getName + ".bk") + + @volatile private var stopped = false + val conf = new Configuration() var fs = file.getFileSystem(conf) val maxAttempts = 3 val executor = Executors.newFixedThreadPool(1) + // Removed code which validates whether there is only one CheckpointWriter per path 'file' since + // I did not notice any errors - reintroduce it ? + class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable { def run() { var attempts = 0 val startTime = System.currentTimeMillis() while (attempts < maxAttempts) { + if (stopped) { + logInfo("Already stopped, ignore checkpoint attempt for " + file) + return + } attempts += 1 try { logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'") - if (fs.exists(file)) { - val bkFile = new Path(file.getParent, file.getName + ".bk") - FileUtil.copy(fs, file, fs, bkFile, true, true, conf) - logDebug("Moved existing checkpoint file to " + bkFile) - } - val fos = fs.create(file) + // This is inherently thread unsafe .. so alleviating it by writing to '.new' and then doing moves : which should be pretty fast. + val fos = fs.create(writeFile) fos.write(bytes) fos.close() - fos.close() + if (fs.exists(file) && fs.rename(file, bakFile)) { + logDebug("Moved existing checkpoint file to " + bakFile) + } + // paranoia + fs.delete(file, false) + fs.rename(writeFile, file) + val finishTime = System.currentTimeMillis(); logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds") @@ -84,6 +99,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging { } def stop() { + stopped = true executor.shutdown() } } diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index f673e5be15485..e7a3f92bc0bb5 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -74,6 +74,7 @@ object MasterFailureTest extends Logging { val operation = (st: DStream[String]) => { val updateFunc = (values: Seq[Long], state: Option[Long]) => { + logInfo("UpdateFunc .. state = " + state.getOrElse(0L) + ", values = " + values) Some(values.foldLeft(0L)(_ + _) + state.getOrElse(0L)) } st.flatMap(_.split(" ")) @@ -159,6 +160,7 @@ object MasterFailureTest extends Logging { // Setup the streaming computation with the given operation System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map()) ssc.checkpoint(checkpointDir.toString) val inputStream = ssc.textFileStream(testDir.toString) @@ -205,6 +207,7 @@ object MasterFailureTest extends Logging { // (iii) Its not timed out yet System.clearProperty("spark.streaming.clock") System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") ssc.start() val startTime = System.currentTimeMillis() while (!killed && !isLastOutputGenerated && !isTimedOut) { @@ -357,13 +360,16 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) // Write the data to a local file and then move it to the target test directory val localFile = new File(localTestDir, (i+1).toString) val hadoopFile = new Path(testDir, (i+1).toString) + val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString) FileUtils.writeStringToFile(localFile, input(i).toString + "\n") var tries = 0 var done = false while (!done && tries < maxTries) { tries += 1 try { - fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) + // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) + fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile) + fs.rename(tempHadoopFile, hadoopFile) done = true } catch { case ioe: IOException => { diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index cf2ed8b1d4bef..e7352deb81da3 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -15,6 +15,7 @@ class BasicOperationsSuite extends TestSuiteBase { after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") } test("map") { diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index cac86deeaf349..607dea77ec24b 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -31,6 +31,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") } var ssc: StreamingContext = null @@ -325,6 +326,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ) ssc = new StreamingContext(checkpointDir) System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") ssc.start() val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches) // the first element will be re-processed data of the last batch before restart @@ -350,4 +352,4 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]] outputStream.output } -} \ No newline at end of file +} diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index a5fa7ab92dd6b..4529e774e9b0a 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -22,10 +22,12 @@ class FailureSuite extends FunSuite with BeforeAndAfter with Logging { val batchDuration = Milliseconds(1000) before { + logInfo("BEFORE ...") FileUtils.deleteDirectory(new File(directory)) } after { + logInfo("AFTER ...") FileUtils.deleteDirectory(new File(directory)) } diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 67dca2ac31215..0acb6db6f2e48 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -41,6 +41,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") } diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index 1b66f3bda20ad..80d827706f63c 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -16,6 +16,7 @@ class WindowOperationsSuite extends TestSuiteBase { after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") } val largerSlideInput = Seq( From 7fa6978a1e8822cf377fbb1e8a8d23adc4ebe12e Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sun, 28 Apr 2013 23:08:10 +0530 Subject: [PATCH 493/696] Allow CheckpointWriter pending tasks to finish --- .../src/main/scala/spark/streaming/Checkpoint.scala | 13 +++++++------ .../main/scala/spark/streaming/DStreamGraph.scala | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 7bd104b8d5b94..4bbad908d007a 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -42,7 +42,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging { private val writeFile = new Path(file.getParent, file.getName + ".next") private val bakFile = new Path(file.getParent, file.getName + ".bk") - @volatile private var stopped = false + private var stopped = false val conf = new Configuration() var fs = file.getFileSystem(conf) @@ -57,10 +57,6 @@ class CheckpointWriter(checkpointDir: String) extends Logging { var attempts = 0 val startTime = System.currentTimeMillis() while (attempts < maxAttempts) { - if (stopped) { - logInfo("Already stopped, ignore checkpoint attempt for " + file) - return - } attempts += 1 try { logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'") @@ -99,8 +95,13 @@ class CheckpointWriter(checkpointDir: String) extends Logging { } def stop() { - stopped = true + synchronized { + if (stopped) return ; + stopped = true + } executor.shutdown() + val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS) + logInfo("CheckpointWriter executor terminated ? " + terminated) } } diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index adb7f3a24d25f..3b331956f5973 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -54,8 +54,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { throw new Exception("Batch duration already set as " + batchDuration + ". cannot set it again.") } + batchDuration = duration } - batchDuration = duration } def remember(duration: Duration) { From 9bd439502e371e1ff9d6184c7182bc414104e39e Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sun, 28 Apr 2013 23:09:08 +0530 Subject: [PATCH 494/696] Remove spurious commit --- project/SparkBuild.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7c004df6fb8b2..7bd6c4c2355f3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -56,6 +56,9 @@ object SparkBuild extends Build { fork := true, javaOptions += "-Xmx2g", + // Only allow one test at a time, even across projects, since they run in the same JVM + concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), + // Shared between both core and streaming. resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), From 3a89a76b874298853cf47510ab33e863abf117d7 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Mon, 29 Apr 2013 00:04:12 +0530 Subject: [PATCH 495/696] Make log message more descriptive to aid in debugging --- streaming/src/main/scala/spark/streaming/Checkpoint.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 4bbad908d007a..66e67cbfa1d78 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -100,8 +100,10 @@ class CheckpointWriter(checkpointDir: String) extends Logging { stopped = true } executor.shutdown() + val startTime = System.currentTimeMillis() val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS) - logInfo("CheckpointWriter executor terminated ? " + terminated) + val endTime = System.currentTimeMillis() + logInfo("CheckpointWriter executor terminated ? " + terminated + ", waited for " + (endTime - startTime) + " ms.") } } From 430c531464a5372237c97394f8f4b6ec344394c0 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Mon, 29 Apr 2013 00:24:30 +0530 Subject: [PATCH 496/696] Remove debug statements --- .../src/main/scala/spark/streaming/util/MasterFailureTest.scala | 1 - streaming/src/test/scala/spark/streaming/FailureSuite.scala | 2 -- 2 files changed, 3 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index e7a3f92bc0bb5..426a9b6f71d88 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -74,7 +74,6 @@ object MasterFailureTest extends Logging { val operation = (st: DStream[String]) => { val updateFunc = (values: Seq[Long], state: Option[Long]) => { - logInfo("UpdateFunc .. state = " + state.getOrElse(0L) + ", values = " + values) Some(values.foldLeft(0L)(_ + _) + state.getOrElse(0L)) } st.flatMap(_.split(" ")) diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index 4529e774e9b0a..a5fa7ab92dd6b 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -22,12 +22,10 @@ class FailureSuite extends FunSuite with BeforeAndAfter with Logging { val batchDuration = Milliseconds(1000) before { - logInfo("BEFORE ...") FileUtils.deleteDirectory(new File(directory)) } after { - logInfo("AFTER ...") FileUtils.deleteDirectory(new File(directory)) } From 6e84635ab904ee2798f1d6acd3a8ed5e01563e54 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 28 Apr 2013 15:58:40 -0700 Subject: [PATCH 497/696] Rename classes from MapZipped* to Zipped* --- .../{MapZippedPartitionsRDD.scala => ZippedPartitionsRDD.scala} | 0 ...MapZippedPartitionsSuite.scala => ZippedPartitionsSuite.scala} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename core/src/main/scala/spark/rdd/{MapZippedPartitionsRDD.scala => ZippedPartitionsRDD.scala} (100%) rename core/src/test/scala/spark/{MapZippedPartitionsSuite.scala => ZippedPartitionsSuite.scala} (100%) diff --git a/core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala similarity index 100% rename from core/src/main/scala/spark/rdd/MapZippedPartitionsRDD.scala rename to core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala diff --git a/core/src/test/scala/spark/MapZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala similarity index 100% rename from core/src/test/scala/spark/MapZippedPartitionsSuite.scala rename to core/src/test/scala/spark/ZippedPartitionsSuite.scala From 15acd49f07c3cde0a381f4abe139b17791a910b4 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 28 Apr 2013 16:03:22 -0700 Subject: [PATCH 498/696] Actually rename classes to ZippedPartitions* (the previous commit only renamed the file) --- core/src/main/scala/spark/RDD.scala | 18 ++++++------- .../scala/spark/rdd/ZippedPartitionsRDD.scala | 26 +++++++++---------- .../scala/spark/ZippedPartitionsSuite.scala | 6 ++--- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index bded55238f8ce..4310f745f37d4 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -35,9 +35,9 @@ import spark.rdd.ShuffledRDD import spark.rdd.SubtractedRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD -import spark.rdd.MapZippedPartitionsRDD2 -import spark.rdd.MapZippedPartitionsRDD3 -import spark.rdd.MapZippedPartitionsRDD4 +import spark.rdd.ZippedPartitionsRDD2 +import spark.rdd.ZippedPartitionsRDD3 +import spark.rdd.ZippedPartitionsRDD4 import spark.storage.StorageLevel import SparkContext._ @@ -441,21 +441,21 @@ abstract class RDD[T: ClassManifest]( def zipPartitions[B: ClassManifest, V: ClassManifest]( f: (Iterator[T], Iterator[B]) => Iterator[V], - rdd2: RDD[B]) = - new MapZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) + rdd2: RDD[B]): RDD[V] = + new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]( f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V], rdd2: RDD[B], - rdd3: RDD[C]) = - new MapZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) + rdd3: RDD[C]): RDD[V] = + new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]( f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], rdd2: RDD[B], rdd3: RDD[C], - rdd4: RDD[D]) = - new MapZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) + rdd4: RDD[D]): RDD[V] = + new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) // Actions (launch a job to return a value to the user program) diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index 3520fd24b0fd6..b3113c1969182 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -3,7 +3,7 @@ package spark.rdd import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} -private[spark] class MapZippedPartition( +private[spark] class ZippedPartitions( idx: Int, @transient rdds: Seq[RDD[_]]) extends Partition { @@ -20,7 +20,7 @@ private[spark] class MapZippedPartition( } } -abstract class MapZippedPartitionsBaseRDD[V: ClassManifest]( +abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( sc: SparkContext, var rdds: Seq[RDD[_]]) extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) { @@ -32,13 +32,13 @@ abstract class MapZippedPartitionsBaseRDD[V: ClassManifest]( } val array = new Array[Partition](sizes(0)) for (i <- 0 until sizes(0)) { - array(i) = new MapZippedPartition(i, rdds) + array(i) = new ZippedPartitions(i, rdds) } array } override def getPreferredLocations(s: Partition): Seq[String] = { - val splits = s.asInstanceOf[MapZippedPartition].partitions + val splits = s.asInstanceOf[ZippedPartitions].partitions val preferredLocations = rdds.zip(splits).map(x => x._1.preferredLocations(x._2)) preferredLocations.reduce((x, y) => x.intersect(y)) } @@ -49,15 +49,15 @@ abstract class MapZippedPartitionsBaseRDD[V: ClassManifest]( } } -class MapZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest]( +class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest]( sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B]) - extends MapZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) { + extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { - val partitions = s.asInstanceOf[MapZippedPartition].partitions + val partitions = s.asInstanceOf[ZippedPartitions].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context)) } @@ -68,17 +68,17 @@ class MapZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManife } } -class MapZippedPartitionsRDD3 +class ZippedPartitionsRDD3 [A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], var rdd3: RDD[C]) - extends MapZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3)) { + extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3)) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { - val partitions = s.asInstanceOf[MapZippedPartition].partitions + val partitions = s.asInstanceOf[ZippedPartitions].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context), rdd3.iterator(partitions(2), context)) @@ -92,7 +92,7 @@ class MapZippedPartitionsRDD3 } } -class MapZippedPartitionsRDD4 +class ZippedPartitionsRDD4 [A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], @@ -100,10 +100,10 @@ class MapZippedPartitionsRDD4 var rdd2: RDD[B], var rdd3: RDD[C], var rdd4: RDD[D]) - extends MapZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4)) { + extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4)) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { - val partitions = s.asInstanceOf[MapZippedPartition].partitions + val partitions = s.asInstanceOf[ZippedPartitions].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context), rdd3.iterator(partitions(2), context), diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala index 834b517cbc000..5f60aa75d7f03 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala @@ -11,20 +11,20 @@ import org.scalacheck.Prop._ import SparkContext._ -object MapZippedPartitionsSuite { +object ZippedPartitionsSuite { def procZippedData(i: Iterator[Int], s: Iterator[String], d: Iterator[Double]) : Iterator[Int] = { Iterator(i.toArray.size, s.toArray.size, d.toArray.size) } } -class MapZippedPartitionsSuite extends FunSuite with LocalSparkContext { +class ZippedPartitionsSuite extends FunSuite with LocalSparkContext { test("print sizes") { sc = new SparkContext("local", "test") val data1 = sc.makeRDD(Array(1, 2, 3, 4), 2) val data2 = sc.makeRDD(Array("1", "2", "3", "4", "5", "6"), 2) val data3 = sc.makeRDD(Array(1.0, 2.0), 2) - val zippedRDD = data1.zipPartitions(MapZippedPartitionsSuite.procZippedData, data2, data3) + val zippedRDD = data1.zipPartitions(ZippedPartitionsSuite.procZippedData, data2, data3) val obtainedSizes = zippedRDD.collect() val expectedSizes = Array(2, 3, 1, 2, 3, 1) From 604d3bf56ce2f77ad391b10842ec1c51daf91a97 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 28 Apr 2013 16:31:07 -0700 Subject: [PATCH 499/696] Rename partition class and add scala doc --- core/src/main/scala/spark/RDD.scala | 6 ++++++ .../main/scala/spark/rdd/ZippedPartitionsRDD.scala | 12 ++++++------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 4310f745f37d4..09e52ebf3ec00 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -439,6 +439,12 @@ abstract class RDD[T: ClassManifest]( */ def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) + /** + * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by + * applying a function to the zipped partitions. Assumes that all the RDDs have the + * *same number of partitions*, but does *not* require them to have the same number + * of elements in each partition. + */ def zipPartitions[B: ClassManifest, V: ClassManifest]( f: (Iterator[T], Iterator[B]) => Iterator[V], rdd2: RDD[B]): RDD[V] = diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index b3113c1969182..fc3f29ffcda3e 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -3,7 +3,7 @@ package spark.rdd import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} -private[spark] class ZippedPartitions( +private[spark] class ZippedPartitionsPartition( idx: Int, @transient rdds: Seq[RDD[_]]) extends Partition { @@ -32,13 +32,13 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( } val array = new Array[Partition](sizes(0)) for (i <- 0 until sizes(0)) { - array(i) = new ZippedPartitions(i, rdds) + array(i) = new ZippedPartitionsPartition(i, rdds) } array } override def getPreferredLocations(s: Partition): Seq[String] = { - val splits = s.asInstanceOf[ZippedPartitions].partitions + val splits = s.asInstanceOf[ZippedPartitionsPartition].partitions val preferredLocations = rdds.zip(splits).map(x => x._1.preferredLocations(x._2)) preferredLocations.reduce((x, y) => x.intersect(y)) } @@ -57,7 +57,7 @@ class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest] extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { - val partitions = s.asInstanceOf[ZippedPartitions].partitions + val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context)) } @@ -78,7 +78,7 @@ class ZippedPartitionsRDD3 extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3)) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { - val partitions = s.asInstanceOf[ZippedPartitions].partitions + val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context), rdd3.iterator(partitions(2), context)) @@ -103,7 +103,7 @@ class ZippedPartitionsRDD4 extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4)) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { - val partitions = s.asInstanceOf[ZippedPartitions].partitions + val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context), rdd3.iterator(partitions(2), context), From bce4089f22f5e17811f63368b164fae66774095f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Apr 2013 22:23:48 -0700 Subject: [PATCH 500/696] Fix BlockManagerSuite to deal with clearing spark.hostPort --- .../test/scala/spark/storage/BlockManagerSuite.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index b8c0f6fb763a9..77f444bcad4fb 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -15,6 +15,8 @@ import org.scalatest.time.SpanSugar._ import spark.JavaSerializer import spark.KryoSerializer import spark.SizeEstimator +import spark.Utils +import spark.util.AkkaUtils import spark.util.ByteBufferInputStream class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { @@ -31,7 +33,12 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val serializer = new KryoSerializer before { - actorSystem = ActorSystem("test") + val hostname = Utils.localHostName + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", hostname, 0) + this.actorSystem = actorSystem + System.setProperty("spark.driver.port", boundPort.toString) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + master = new BlockManagerMaster( actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true)))) @@ -44,6 +51,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } after { + System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") + if (store != null) { store.stop() store = null From 0f45347c7b7243dbf54569f057a3605f96d614af Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Apr 2013 22:29:27 -0700 Subject: [PATCH 501/696] More unit test fixes --- core/src/test/scala/spark/MapOutputTrackerSuite.scala | 3 +++ core/src/test/scala/spark/storage/BlockManagerSuite.scala | 5 ++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 3abc584b6a177..e95818db61028 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -81,6 +81,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) + System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext + System.setProperty("spark.hostPort", "localhost:" + boundPort) + val masterTracker = new MapOutputTracker() masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 77f444bcad4fb..5a11a4483bb1c 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -33,11 +33,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val serializer = new KryoSerializer before { - val hostname = Utils.localHostName - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", hostname, 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0) this.actorSystem = actorSystem System.setProperty("spark.driver.port", boundPort.toString) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + System.setProperty("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true)))) From 224fbac0612d5c35259cc9f4963dcd4a65ecc832 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Apr 2013 10:10:14 -0700 Subject: [PATCH 502/696] Spark-742: TaskMetrics should not employ per-record timing. This patch does three things: 1. Makes TimedIterator a trait with two implementations (one a no-op) 2. Makes the default behavior to use the no-op implementation 3. Removes DelegateBlockFetchTracker. This is just cleanup, but it seems like the triat doesn't really reduce complexity in any way. In the future we can add other implementations, e.g. ones which perform sampling. --- .../spark/BlockStoreShuffleFetcher.scala | 23 +++++++++------- .../scala/spark/executor/TaskMetrics.scala | 2 +- .../storage/DelegateBlockFetchTracker.scala | 12 --------- .../main/scala/spark/util/TimedIterator.scala | 26 ++++++++++++++++--- .../spark/scheduler/SparkListenerSuite.scala | 2 +- 5 files changed, 37 insertions(+), 28 deletions(-) delete mode 100644 core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index c27ed36406119..83c22b1f144c6 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -4,8 +4,8 @@ import executor.{ShuffleReadMetrics, TaskMetrics} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} -import util.{CompletionIterator, TimedIterator} +import spark.storage.BlockManagerId +import util.{NoOpTimedIterator, SystemTimedIterator, CompletionIterator, TimedIterator} private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { @@ -49,17 +49,20 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } val blockFetcherItr = blockManager.getMultiple(blocksByAddress) - val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker - itr.setDelegate(blockFetcherItr) + val itr = if (System.getProperty("per.record.shuffle.metrics", "false").toBoolean) { + new SystemTimedIterator(blockFetcherItr.flatMap(unpackBlock)) + } else { + new NoOpTimedIterator(blockFetcherItr.flatMap(unpackBlock)) + } CompletionIterator[(K,V), Iterator[(K,V)]](itr, { val shuffleMetrics = new ShuffleReadMetrics shuffleMetrics.shuffleReadMillis = itr.getNetMillis - shuffleMetrics.remoteFetchTime = itr.remoteFetchTime - shuffleMetrics.fetchWaitTime = itr.fetchWaitTime - shuffleMetrics.remoteBytesRead = itr.remoteBytesRead - shuffleMetrics.totalBlocksFetched = itr.totalBlocks - shuffleMetrics.localBlocksFetched = itr.numLocalBlocks - shuffleMetrics.remoteBlocksFetched = itr.numRemoteBlocks + shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime + shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime + shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead + shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks + shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks + shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks metrics.shuffleReadMetrics = Some(shuffleMetrics) }) } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 93bbb6b4587a5..45f6d43971119 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -51,7 +51,7 @@ class ShuffleReadMetrics extends Serializable { /** * Total time to read shuffle data */ - var shuffleReadMillis: Long = _ + var shuffleReadMillis: Option[Long] = _ /** * Total time that is spent blocked waiting for shuffle to fetch data diff --git a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala deleted file mode 100644 index f6c28dce52ad5..0000000000000 --- a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala +++ /dev/null @@ -1,12 +0,0 @@ -package spark.storage - -private[spark] trait DelegateBlockFetchTracker extends BlockFetchTracker { - var delegate : BlockFetchTracker = _ - def setDelegate(d: BlockFetchTracker) {delegate = d} - def totalBlocks = delegate.totalBlocks - def numLocalBlocks = delegate.numLocalBlocks - def numRemoteBlocks = delegate.numRemoteBlocks - def remoteFetchTime = delegate.remoteFetchTime - def fetchWaitTime = delegate.fetchWaitTime - def remoteBytesRead = delegate.remoteBytesRead -} diff --git a/core/src/main/scala/spark/util/TimedIterator.scala b/core/src/main/scala/spark/util/TimedIterator.scala index 539b01f4ce47d..49f1276b4e193 100644 --- a/core/src/main/scala/spark/util/TimedIterator.scala +++ b/core/src/main/scala/spark/util/TimedIterator.scala @@ -1,13 +1,21 @@ package spark.util /** - * A utility for tracking the total time an iterator takes to iterate through its elements. + * A utility for tracking the the time an iterator takes to iterate through its elements. + */ +trait TimedIterator { + def getNetMillis: Option[Long] + def getAverageTimePerItem: Option[Double] +} + +/** + * A TimedIterator which uses System.currentTimeMillis() on every call to next(). * * In general, this should only be used if you expect it to take a considerable amount of time * (eg. milliseconds) to get each element -- otherwise, the timing won't be very accurate, * and you are probably just adding more overhead */ -class TimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] { +class SystemTimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] with TimedIterator { private var netMillis = 0l private var nElems = 0 def hasNext = { @@ -26,7 +34,17 @@ class TimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] { r } - def getNetMillis = netMillis - def getAverageTimePerItem = netMillis / nElems.toDouble + def getNetMillis = Some(netMillis) + def getAverageTimePerItem = Some(netMillis / nElems.toDouble) } + +/** + * A TimedIterator which doesn't perform any timing measurements. + */ +class NoOpTimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] with TimedIterator { + def hasNext = sub.hasNext + def next = sub.next + def getNetMillis = None + def getAverageTimePerItem = None +} diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index 2f5af10e69c7f..5ccab369db214 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -57,7 +57,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) - sm.shuffleReadMillis should be > (0l) + sm.shuffleReadMillis.get should be > (0l) sm.localBlocksFetched should be > (0) sm.remoteBlocksFetched should be (0) sm.remoteBytesRead should be (0l) From 540be6b1544d26c7db79ec84a98fc6696c7c6434 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Apr 2013 11:32:07 -0700 Subject: [PATCH 503/696] Modified version of the fix which just removes all per-record tracking. --- .../spark/BlockStoreShuffleFetcher.scala | 9 +--- .../scala/spark/executor/TaskMetrics.scala | 5 -- .../main/scala/spark/util/TimedIterator.scala | 50 ------------------- .../spark/scheduler/SparkListenerSuite.scala | 1 - 4 files changed, 2 insertions(+), 63 deletions(-) delete mode 100644 core/src/main/scala/spark/util/TimedIterator.scala diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 83c22b1f144c6..ce61d27448b37 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import spark.storage.BlockManagerId -import util.{NoOpTimedIterator, SystemTimedIterator, CompletionIterator, TimedIterator} +import util.CompletionIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { @@ -49,14 +49,9 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } val blockFetcherItr = blockManager.getMultiple(blocksByAddress) - val itr = if (System.getProperty("per.record.shuffle.metrics", "false").toBoolean) { - new SystemTimedIterator(blockFetcherItr.flatMap(unpackBlock)) - } else { - new NoOpTimedIterator(blockFetcherItr.flatMap(unpackBlock)) - } + val itr = blockFetcherItr.flatMap(unpackBlock) CompletionIterator[(K,V), Iterator[(K,V)]](itr, { val shuffleMetrics = new ShuffleReadMetrics - shuffleMetrics.shuffleReadMillis = itr.getNetMillis shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 45f6d43971119..a7c56c237199b 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -48,11 +48,6 @@ class ShuffleReadMetrics extends Serializable { */ var localBlocksFetched: Int = _ - /** - * Total time to read shuffle data - */ - var shuffleReadMillis: Option[Long] = _ - /** * Total time that is spent blocked waiting for shuffle to fetch data */ diff --git a/core/src/main/scala/spark/util/TimedIterator.scala b/core/src/main/scala/spark/util/TimedIterator.scala deleted file mode 100644 index 49f1276b4e193..0000000000000 --- a/core/src/main/scala/spark/util/TimedIterator.scala +++ /dev/null @@ -1,50 +0,0 @@ -package spark.util - -/** - * A utility for tracking the the time an iterator takes to iterate through its elements. - */ -trait TimedIterator { - def getNetMillis: Option[Long] - def getAverageTimePerItem: Option[Double] -} - -/** - * A TimedIterator which uses System.currentTimeMillis() on every call to next(). - * - * In general, this should only be used if you expect it to take a considerable amount of time - * (eg. milliseconds) to get each element -- otherwise, the timing won't be very accurate, - * and you are probably just adding more overhead - */ -class SystemTimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] with TimedIterator { - private var netMillis = 0l - private var nElems = 0 - def hasNext = { - val start = System.currentTimeMillis() - val r = sub.hasNext - val end = System.currentTimeMillis() - netMillis += (end - start) - r - } - def next = { - val start = System.currentTimeMillis() - val r = sub.next - val end = System.currentTimeMillis() - netMillis += (end - start) - nElems += 1 - r - } - - def getNetMillis = Some(netMillis) - def getAverageTimePerItem = Some(netMillis / nElems.toDouble) - -} - -/** - * A TimedIterator which doesn't perform any timing measurements. - */ -class NoOpTimedIterator[+A](val sub: Iterator[A]) extends Iterator[A] with TimedIterator { - def hasNext = sub.hasNext - def next = sub.next - def getNetMillis = None - def getAverageTimePerItem = None -} diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index 5ccab369db214..42a87d8b90fe5 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -57,7 +57,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) - sm.shuffleReadMillis.get should be > (0l) sm.localBlocksFetched should be > (0) sm.remoteBlocksFetched should be (0) sm.remoteBytesRead should be (0l) From 016ce1fa9c9ebbe45559b1cbd95a3674510fe880 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Apr 2013 12:02:27 -0700 Subject: [PATCH 504/696] Using full package name for util --- core/src/main/scala/spark/BlockStoreShuffleFetcher.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index ce61d27448b37..2987dbbe5870e 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import spark.storage.BlockManagerId -import util.CompletionIterator +import spark.util.CompletionIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { From f1f92c88eb2960a16d33bf7dd291c8ce58f665de Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Apr 2013 17:08:45 -0700 Subject: [PATCH 505/696] Build against Hadoop 1 by default --- project/SparkBuild.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7bd6c4c2355f3..f2410085d8941 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -11,8 +11,9 @@ import twirl.sbt.TwirlPlugin._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - //val HADOOP_VERSION = "1.0.4" - //val HADOOP_MAJOR_VERSION = "1" + val HADOOP_VERSION = "1.0.4" + val HADOOP_MAJOR_VERSION = "1" + val HADOOP_YARN = false // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" @@ -20,10 +21,9 @@ object SparkBuild extends Build { //val HADOOP_YARN = false // For Hadoop 2 YARN support - // val HADOOP_VERSION = "0.23.7" - val HADOOP_VERSION = "2.0.2-alpha" - val HADOOP_MAJOR_VERSION = "2" - val HADOOP_YARN = true + //val HADOOP_VERSION = "2.0.2-alpha" + //val HADOOP_MAJOR_VERSION = "2" + //val HADOOP_YARN = true lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming) From 7007201201981c6fb002e3008d97a6d6248f4dba Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Apr 2013 23:07:03 -0700 Subject: [PATCH 506/696] Added a shuffle block manager so it is easier in the future to consolidate shuffle output files. --- .../spark/scheduler/ShuffleMapTask.scala | 50 +++++++++----- .../scala/spark/storage/BlockManager.scala | 16 +++-- .../spark/storage/BlockObjectWriter.scala | 43 +++++++++--- .../main/scala/spark/storage/DiskStore.scala | 65 +++++++++++++------ .../spark/storage/ShuffleBlockManager.scala | 52 +++++++++++++++ 5 files changed, 175 insertions(+), 51 deletions(-) create mode 100644 core/src/main/scala/spark/storage/ShuffleBlockManager.scala diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 51ec89eb7448a..124d2d7e26129 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -86,8 +86,14 @@ private[spark] class ShuffleMapTask( protected def this() = this(0, null, null, 0, null) - // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. - private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + // Data locality is on a per host basis, not hyper specific to container (host:port). + // Unique on set of hosts. + // TODO(rxin): The above statement seems problematic. Even if partitions are on the same host, + // the worker would still need to serialize / deserialize those data when they are in + // different jvm processes. Often that is very costly ... + @transient + private val preferredLocs: Seq[String] = + if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq { // DEBUG code @@ -131,31 +137,32 @@ private[spark] class ShuffleMapTask( val taskContext = new TaskContext(stageId, partition, attemptId) metrics = Some(taskContext.taskMetrics) + + val blockManager = SparkEnv.get.blockManager + var shuffle: ShuffleBlockManager#Shuffle = null + var buckets: ShuffleWriterGroup = null + try { // Obtain all the block writers for shuffle blocks. - val blockManager = SparkEnv.get.blockManager - val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => - val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId - blockManager.getDiskBlockWriter(blockId, Serializer.get(dep.serializerClass)) - } + val ser = Serializer.get(dep.serializerClass) + shuffle = blockManager.shuffleBlockManager.forShuffle(dep.shuffleId, numOutputSplits, ser) + buckets = shuffle.acquireWriters(partition) // Write the map output to its associated buckets. for (elem <- rdd.iterator(split, taskContext)) { val pair = elem.asInstanceOf[(Any, Any)] val bucketId = dep.partitioner.getPartition(pair._1) - buckets(bucketId).write(pair) + buckets.writers(bucketId).write(pair) } - // Close the bucket writers and get the sizes of each block. - val compressedSizes = new Array[Byte](numOutputSplits) - var i = 0 + // Commit the writes. Get the size of each bucket block (total block size). var totalBytes = 0L - while (i < numOutputSplits) { - buckets(i).close() - val size = buckets(i).size() + val compressedSizes: Array[Byte] = buckets.writers.map { writer: BlockObjectWriter => + writer.commit() + writer.close() + val size = writer.size() totalBytes += size - compressedSizes(i) = MapOutputTracker.compressSize(size) - i += 1 + MapOutputTracker.compressSize(size) } // Update shuffle metrics. @@ -164,7 +171,18 @@ private[spark] class ShuffleMapTask( metrics.get.shuffleWriteMetrics = Some(shuffleMetrics) return new MapStatus(blockManager.blockManagerId, compressedSizes) + } catch { case e: Exception => + // If there is an exception from running the task, revert the partial writes + // and throw the exception upstream to Spark. + if (buckets != null) { + buckets.writers.foreach(_.revertPartialWrites()) + } + throw e } finally { + // Release the writers back to the shuffle block manager. + if (shuffle != null && buckets != null) { + shuffle.releaseWriters(buckets) + } // Execute the callbacks on task completion. taskContext.executeOnCompleteCallbacks() } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9190c96c71fa9..b94d729923d99 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -88,6 +88,8 @@ class BlockManager( } } + val shuffleBlockManager = new ShuffleBlockManager(this) + private val blockInfo = new TimeStampedHashMap[String, BlockInfo] private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) @@ -391,7 +393,7 @@ class BlockManager( // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work - if (blockId.startsWith("shuffle_")) { + if (ShuffleBlockManager.isShuffle(blockId)) { return diskStore.getBytes(blockId) match { case Some(bytes) => Some(bytes) @@ -508,12 +510,12 @@ class BlockManager( /** * A short circuited method to get a block writer that can write data directly to disk. - * This is currently used for writing shuffle files out. + * This is currently used for writing shuffle files out. Callers should handle error + * cases. */ def getDiskBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { val writer = diskStore.getBlockWriter(blockId, serializer) writer.registerCloseEventHandler(() => { - // TODO(rxin): This doesn't handle error cases. val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) blockInfo.put(blockId, myInfo) myInfo.markReady(writer.size()) @@ -872,7 +874,7 @@ class BlockManager( } def shouldCompress(blockId: String): Boolean = { - if (blockId.startsWith("shuffle_")) { + if (ShuffleBlockManager.isShuffle(blockId)) { compressShuffle } else if (blockId.startsWith("broadcast_")) { compressBroadcast @@ -887,7 +889,11 @@ class BlockManager( * Wrap an output stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: OutputStream): OutputStream = { - if (shouldCompress(blockId)) new LZFOutputStream(s) else s + if (shouldCompress(blockId)) { + (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + } else { + s + } } /** diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 657a7e9143ab7..42e2b07d5ca5a 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -3,25 +3,48 @@ package spark.storage import java.nio.ByteBuffer +/** + * An interface for writing JVM objects to some underlying storage. This interface allows + * appending data to an existing block, and can guarantee atomicity in the case of faults + * as it allows the caller to revert partial writes. + * + * This interface does not support concurrent writes. + */ abstract class BlockObjectWriter(val blockId: String) { - // TODO(rxin): What if there is an exception when the block is being written out? - var closeEventHandler: () => Unit = _ - def registerCloseEventHandler(handler: () => Unit) { - closeEventHandler = handler + def open(): BlockObjectWriter + + def close() { + closeEventHandler() } - def write(value: Any) + def isOpen: Boolean - def writeAll(value: Iterator[Any]) { - value.foreach(write) + def registerCloseEventHandler(handler: () => Unit) { + closeEventHandler = handler } - def close() { - closeEventHandler() - } + /** + * Flush the partial writes and commit them as a single atomic block. Return the + * number of bytes written for this commit. + */ + def commit(): Long + + /** + * Reverts writes that haven't been flushed yet. Callers should invoke this function + * when there are runtime exceptions. + */ + def revertPartialWrites() + + /** + * Writes an object. + */ + def write(value: Any) + /** + * Size of the valid writes, in bytes. + */ def size(): Long } diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index b527a3c708b2e..f23cd5475f383 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -12,7 +12,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import spark.Utils import spark.executor.ExecutorExitCode -import spark.serializer.Serializer +import spark.serializer.{Serializer, SerializationStream} /** @@ -21,35 +21,58 @@ import spark.serializer.Serializer private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { - private val mapMode = MapMode.READ_ONLY - private var mapOpenMode = "r" - class DiskBlockObjectWriter(blockId: String, serializer: Serializer) extends BlockObjectWriter(blockId) { private val f: File = createFile(blockId /*, allowAppendExisting */) - private val bs: OutputStream = blockManager.wrapForCompression(blockId, - new FastBufferedOutputStream(new FileOutputStream(f))) - private val objOut = serializer.newInstance().serializeStream(bs) - - private var _size: Long = -1L - override def write(value: Any) { - objOut.writeObject(value) + private var repositionableStream: FastBufferedOutputStream = null + private var bs: OutputStream = null + private var objOut: SerializationStream = null + private var validLength = 0L + + override def open(): DiskBlockObjectWriter = { + println("------------------------------------------------- opening " + f) + repositionableStream = new FastBufferedOutputStream(new FileOutputStream(f)) + bs = blockManager.wrapForCompression(blockId, repositionableStream) + objOut = serializer.newInstance().serializeStream(bs) + this } override def close() { objOut.close() bs.close() + objOut = null + bs = null + repositionableStream = null + // Invoke the close callback handler. super.close() } - override def size(): Long = { - if (_size < 0) { - _size = f.length() - } - _size + override def isOpen: Boolean = objOut != null + + // Flush the partial writes, and set valid length to be the length of the entire file. + // Return the number of bytes written for this commit. + override def commit(): Long = { + bs.flush() + repositionableStream.position() + } + + override def revertPartialWrites() { + // Flush the outstanding writes and delete the file. + objOut.close() + bs.close() + objOut = null + bs = null + repositionableStream = null + f.delete() + } + + override def write(value: Any) { + objOut.writeObject(value) } + + override def size(): Long = validLength } val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @@ -90,9 +113,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private def getFileBytes(file: File): ByteBuffer = { val length = file.length() - val channel = new RandomAccessFile(file, mapOpenMode).getChannel() + val channel = new RandomAccessFile(file, "r").getChannel() val buffer = try { - channel.map(mapMode, 0, length) + channel.map(MapMode.READ_ONLY, 0, length) } finally { channel.close() } @@ -230,12 +253,14 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } private def addShutdownHook() { - localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir) ) + localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") try { - localDirs.foreach(localDir => if (! Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir)) + localDirs.foreach { localDir => + if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } } catch { case t: Throwable => logError("Exception while deleting local spark dirs", t) } diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala new file mode 100644 index 0000000000000..2b1138e7a0b74 --- /dev/null +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -0,0 +1,52 @@ +package spark.storage + +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable.ArrayBuffer + +import spark.serializer.Serializer + + +private[spark] +class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter]) + + +private[spark] +class ShuffleBlockManager(blockManager: BlockManager) { + + val shuffles = new ConcurrentHashMap[Int, Shuffle] + + def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): Shuffle = { + new Shuffle(shuffleId, numBuckets, serializer) + } + + class Shuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer) { + + // Get a group of writers for a map task. + def acquireWriters(mapId: Int): ShuffleWriterGroup = { + val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => + val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) + blockManager.getDiskBlockWriter(blockId, serializer).open() + } + new ShuffleWriterGroup(mapId, writers) + } + + def releaseWriters(group: ShuffleWriterGroup) = { + // Nothing really to release here. + } + } +} + + +private[spark] +object ShuffleBlockManager { + + // Returns the block id for a given shuffle block. + def blockId(shuffleId: Int, bucketId: Int, groupId: Int): String = { + "shuffle_" + shuffleId + "_" + groupId + "_" + bucketId + } + + // Returns true if the block is a shuffle block. + def isShuffle(blockId: String): Boolean = blockId.startsWith("shuffle_") +} From 1055785a836ab2361239f0937a1a22fee953e029 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Apr 2013 23:33:56 -0700 Subject: [PATCH 507/696] Allow specifying the shuffle write file buffer size. The default buffer size is 8KB in FastBufferedOutputStream, which is too small and would cause a lot of disk seeks. --- core/src/main/scala/spark/storage/BlockManager.scala | 5 +++-- core/src/main/scala/spark/storage/DiskStore.scala | 10 +++++----- .../main/scala/spark/storage/ShuffleBlockManager.scala | 3 ++- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index b94d729923d99..6e0ca9204dc46 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -513,8 +513,9 @@ class BlockManager( * This is currently used for writing shuffle files out. Callers should handle error * cases. */ - def getDiskBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { - val writer = diskStore.getBlockWriter(blockId, serializer) + def getDiskBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int) + : BlockObjectWriter = { + val writer = diskStore.getBlockWriter(blockId, serializer, bufferSize) writer.registerCloseEventHandler(() => { val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) blockInfo.put(blockId, myInfo) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index f23cd5475f383..4cddcc86fc3ad 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -21,7 +21,7 @@ import spark.serializer.{Serializer, SerializationStream} private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { - class DiskBlockObjectWriter(blockId: String, serializer: Serializer) + class DiskBlockObjectWriter(blockId: String, serializer: Serializer, bufferSize: Int) extends BlockObjectWriter(blockId) { private val f: File = createFile(blockId /*, allowAppendExisting */) @@ -32,7 +32,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private var validLength = 0L override def open(): DiskBlockObjectWriter = { - println("------------------------------------------------- opening " + f) repositionableStream = new FastBufferedOutputStream(new FileOutputStream(f)) bs = blockManager.wrapForCompression(blockId, repositionableStream) objOut = serializer.newInstance().serializeStream(bs) @@ -55,7 +54,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Return the number of bytes written for this commit. override def commit(): Long = { bs.flush() - repositionableStream.position() + validLength = repositionableStream.position() + validLength } override def revertPartialWrites() { @@ -86,8 +86,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() - def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { - new DiskBlockObjectWriter(blockId, serializer) + def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int): BlockObjectWriter = { + new DiskBlockObjectWriter(blockId, serializer, bufferSize) } override def getSize(blockId: String): Long = { diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 2b1138e7a0b74..2b22dad459394 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -25,9 +25,10 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Get a group of writers for a map task. def acquireWriters(mapId: Int): ShuffleWriterGroup = { + val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) - blockManager.getDiskBlockWriter(blockId, serializer).open() + blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() } new ShuffleWriterGroup(mapId, writers) } From e46d547ccd43c0fb3a79a30a7c43a78afba6f93f Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 30 Apr 2013 16:15:56 +0530 Subject: [PATCH 508/696] Fix issues reported by Reynold --- .../spark/network/ConnectionManager.scala | 64 +++++++++++++++---- run | 7 +- 2 files changed, 56 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 0c6bdb155972f..a79fce8697fa1 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -188,6 +188,38 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } ) } + // MUST be called within selector loop - else deadlock. + private def triggerForceCloseByException(key: SelectionKey, e: Exception) { + try { + key.interestOps(0) + } catch { + // ignore exceptions + case e: Exception => logDebug("Ignoring exception", e) + } + + val conn = connectionsByKey.getOrElse(key, null) + if (conn == null) return + + // Pushing to connect threadpool + handleConnectExecutor.execute(new Runnable { + override def run() { + try { + conn.callOnExceptionCallback(e) + } catch { + // ignore exceptions + case e: Exception => logDebug("Ignoring exception", e) + } + try { + conn.close() + } catch { + // ignore exceptions + case e: Exception => logDebug("Ignoring exception", e) + } + } + }) + } + + def run() { try { while(!selectorThread.isInterrupted) { @@ -235,18 +267,26 @@ private[spark] class ConnectionManager(port: Int) extends Logging { while (selectedKeys.hasNext()) { val key = selectedKeys.next selectedKeys.remove() - if (key.isValid) { - if (key.isAcceptable) { - acceptConnection(key) - } else - if (key.isConnectable) { - triggerConnect(key) - } else - if (key.isReadable) { - triggerRead(key) - } else - if (key.isWritable) { - triggerWrite(key) + try { + if (key.isValid) { + if (key.isAcceptable) { + acceptConnection(key) + } else + if (key.isConnectable) { + triggerConnect(key) + } else + if (key.isReadable) { + triggerRead(key) + } else + if (key.isWritable) { + triggerWrite(key) + } + } + } catch { + // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) } } } diff --git a/run b/run index 756f8703f2502..0a58ac4a36eb7 100755 --- a/run +++ b/run @@ -95,6 +95,7 @@ export JAVA_OPTS CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" +REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" STREAMING_DIR="$FWDIR/streaming" @@ -125,8 +126,8 @@ if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH+=":$FWDIR/lib_managed/bundles/*" fi CLASSPATH+=":$REPL_DIR/lib/*" -if [ -e repl-bin/target ]; then - for jar in `find "repl-bin/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do +if [ -e $REPL_BIN_DIR/target ]; then + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH+=":$jar" done fi @@ -134,7 +135,6 @@ CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH+=":$jar" done -export CLASSPATH # Needed for spark-shell # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack # to avoid the -sources and -doc packages that are built by publish-local. @@ -163,4 +163,5 @@ else EXTRA_ARGS="$JAVA_OPTS" fi +export CLASSPATH # Needed for spark-shell exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" From 48854e1dbf1d02e1e19f59d0aee0e281d41b3b45 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 30 Apr 2013 23:59:33 +0530 Subject: [PATCH 509/696] If key is not valid, close connection --- .gitignore | 2 -- core/src/main/scala/spark/network/ConnectionManager.scala | 3 +++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 155e785b01beb..5bb2f33574599 100644 --- a/.gitignore +++ b/.gitignore @@ -29,8 +29,6 @@ project/build/target/ project/plugins/target/ project/plugins/lib_managed/ project/plugins/src_managed/ -logs/ -log/ spark-tests.log streaming-tests.log dependency-reduced-pom.xml diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index a79fce8697fa1..2d9b4be4b31a9 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -281,6 +281,9 @@ private[spark] class ConnectionManager(port: Int) extends Logging { if (key.isWritable) { triggerWrite(key) } + } else { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() } } catch { // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. From 538614acfe95b0c064679122af3bc990b669e4e0 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:05:32 +0530 Subject: [PATCH 510/696] Be more aggressive and defensive in select also --- .../spark/network/ConnectionManager.scala | 83 ++++++++++++------- 1 file changed, 55 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 2d9b4be4b31a9..9b00fddd40557 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -254,7 +254,32 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } } - val selectedKeysCount = selector.select() + val selectedKeysCount = + try { + selector.select() + } catch { + case e: CancelledKeyException => { + // Some keys within the selectors list are invalid/closed. clear them. + val allKeys = selector.keys().iterator() + + while (allKeys.hasNext()) { + val key = allKeys.next() + try { + if (! key.isValid) { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) + } + } + } + } + 0 + } + if (selectedKeysCount == 0) { logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys") } @@ -262,34 +287,36 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("Selector thread was interrupted!") return } - - val selectedKeys = selector.selectedKeys().iterator() - while (selectedKeys.hasNext()) { - val key = selectedKeys.next - selectedKeys.remove() - try { - if (key.isValid) { - if (key.isAcceptable) { - acceptConnection(key) - } else - if (key.isConnectable) { - triggerConnect(key) - } else - if (key.isReadable) { - triggerRead(key) - } else - if (key.isWritable) { - triggerWrite(key) + + if (0 != selectedKeysCount) { + val selectedKeys = selector.selectedKeys().iterator() + while (selectedKeys.hasNext()) { + val key = selectedKeys.next + selectedKeys.remove() + try { + if (key.isValid) { + if (key.isAcceptable) { + acceptConnection(key) + } else + if (key.isConnectable) { + triggerConnect(key) + } else + if (key.isReadable) { + triggerRead(key) + } else + if (key.isWritable) { + triggerWrite(key) + } + } else { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) } - } else { - logInfo("Key not valid ? " + key) - throw new CancelledKeyException() - } - } catch { - // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. - case e: CancelledKeyException => { - logInfo("key already cancelled ? " + key, e) - triggerForceCloseByException(key, e) } } } From 0f45477be16254971763cbc07feac7460cffd0bd Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:10:02 +0530 Subject: [PATCH 511/696] Change indentation --- .../spark/network/ConnectionManager.scala | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 9b00fddd40557..925d076951c6c 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -255,30 +255,30 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } val selectedKeysCount = - try { - selector.select() - } catch { - case e: CancelledKeyException => { - // Some keys within the selectors list are invalid/closed. clear them. - val allKeys = selector.keys().iterator() - - while (allKeys.hasNext()) { - val key = allKeys.next() - try { - if (! key.isValid) { - logInfo("Key not valid ? " + key) - throw new CancelledKeyException() - } - } catch { - case e: CancelledKeyException => { - logInfo("key already cancelled ? " + key, e) - triggerForceCloseByException(key, e) + try { + selector.select() + } catch { + case e: CancelledKeyException => { + // Some keys within the selectors list are invalid/closed. clear them. + val allKeys = selector.keys().iterator() + + while (allKeys.hasNext()) { + val key = allKeys.next() + try { + if (! key.isValid) { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) + } } } } + 0 } - 0 - } if (selectedKeysCount == 0) { logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys") From 3b748ced2258246bd9b7c250363645cea27cf622 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:30:30 +0530 Subject: [PATCH 512/696] Be more aggressive and defensive in all uses of SelectionKey in select loop --- .../spark/network/ConnectionManager.scala | 47 ++++++++++++------- 1 file changed, 30 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 925d076951c6c..03926a603852a 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -232,24 +232,37 @@ private[spark] class ConnectionManager(port: Int) extends Logging { while(!keyInterestChangeRequests.isEmpty) { val (key, ops) = keyInterestChangeRequests.dequeue - val connection = connectionsByKey.getOrElse(key, null) - if (connection != null) { - val lastOps = key.interestOps() - key.interestOps(ops) - - // hot loop - prevent materialization of string if trace not enabled. - if (isTraceEnabled()) { - def intToOpStr(op: Int): String = { - val opStrs = ArrayBuffer[String]() - if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ" - if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE" - if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT" - if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT" - if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " - } - logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() + - "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") + try { + if (key.isValid) { + val connection = connectionsByKey.getOrElse(key, null) + if (connection != null) { + val lastOps = key.interestOps() + key.interestOps(ops) + + // hot loop - prevent materialization of string if trace not enabled. + if (isTraceEnabled()) { + def intToOpStr(op: Int): String = { + val opStrs = ArrayBuffer[String]() + if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ" + if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE" + if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT" + if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT" + if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " + } + + logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() + + "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") + } + } + } else { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) } } } From c446ac31d7065d227f168a7f27010bdf98ef7ad1 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:32:30 +0530 Subject: [PATCH 513/696] Spurious commit, reverting gitignore change --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 5bb2f33574599..155e785b01beb 100644 --- a/.gitignore +++ b/.gitignore @@ -29,6 +29,8 @@ project/build/target/ project/plugins/target/ project/plugins/lib_managed/ project/plugins/src_managed/ +logs/ +log/ spark-tests.log streaming-tests.log dependency-reduced-pom.xml From 60cabb35cbfd2af0e5ba34c4a416aa2640091acc Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 01:17:14 +0530 Subject: [PATCH 514/696] Add addition catch block for exception too --- .../scala/spark/network/ConnectionManager.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 03926a603852a..0eb03630d00fe 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -264,6 +264,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) } + case e: Exception => { + logError("Exception processing key " + key, e) + triggerForceCloseByException(key, e) + } } } @@ -271,6 +275,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { try { selector.select() } catch { + // Explicitly only dealing with CancelledKeyException here since other exceptions should be dealt with differently. case e: CancelledKeyException => { // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() @@ -287,6 +292,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) } + case e: Exception => { + logError("Exception processing key " + key, e) + triggerForceCloseByException(key, e) + } } } } @@ -330,6 +339,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) } + case e: Exception => { + logError("Exception processing key " + key, e) + triggerForceCloseByException(key, e) + } } } } From dd7bef31472e8c7dedc93bc1519be5900784c736 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Apr 2013 15:02:32 -0700 Subject: [PATCH 515/696] Two minor fixes according to Ryan LeCompte's review. --- core/src/main/scala/spark/storage/BlockManager.scala | 7 ++----- .../src/main/scala/spark/storage/ShuffleBlockManager.scala | 7 ------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 6e0ca9204dc46..09572b19db2db 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -296,11 +296,8 @@ class BlockManager( * never deletes (recent) items. */ def getLocalFromDisk(blockId: String, serializer: Serializer): Option[Iterator[Any]] = { - diskStore.getValues(blockId, serializer) match { - case Some(iterator) => Some(iterator) - case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") - } + diskStore.getValues(blockId, serializer).orElse( + sys.error("Block " + blockId + " not found on disk, though it should be")) } /** diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 2b22dad459394..1903df0817580 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -1,10 +1,5 @@ package spark.storage -import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable.ArrayBuffer - import spark.serializer.Serializer @@ -15,8 +10,6 @@ class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter]) private[spark] class ShuffleBlockManager(blockManager: BlockManager) { - val shuffles = new ConcurrentHashMap[Int, Shuffle] - def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): Shuffle = { new Shuffle(shuffleId, numBuckets, serializer) } From d960e7e0f83385d8f43129d53c189b3036936daf Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 20:24:00 +0530 Subject: [PATCH 516/696] a) Add support for hyper local scheduling - specific to a host + port - before trying host local scheduling. b) Add some fixes to test code to ensure it passes (and fixes some other issues). c) Fix bug in task scheduling which incorrectly used availableCores instead of all cores on the node. --- core/src/main/scala/spark/SparkEnv.scala | 21 ++- core/src/main/scala/spark/Utils.scala | 17 +- .../scala/spark/deploy/worker/Worker.scala | 5 +- core/src/main/scala/spark/rdd/BlockRDD.scala | 9 +- core/src/main/scala/spark/rdd/ZippedRDD.scala | 2 + .../scala/spark/scheduler/DAGScheduler.scala | 7 +- .../scala/spark/scheduler/ResultTask.scala | 4 +- .../spark/scheduler/ShuffleMapTask.scala | 4 +- .../scheduler/cluster/ClusterScheduler.scala | 64 +++++++- .../scheduler/cluster/TaskSetManager.scala | 155 ++++++++++++------ .../scala/spark/storage/BlockManager.scala | 43 +++-- .../scala/spark/MapOutputTrackerSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 4 +- .../spark/storage/BlockManagerSuite.scala | 2 + 14 files changed, 244 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ffb40bab3a3c4..5b4a4640109de 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -29,7 +29,11 @@ class SparkEnv ( val blockManager: BlockManager, val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, - val sparkFilesDir: String + val sparkFilesDir: String, + // To be set only as part of initialization of SparkContext. + // (executorId, defaultHostPort) => executorHostPort + // If executorId is NOT found, return defaultHostPort + var executorIdToHostPort: (String, String) => String ) { def stop() { @@ -44,6 +48,17 @@ class SparkEnv ( // down, but let's call it anyway in case it gets fixed in a later release actorSystem.awaitTermination() } + + + def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = { + val env = SparkEnv.get + if (env.executorIdToHostPort == null) { + // default to using host, not host port. Relevant to non cluster modes. + return defaultHostPort + } + + env.executorIdToHostPort(executorId, defaultHostPort) + } } object SparkEnv extends Logging { @@ -162,7 +177,7 @@ object SparkEnv extends Logging { blockManager, connectionManager, httpFileServer, - sparkFilesDir) + sparkFilesDir, + null) } - } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 9f48cbe490a27..279daf04edfb3 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -357,21 +357,26 @@ private object Utils extends Logging { Utils.logErrorWithStack("Unexpected to have port " + port + " which is not valid in " + hostPort + ". Message " + message) } } + + // Used by DEBUG code : remove when all testing done + def logErrorWithStack(msg: String) { + try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } + // temp code for debug + System.exit(-1) + } */ // Once testing is complete in various modes, replace with this ? def checkHost(host: String, message: String = "") {} def checkHostPort(hostPort: String, message: String = "") {} - def getUserNameFromEnvironment(): String = { - SparkHadoopUtil.getUserNameFromEnvironment - } - // Used by DEBUG code : remove when all testing done def logErrorWithStack(msg: String) { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } - // temp code for debug - System.exit(-1) + } + + def getUserNameFromEnvironment(): String = { + SparkHadoopUtil.getUserNameFromEnvironment } // Typically, this will be of order of number of nodes in cluster diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 1a7da0f7bf911..3dc2207170730 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -54,7 +54,10 @@ private[spark] class Worker( def createWorkDir() { workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) try { - if ( (workDir.exists() && !workDir.isDirectory) || (!workDir.exists() && !workDir.mkdirs()) ) { + // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() + // So attempting to create and then check if directory was created or not. + workDir.mkdirs() + if ( !workDir.exists() || !workDir.isDirectory) { logError("Failed to create work directory " + workDir) System.exit(1) } diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 7348c4f15bad6..719d4bf03e34b 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,7 +1,7 @@ package spark.rdd -import scala.collection.mutable.HashMap import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import spark.storage.BlockManager private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { val index = idx @@ -11,12 +11,7 @@ private[spark] class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { - @transient lazy val locations_ = { - val blockManager = SparkEnv.get.blockManager - /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ - val locations = blockManager.getLocations(blockIds) - HashMap(blockIds.zip(locations):_*) - } + @transient lazy val locations_ = BlockManager.blockIdsToExecutorLocations(blockIds, SparkEnv.get) override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 35b0e06785cf3..e80250a99bdf0 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -49,6 +49,8 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( override def getPreferredLocations(s: Partition): Seq[String] = { val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions + // TODO: becomes complicated - intersect on hostPort if available, else fallback to host (removing intersected hostPort's). + // Since I am not very sure about this RDD, leaving it to others to comment better ! rdd1.preferredLocations(partition1).intersect(rdd2.preferredLocations(partition2)) } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 1440b93e65cf9..8072c60bb72f0 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -12,7 +12,7 @@ import spark.executor.TaskMetrics import spark.partial.ApproximateActionListener import spark.partial.ApproximateEvaluator import spark.partial.PartialResult -import spark.storage.BlockManagerMaster +import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} /** @@ -117,9 +117,8 @@ class DAGScheduler( private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray - cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map { - locations => locations.map(_.hostPort).toList - }.toArray + val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env) + cacheLocs(rdd.id) = blockIds.map(locs.getOrElse(_, Nil)) } cacheLocs(rdd.id) } diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 89dc6640b2fe2..c43cbe5ed4be8 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -71,11 +71,11 @@ private[spark] class ResultTask[T, U]( } // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. - val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { // DEBUG code - preferredLocs.foreach (host => Utils.checkHost(host, "preferredLocs : " + preferredLocs)) + preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) } override def run(attemptId: Long): U = { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 7dc6da4573391..0b848af2f3156 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -85,11 +85,11 @@ private[spark] class ShuffleMapTask( protected def this() = this(0, null, null, 0, null) // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. - private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { // DEBUG code - preferredLocs.foreach (host => Utils.checkHost(host, "preferredLocs : " + preferredLocs)) + preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) } var split = if (rdd == null) { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index a9d9c5e44c72b..3c72ce42062ee 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -79,9 +79,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host - val executorsByHostPort = new HashMap[String, HashSet[String]] + private val executorsByHostPort = new HashMap[String, HashSet[String]] - val executorIdToHostPort = new HashMap[String, String] + private val executorIdToHostPort = new HashMap[String, String] // JAR server, if any JARs were added by the user to the SparkContext var jarServer: HttpServer = null @@ -102,6 +102,14 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context + // resolve executorId to hostPort mapping. + def executorToHostPort(executorId: String, defaultHostPort: String): String = { + executorIdToHostPort.getOrElse(executorId, defaultHostPort) + } + + // Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler + // Will that be a design violation ? + SparkEnv.get.executorIdToHostPort = executorToHostPort } def newTaskId(): Long = nextTaskId.getAndIncrement() @@ -209,13 +217,30 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + // merge availableCpus into hostToAvailableCpus block ? val availableCpus = offers.map(o => o.cores).toArray + val hostToAvailableCpus = { + val map = new HashMap[String, Int]() + for (offer <- offers) { + val hostPort = offer.hostPort + val cores = offer.cores + // DEBUG code + Utils.checkHostPort(hostPort) + + val host = Utils.parseHostPort(hostPort)._1 + + map.put(host, map.getOrElse(host, 0) + cores) + } + + map + } var launchedTask = false for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { // Split offers based on host local, rack local and off-rack tasks. + val hyperLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val hostLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val otherOffers = new HashMap[String, ArrayBuffer[Int]]() @@ -224,8 +249,17 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val hostPort = offers(i).hostPort // DEBUG code Utils.checkHostPort(hostPort) + + val numHyperLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) + if (numHyperLocalTasks > 0){ + val list = hyperLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) + for (j <- 0 until numHyperLocalTasks) list += i + } + val host = Utils.parseHostPort(hostPort)._1 - val numHostLocalTasks = math.max(0, math.min(manager.numPendingTasksForHost(hostPort), availableCpus(i))) + val numHostLocalTasks = math.max(0, + // Remove hyper local tasks (which are also host local btw !) from this + math.min(manager.numPendingTasksForHost(hostPort) - numHyperLocalTasks, hostToAvailableCpus(host))) if (numHostLocalTasks > 0){ val list = hostLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numHostLocalTasks) list += i @@ -233,7 +267,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val numRackLocalTasks = math.max(0, // Remove host local tasks (which are also rack local btw !) from this - math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numHostLocalTasks, availableCpus(i))) + math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numHyperLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) if (numRackLocalTasks > 0){ val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numRackLocalTasks) list += i @@ -246,12 +280,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } val offersPriorityList = new ArrayBuffer[Int]( - hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) - // First host local, then rack, then others + hyperLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) + + // First hyper local, then host local, then rack, then others + + // numHostLocalOffers contains count of both hyper local and host offers. val numHostLocalOffers = { + val hyperLocalPriorityList = ClusterScheduler.prioritizeContainers(hyperLocalOffers) + offersPriorityList ++= hyperLocalPriorityList + val hostLocalPriorityList = ClusterScheduler.prioritizeContainers(hostLocalOffers) offersPriorityList ++= hostLocalPriorityList - hostLocalPriorityList.size + + hyperLocalPriorityList.size + hostLocalPriorityList.size } val numRackLocalOffers = { val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) @@ -477,6 +518,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } def getExecutorsAliveOnHost(host: String): Option[Set[String]] = { + Utils.checkHost(host) + val retval = hostToAliveHostPorts.get(host) if (retval.isDefined) { return Some(retval.get.toSet) @@ -485,6 +528,13 @@ private[spark] class ClusterScheduler(val sc: SparkContext) None } + def isExecutorAliveOnHostPort(hostPort: String): Boolean = { + // Even if hostPort is a host, it does not matter - it is just a specific check. + // But we do have to ensure that only hostPort get into hostPortsAlive ! + // So no check against Utils.checkHostPort + hostPortsAlive.contains(hostPort) + } + // By default, rack is unknown def getRackForHost(value: String): Option[String] = None diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 27e713e2c43dd..f5c0058554b7e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -13,14 +13,18 @@ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer -private[spark] object TaskLocality extends Enumeration("HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { +private[spark] object TaskLocality extends Enumeration("HYPER_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { - val HOST_LOCAL, RACK_LOCAL, ANY = Value + // hyper local is expected to be used ONLY within tasksetmanager for now. + val HYPER_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { + // Must not be the constraint. + assert (constraint != TaskLocality.HYPER_LOCAL) + constraint match { case TaskLocality.HOST_LOCAL => condition == TaskLocality.HOST_LOCAL case TaskLocality.RACK_LOCAL => condition == TaskLocality.HOST_LOCAL || condition == TaskLocality.RACK_LOCAL @@ -32,7 +36,11 @@ private[spark] object TaskLocality extends Enumeration("HOST_LOCAL", "RACK_LOCAL def parse(str: String): TaskLocality = { // better way to do this ? try { - TaskLocality.withName(str) + val retval = TaskLocality.withName(str) + // Must not specify HYPER_LOCAL ! + assert (retval != TaskLocality.HYPER_LOCAL) + + retval } catch { case nEx: NoSuchElementException => { logWarning("Invalid task locality specified '" + str + "', defaulting to HOST_LOCAL"); @@ -133,35 +141,55 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } - private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, rackLocal: Boolean = false): ArrayBuffer[String] = { - // DEBUG code - _taskPreferredLocations.foreach(h => Utils.checkHost(h, "taskPreferredLocation " + _taskPreferredLocations)) - - val taskPreferredLocations = if (! rackLocal) _taskPreferredLocations else { - // Expand set to include all 'seen' rack local hosts. - // This works since container allocation/management happens within master - so any rack locality information is updated in msater. - // Best case effort, and maybe sort of kludge for now ... rework it later ? - val hosts = new HashSet[String] - _taskPreferredLocations.foreach(h => { - val rackOpt = scheduler.getRackForHost(h) - if (rackOpt.isDefined) { - val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) - if (hostsOpt.isDefined) { - hosts ++= hostsOpt.get + // Note that it follows the hierarchy. + // if we search for HOST_LOCAL, the output will include HYPER_LOCAL and + // if we search for RACK_LOCAL, it will include HYPER_LOCAL & HOST_LOCAL + private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, + taskLocality: TaskLocality.TaskLocality): HashSet[String] = { + + if (TaskLocality.HYPER_LOCAL == taskLocality) { + // straight forward comparison ! Special case it. + val retval = new HashSet[String]() + scheduler.synchronized { + for (location <- _taskPreferredLocations) { + if (scheduler.isExecutorAliveOnHostPort(location)) { + retval += location } } + } - // Ensure that irrespective of what scheduler says, host is always added ! - hosts += h - }) - - hosts + return retval } - val retval = new ArrayBuffer[String] + val taskPreferredLocations = + if (TaskLocality.HOST_LOCAL == taskLocality) { + _taskPreferredLocations + } else { + assert (TaskLocality.RACK_LOCAL == taskLocality) + // Expand set to include all 'seen' rack local hosts. + // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // Best case effort, and maybe sort of kludge for now ... rework it later ? + val hosts = new HashSet[String] + _taskPreferredLocations.foreach(h => { + val rackOpt = scheduler.getRackForHost(h) + if (rackOpt.isDefined) { + val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) + if (hostsOpt.isDefined) { + hosts ++= hostsOpt.get + } + } + + // Ensure that irrespective of what scheduler says, host is always added ! + hosts += h + }) + + hosts + } + + val retval = new HashSet[String] scheduler.synchronized { for (prefLocation <- taskPreferredLocations) { - val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(prefLocation) + val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1) if (aliveLocationsOpt.isDefined) { retval ++= aliveLocationsOpt.get } @@ -175,29 +203,37 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe private def addPendingTask(index: Int) { // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched) - val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, true) + val hyperLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HYPER_LOCAL) + val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. + assert (hyperLocalLocations.size == 0) assert (hostLocalLocations.size == 0) pendingTasksWithNoPrefs += index } else { - // host locality - for (hostPort <- hostLocalLocations) { + // hyper local locality + for (hostPort <- hyperLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer()) hostPortList += index + } + + // host locality (includes hyper local) + for (hostPort <- hostLocalLocations) { + // DEBUG Code + Utils.checkHostPort(hostPort) val host = Utils.parseHostPort(hostPort)._1 val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) hostList += index } - // rack locality + // rack locality (includes hyper local and host local) for (rackLocalHostPort <- rackLocalLocations) { // DEBUG Code Utils.checkHostPort(rackLocalHostPort) @@ -233,6 +269,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) } + // Number of pending tasks for a given host Port (which would be hyper local) + def numPendingTasksForHostPort(hostPort: String): Int = { + getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + // Number of pending tasks for a given host (which would be data local) def numPendingTasksForHost(hostPort: String): Int = { getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) @@ -270,7 +311,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe if (speculatableTasks.size > 0) { val localTask = speculatableTasks.find { index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched) + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) val attemptLocs = taskAttempts(index).map(_.hostPort) (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) } @@ -284,7 +325,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { val rackTask = speculatableTasks.find { index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, true) + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) val attemptLocs = taskAttempts(index).map(_.hostPort) locations.contains(hostPort) && !attemptLocs.contains(hostPort) } @@ -311,6 +352,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + val hyperLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (hyperLocalTask != None) { + return hyperLocalTask + } + val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) if (localTask != None) { return localTask @@ -341,30 +387,31 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return findSpeculativeTask(hostPort, locality) } - // Does a host count as a preferred location for a task? This is true if - // either the task has preferred locations and this host is one, or it has - // no preferred locations (in which we still count the launch as preferred). - private def isPreferredLocation(task: Task[_], hostPort: String): Boolean = { + private def isHyperLocalLocation(task: Task[_], hostPort: String): Boolean = { + Utils.checkHostPort(hostPort) + val locs = task.preferredLocations - // DEBUG code - locs.foreach(h => Utils.checkHost(h, "preferredLocation " + locs)) - if (locs.contains(hostPort) || locs.isEmpty) return true + locs.contains(hostPort) + } + + private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = { + val locs = task.preferredLocations + + // If no preference, consider it as host local + if (locs.isEmpty) return true val host = Utils.parseHostPort(hostPort)._1 - locs.contains(host) + locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined } // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). // This is true if either the task has preferred locations and this host is one, or it has // no preferred locations (in which we still count the launch as preferred). - def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { + private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { val locs = task.preferredLocations - // DEBUG code - locs.foreach(h => Utils.checkHost(h, "preferredLocation " + locs)) - val preferredRacks = new HashSet[String]() for (preferredHost <- locs) { val rack = sched.getRackForHost(preferredHost) @@ -395,8 +442,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val task = tasks(index) val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch - val taskLocality = if (isPreferredLocation(task, hostPort)) TaskLocality.HOST_LOCAL else - if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else TaskLocality.ANY + val taskLocality = + if (isHyperLocalLocation(task, hostPort)) TaskLocality.HYPER_LOCAL else + if (isHostLocalLocation(task, hostPort)) TaskLocality.HOST_LOCAL else + if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else + TaskLocality.ANY val prefStr = taskLocality.toString logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, execId, hostPort, prefStr)) @@ -552,15 +602,22 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe def executorLost(execId: String, hostPort: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) + // If some task has preferred locations only on hostname, and there are no more executors there, // put it in the no-prefs list to avoid the wait from delay scheduling - for (index <- getPendingTasksForHostPort(hostPort)) { - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, true) + + // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to + // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. + // Note: NOT checking hyper local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no hyper local node for the task) + for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { + // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) if (newLocs.isEmpty) { - assert (findPreferredLocations(tasks(index).preferredLocations, sched).isEmpty) pendingTasksWithNoPrefs += index } } + // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage if (tasks(0).isInstanceOf[ShuffleMapTask]) { for ((tid, info) <- taskInfos if info.executorId == execId) { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 6e861ac734604..7a0d6ced3edf4 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -4,7 +4,7 @@ import java.io.{InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet, Queue} import scala.collection.JavaConversions._ import akka.actor.{ActorSystem, Cancellable, Props} @@ -271,23 +271,12 @@ class BlockManager( } - /** - * Get locations of the block. - */ - def getLocations(blockId: String): Seq[String] = { - val startTimeMs = System.currentTimeMillis - var managers = master.getLocations(blockId) - val locations = managers.map(_.hostPort) - logDebug("Got block locations in " + Utils.getUsedTimeMs(startTimeMs)) - return locations - } - /** * Get locations of an array of blocks. */ - def getLocations(blockIds: Array[String]): Array[Seq[String]] = { + def getLocationBlockIds(blockIds: Array[String]): Array[Seq[BlockManagerId]] = { val startTimeMs = System.currentTimeMillis - val locations = master.getLocations(blockIds).map(_.map(_.hostPort).toSeq).toArray + val locations = master.getLocations(blockIds).toArray logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) return locations } @@ -947,6 +936,32 @@ object BlockManager extends Logging { } } } + + def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv): HashMap[String, List[String]] = { + val blockManager = env.blockManager + /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ + val locationBlockIds = blockManager.getLocationBlockIds(blockIds) + + // Convert from block master locations to executor locations (we need that for task scheduling) + val executorLocations = new HashMap[String, List[String]]() + for (i <- 0 until blockIds.length) { + val blockId = blockIds(i) + val blockLocations = locationBlockIds(i) + + val executors = new HashSet[String]() + + for (bkLocation <- blockLocations) { + val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host) + executors += executorHostPort + // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + } + + executorLocations.put(blockId, executors.toSeq.toList) + } + + executorLocations + } + } class BlockFetcherIterator( diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 3abc584b6a177..875975ca43e1d 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -80,12 +80,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { } test("remote fetch") { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) val masterTracker = new MapOutputTracker() masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0) + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() slaveTracker.trackerActor = slaveSystem.actorFor( "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index c0f8986de8dd5..16554eac6e684 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -385,12 +385,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(results === Map(0 -> 42)) } - /** Assert that the supplied TaskSet has exactly the given preferredLocations. */ + /** Assert that the supplied TaskSet has exactly the given preferredLocations. Note, converts taskSet's locations to host only. */ private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { assert(locations.size === taskSet.tasks.size) for ((expectLocs, taskLocs) <- taskSet.tasks.map(_.preferredLocations).zip(locations)) { - assert(expectLocs === taskLocs) + assert(expectLocs.map(loc => spark.Utils.parseHostPort(loc)._1) === taskLocs) } } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index b8c0f6fb763a9..3fc2825255acf 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -41,6 +41,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() + // Set some value ... + System.setProperty("spark.hostPort", spark.Utils.localHostName() + ":" + 1111) } after { From 27764a00f40391b94fa05abb11484c442607f6f7 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 20:56:05 +0530 Subject: [PATCH 517/696] Fix some npe introduced accidentally --- .../scala/spark/scheduler/DAGScheduler.scala | 2 +- .../scala/spark/storage/BlockManager.scala | 30 ++++++++++++++----- 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 8072c60bb72f0..b18248d2b5c41 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -117,7 +117,7 @@ class DAGScheduler( private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray - val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env) + val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env, blockManagerMaster) cacheLocs(rdd.id) = blockIds.map(locs.getOrElse(_, Nil)) } cacheLocs(rdd.id) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 7a0d6ced3edf4..040082e600673 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -937,10 +937,16 @@ object BlockManager extends Logging { } } - def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv): HashMap[String, List[String]] = { - val blockManager = env.blockManager - /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ - val locationBlockIds = blockManager.getLocationBlockIds(blockIds) + def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv, blockManagerMaster: BlockManagerMaster = null): HashMap[String, List[String]] = { + // env == null and blockManagerMaster != null is used in tests + assert (env != null || blockManagerMaster != null) + val locationBlockIds: Seq[Seq[BlockManagerId]] = + if (env != null) { + val blockManager = env.blockManager + blockManager.getLocationBlockIds(blockIds) + } else { + blockManagerMaster.getLocations(blockIds) + } // Convert from block master locations to executor locations (we need that for task scheduling) val executorLocations = new HashMap[String, List[String]]() @@ -950,10 +956,18 @@ object BlockManager extends Logging { val executors = new HashSet[String]() - for (bkLocation <- blockLocations) { - val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host) - executors += executorHostPort - // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + if (env != null) { + for (bkLocation <- blockLocations) { + val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host) + executors += executorHostPort + // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + } + } else { + // Typically while testing, etc - revert to simply using host. + for (bkLocation <- blockLocations) { + executors += bkLocation.host + // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + } } executorLocations.put(blockId, executors.toSeq.toList) From 848156273178bed5763bcbc91baa788bd4a57f6e Mon Sep 17 00:00:00 2001 From: harshars Date: Mon, 25 Mar 2013 20:09:07 -0700 Subject: [PATCH 518/696] Merged Ram's commit on removing RDDs. Conflicts: core/src/main/scala/spark/SparkContext.scala --- core/src/main/scala/spark/SparkContext.scala | 62 ++++++++++++------- .../test/scala/spark/DistributedSuite.scala | 12 ++++ core/src/test/scala/spark/RDDSuite.scala | 7 +++ 3 files changed, 59 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 5f5ec0b0f4d29..8bee1d65a21ad 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,47 +1,50 @@ package spark import java.io._ -import java.util.concurrent.atomic.AtomicInteger import java.net.URI +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicInteger +import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.generic.Growable -import scala.collection.mutable.HashMap -import scala.collection.JavaConversions._ +import scala.collection.mutable.{ConcurrentMap, HashMap} + +import akka.actor.Actor._ -import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.SequenceFileInputFormat -import org.apache.hadoop.io.Writable -import org.apache.hadoop.io.IntWritable -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.FloatWritable -import org.apache.hadoop.io.DoubleWritable +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.ArrayWritable import org.apache.hadoop.io.BooleanWritable import org.apache.hadoop.io.BytesWritable -import org.apache.hadoop.io.ArrayWritable +import org.apache.hadoop.io.DoubleWritable +import org.apache.hadoop.io.FloatWritable +import org.apache.hadoop.io.IntWritable +import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text +import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.FileInputFormat +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.SequenceFileInputFormat import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} +import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} + import org.apache.mesos.MesosNativeLibrary -import spark.deploy.{SparkHadoopUtil, LocalSparkCluster} -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler._ +import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.BlockManagerUI +import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.storage.{StorageStatus, StorageUtils, RDDInfo} + /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -97,7 +100,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]() + private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]]() private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) @@ -520,6 +523,21 @@ class SparkContext( env.blockManager.master.getStorageStatus } + def removeRDD(id: Int): Unit = { + val storageStatusList = getExecutorStorageStatus + val groupedRddBlocks = storageStatusList.flatMap(_.blocks).toMap + logInfo("RDD to remove: " + id) + groupedRddBlocks.foreach(x => { + val k = x._1.substring(0,x._1.lastIndexOf('_')) + val rdd_id = "rdd_" + id + logInfo("RDD to check: " + rdd_id) + if(k.equals(rdd_id)) { + env.blockManager.master.removeBlock(x._1) + } + }) + persistentRdds.remove(id) + } + /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. @@ -743,7 +761,7 @@ class SparkContext( /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { - persistentRdds.clearOldValues(cleanupTime) + // do nothing. this needs to be removed. } } diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index c9b4707def82b..c7f6ab3133b95 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -252,6 +252,18 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter assert(data2.count === 2) } } + + test("remove RDDs cleanly") { + DistributedSuite.amMaster = true + sc = new SparkContext("local-cluster[3,1,512]", "test") + val data = sc.parallelize(Seq(true, false, false, false), 4) + data.persist(StorageLevel.MEMORY_ONLY_2) + data.count + sc.removeRDD(data.id) + assert(sc.persistentRdds.isEmpty == true) + assert(sc.getRDDStorageInfo.isEmpty == true) + + } } object DistributedSuite { diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 7fbdd44340abe..88b7ab9f52b82 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -100,6 +100,13 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(rdd.collect().toList === List(1, 2, 3, 4)) } + test("remove RDD") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1,2,3,4), 2).cache() + sc.removeRDD(rdd.id) + assert(sc.persistentRdds.empty == true) + } + test("caching with failures") { sc = new SparkContext("local", "test") val onlySplit = new Partition { override def index: Int = 0 } From 3227ec8edde05cff27c1f9de8861d18b3cda1aae Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 16:07:44 -0700 Subject: [PATCH 519/696] Cleaned up Ram's code. Moved SparkContext.remove to RDD.unpersist. Also updated unit tests to make sure they are properly testing for concurrency. --- core/src/main/scala/spark/RDD.scala | 17 +++++++++++ core/src/main/scala/spark/SparkContext.scala | 25 ++++------------ .../scala/spark/storage/BlockManagerUI.scala | 4 +-- .../test/scala/spark/DistributedSuite.scala | 30 ++++++++++++++----- core/src/test/scala/spark/RDDSuite.scala | 27 +++++++++++++---- 5 files changed, 68 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 09e52ebf3ec00..c77f9915c021f 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -140,6 +140,23 @@ abstract class RDD[T: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): RDD[T] = persist() + /** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */ + def unpersist(): RDD[T] = { + logInfo("Removing RDD " + id + " from persistence list") + val rddBlockPrefix = "rdd_" + id + "_" + // Get the list of blocks in block manager, and remove ones that are part of this RDD. + // The runtime complexity is linear to the number of blocks persisted in the cluster. + // It could be expensive if the cluster is large and has a lot of blocks persisted. + sc.getExecutorStorageStatus().flatMap(_.blocks).foreach { case(blockId, status) => + if (blockId.startsWith(rddBlockPrefix)) { + sc.env.blockManager.master.removeBlock(blockId) + } + } + sc.persistentRdds.remove(id) + storageLevel = StorageLevel.NONE + this + } + /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ def getStorageLevel = storageLevel diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 8bee1d65a21ad..b686c595b8b4b 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -100,7 +100,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]]() + private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) @@ -508,36 +508,21 @@ class SparkContext( * Return information about what RDDs are cached, if they are in mem or on disk, how much space * they take, etc. */ - def getRDDStorageInfo : Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) + def getRDDStorageInfo(): Array[RDDInfo] = { + StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus(), this) } - def getStageInfo: Map[Stage,StageInfo] = { + def getStageInfo(): Map[Stage,StageInfo] = { dagScheduler.stageToInfos } /** * Return information about blocks stored in all of the slaves */ - def getExecutorStorageStatus : Array[StorageStatus] = { + def getExecutorStorageStatus(): Array[StorageStatus] = { env.blockManager.master.getStorageStatus } - def removeRDD(id: Int): Unit = { - val storageStatusList = getExecutorStorageStatus - val groupedRddBlocks = storageStatusList.flatMap(_.blocks).toMap - logInfo("RDD to remove: " + id) - groupedRddBlocks.foreach(x => { - val k = x._1.substring(0,x._1.lastIndexOf('_')) - val rdd_id = "rdd_" + id - logInfo("RDD to check: " + rdd_id) - if(k.equals(rdd_id)) { - env.blockManager.master.removeBlock(x._1) - } - }) - persistentRdds.remove(id) - } - /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 07da5720440cd..c9e4519efe889 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -45,7 +45,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, path("") { completeWith { // Request the current storage status from the Master - val storageStatusList = sc.getExecutorStorageStatus + val storageStatusList = sc.getExecutorStorageStatus() // Calculate macro-level statistics val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) @@ -60,7 +60,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, parameter("id") { id => completeWith { val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus + val storageStatusList = sc.getExecutorStorageStatus() val filteredStorageStatusList = StorageUtils. filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index c7f6ab3133b95..ab3e197035712 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -3,8 +3,10 @@ package spark import network.ConnectionManagerId import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers import org.scalatest.prop.Checkers +import org.scalatest.time.{Span, Millis} import org.scalacheck.Arbitrary._ import org.scalacheck.Gen import org.scalacheck.Prop._ @@ -252,24 +254,36 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter assert(data2.count === 2) } } - - test("remove RDDs cleanly") { + + test("unpersist RDDs") { DistributedSuite.amMaster = true sc = new SparkContext("local-cluster[3,1,512]", "test") val data = sc.parallelize(Seq(true, false, false, false), 4) data.persist(StorageLevel.MEMORY_ONLY_2) data.count - sc.removeRDD(data.id) + assert(sc.persistentRdds.isEmpty == false) + data.unpersist() assert(sc.persistentRdds.isEmpty == true) + + failAfter(Span(3000, Millis)) { + try { + while (! sc.getRDDStorageInfo.isEmpty) { + Thread.sleep(200) + } + } catch { + case e: Exception => + // Do nothing. We might see exceptions because block manager + // is racing this thread to remove entries from the driver. + } + } assert(sc.getRDDStorageInfo.isEmpty == true) - } } object DistributedSuite { // Indicates whether this JVM is marked for failure. var mark = false - + // Set by test to remember if we are in the driver program so we can assert // that we are not. var amMaster = false @@ -286,9 +300,9 @@ object DistributedSuite { // Act like an identity function, but if mark was set to true previously, fail, // crashing the entire JVM. def failOnMarkedIdentity(item: Boolean): Boolean = { - if (mark) { + if (mark) { System.exit(42) - } + } item - } + } } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 88b7ab9f52b82..cee6312572710 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -2,6 +2,8 @@ package spark import scala.collection.mutable.HashMap import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.time.{Span, Millis} import spark.SparkContext._ import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD} @@ -100,11 +102,26 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(rdd.collect().toList === List(1, 2, 3, 4)) } - test("remove RDD") { - sc = new SparkContext("local", "test") - val rdd = sc.makeRDD(Array(1,2,3,4), 2).cache() - sc.removeRDD(rdd.id) - assert(sc.persistentRdds.empty == true) + test("unpersist RDD") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + rdd.count + assert(sc.persistentRdds.isEmpty == false) + rdd.unpersist() + assert(sc.persistentRdds.isEmpty == true) + + failAfter(Span(3000, Millis)) { + try { + while (! sc.getRDDStorageInfo.isEmpty) { + Thread.sleep(200) + } + } catch { + case e: Exception => + // Do nothing. We might see exceptions because block manager + // is racing this thread to remove entries from the driver. + } + } + assert(sc.getRDDStorageInfo.isEmpty == true) } test("caching with failures") { From 34637b97ec7ebdd356653324f15345b00b3a2ac2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 16:12:37 -0700 Subject: [PATCH 520/696] Added SparkContext.cleanup back. Not sure why it was removed before ... --- core/src/main/scala/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b686c595b8b4b..401e55d615b81 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -746,7 +746,7 @@ class SparkContext( /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { - // do nothing. this needs to be removed. + persistentRdds.clearOldValues(cleanupTime) } } From 204eb32e14e8fce5e4b4cf602375ae9b4ed136c9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 16:14:58 -0700 Subject: [PATCH 521/696] Changed the type of the persistentRdds hashmap back to TimeStampedHashMap. --- core/src/main/scala/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 401e55d615b81..d7d450d9580b8 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -100,7 +100,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]] + private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) From 207afe4088219a0c7350b3f80eb60e86c97e140f Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 18 Apr 2013 12:08:11 -0700 Subject: [PATCH 522/696] Remove spark-repl's extraneous dependency on spark-streaming --- project/SparkBuild.scala | 2 +- repl/pom.xml | 14 -------------- 2 files changed, 1 insertion(+), 15 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f2410085d8941..190d72343587e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -29,7 +29,7 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) - lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn (streaming) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) diff --git a/repl/pom.xml b/repl/pom.xml index 038da5d9881c6..92a2020b48d0b 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -96,13 +96,6 @@ hadoop1 runtime - - org.spark-project - spark-streaming - ${project.version} - hadoop1 - runtime - org.apache.hadoop hadoop-core @@ -147,13 +140,6 @@ hadoop2 runtime - - org.spark-project - spark-streaming - ${project.version} - hadoop2 - runtime - org.apache.hadoop hadoop-core From 609a817f52d8db05711c0d4529dd1448ed8c4fe0 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 2 May 2013 06:44:33 +0530 Subject: [PATCH 523/696] Integrate review comments on pull request --- core/src/main/scala/spark/SparkEnv.scala | 9 ++-- core/src/main/scala/spark/Utils.scala | 4 +- .../scala/spark/scheduler/ResultTask.scala | 1 - .../spark/scheduler/ShuffleMapTask.scala | 1 - .../scheduler/cluster/ClusterScheduler.scala | 32 ++++++------ .../scheduler/cluster/TaskSetManager.scala | 50 +++++++++---------- 6 files changed, 47 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 5b4a4640109de..2ee25e547dedf 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -33,8 +33,7 @@ class SparkEnv ( // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort - var executorIdToHostPort: (String, String) => String - ) { + var executorIdToHostPort: Option[(String, String) => String]) { def stop() { httpFileServer.stop() @@ -52,12 +51,12 @@ class SparkEnv ( def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = { val env = SparkEnv.get - if (env.executorIdToHostPort == null) { + if (env.executorIdToHostPort.isEmpty) { // default to using host, not host port. Relevant to non cluster modes. return defaultHostPort } - env.executorIdToHostPort(executorId, defaultHostPort) + env.executorIdToHostPort.get(executorId, defaultHostPort) } } @@ -178,6 +177,6 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - null) + None) } } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 279daf04edfb3..0e348f8189b89 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -335,7 +335,6 @@ private object Utils extends Logging { retval } - /* // Used by DEBUG code : remove when all testing done private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$") def checkHost(host: String, message: String = "") { @@ -364,8 +363,8 @@ private object Utils extends Logging { // temp code for debug System.exit(-1) } - */ +/* // Once testing is complete in various modes, replace with this ? def checkHost(host: String, message: String = "") {} def checkHostPort(hostPort: String, message: String = "") {} @@ -374,6 +373,7 @@ private object Utils extends Logging { def logErrorWithStack(msg: String) { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } } +*/ def getUserNameFromEnvironment(): String = { SparkHadoopUtil.getUserNameFromEnvironment diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index c43cbe5ed4be8..83166bce224f3 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -70,7 +70,6 @@ private[spark] class ResultTask[T, U]( rdd.partitions(partition) } - // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 0b848af2f3156..4b36e71c32bc2 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -84,7 +84,6 @@ private[spark] class ShuffleMapTask( protected def this() = this(0, null, null, 0, null) - // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3c72ce42062ee..49fc449e8608d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -73,7 +73,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val activeExecutorIds = new HashSet[String] // TODO: We might want to remove this and merge it with execId datastructures - but later. - // Which hosts in the cluster are alive (contains hostPort's) - used for hyper local and local task locality. + // Which hosts in the cluster are alive (contains hostPort's) - used for instance local and node local task locality. private val hostPortsAlive = new HashSet[String] private val hostToAliveHostPorts = new HashMap[String, HashSet[String]] @@ -109,7 +109,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler // Will that be a design violation ? - SparkEnv.get.executorIdToHostPort = executorToHostPort + SparkEnv.get.executorIdToHostPort = Some(executorToHostPort) } def newTaskId(): Long = nextTaskId.getAndIncrement() @@ -240,7 +240,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { // Split offers based on host local, rack local and off-rack tasks. - val hyperLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + val instanceLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val hostLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val otherOffers = new HashMap[String, ArrayBuffer[Int]]() @@ -250,16 +250,16 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // DEBUG code Utils.checkHostPort(hostPort) - val numHyperLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) - if (numHyperLocalTasks > 0){ - val list = hyperLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) - for (j <- 0 until numHyperLocalTasks) list += i + val numInstanceLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) + if (numInstanceLocalTasks > 0){ + val list = instanceLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) + for (j <- 0 until numInstanceLocalTasks) list += i } val host = Utils.parseHostPort(hostPort)._1 val numHostLocalTasks = math.max(0, - // Remove hyper local tasks (which are also host local btw !) from this - math.min(manager.numPendingTasksForHost(hostPort) - numHyperLocalTasks, hostToAvailableCpus(host))) + // Remove instance local tasks (which are also host local btw !) from this + math.min(manager.numPendingTasksForHost(hostPort) - numInstanceLocalTasks, hostToAvailableCpus(host))) if (numHostLocalTasks > 0){ val list = hostLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numHostLocalTasks) list += i @@ -267,7 +267,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val numRackLocalTasks = math.max(0, // Remove host local tasks (which are also rack local btw !) from this - math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numHyperLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) + math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numInstanceLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) if (numRackLocalTasks > 0){ val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numRackLocalTasks) list += i @@ -280,19 +280,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } val offersPriorityList = new ArrayBuffer[Int]( - hyperLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) + instanceLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) - // First hyper local, then host local, then rack, then others + // First instance local, then host local, then rack, then others - // numHostLocalOffers contains count of both hyper local and host offers. + // numHostLocalOffers contains count of both instance local and host offers. val numHostLocalOffers = { - val hyperLocalPriorityList = ClusterScheduler.prioritizeContainers(hyperLocalOffers) - offersPriorityList ++= hyperLocalPriorityList + val instanceLocalPriorityList = ClusterScheduler.prioritizeContainers(instanceLocalOffers) + offersPriorityList ++= instanceLocalPriorityList val hostLocalPriorityList = ClusterScheduler.prioritizeContainers(hostLocalOffers) offersPriorityList ++= hostLocalPriorityList - hyperLocalPriorityList.size + hostLocalPriorityList.size + instanceLocalPriorityList.size + hostLocalPriorityList.size } val numRackLocalOffers = { val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index f5c0058554b7e..5f3faaa5c3173 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -13,17 +13,17 @@ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer -private[spark] object TaskLocality extends Enumeration("HYPER_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { +private[spark] object TaskLocality extends Enumeration("INSTANCE_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { - // hyper local is expected to be used ONLY within tasksetmanager for now. - val HYPER_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value + // instance local is expected to be used ONLY within tasksetmanager for now. + val INSTANCE_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { // Must not be the constraint. - assert (constraint != TaskLocality.HYPER_LOCAL) + assert (constraint != TaskLocality.INSTANCE_LOCAL) constraint match { case TaskLocality.HOST_LOCAL => condition == TaskLocality.HOST_LOCAL @@ -37,8 +37,8 @@ private[spark] object TaskLocality extends Enumeration("HYPER_LOCAL", "HOST_LOCA // better way to do this ? try { val retval = TaskLocality.withName(str) - // Must not specify HYPER_LOCAL ! - assert (retval != TaskLocality.HYPER_LOCAL) + // Must not specify INSTANCE_LOCAL ! + assert (retval != TaskLocality.INSTANCE_LOCAL) retval } catch { @@ -84,7 +84,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node (hyper local to container). These collections are actually + // List of pending tasks for each node (instance local to container). These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put @@ -142,12 +142,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } // Note that it follows the hierarchy. - // if we search for HOST_LOCAL, the output will include HYPER_LOCAL and - // if we search for RACK_LOCAL, it will include HYPER_LOCAL & HOST_LOCAL + // if we search for HOST_LOCAL, the output will include INSTANCE_LOCAL and + // if we search for RACK_LOCAL, it will include INSTANCE_LOCAL & HOST_LOCAL private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - if (TaskLocality.HYPER_LOCAL == taskLocality) { + if (TaskLocality.INSTANCE_LOCAL == taskLocality) { // straight forward comparison ! Special case it. val retval = new HashSet[String]() scheduler.synchronized { @@ -203,19 +203,19 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe private def addPendingTask(index: Int) { // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val hyperLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HYPER_LOCAL) + val instanceLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.INSTANCE_LOCAL) val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. - assert (hyperLocalLocations.size == 0) + assert (instanceLocalLocations.size == 0) assert (hostLocalLocations.size == 0) pendingTasksWithNoPrefs += index } else { - // hyper local locality - for (hostPort <- hyperLocalLocations) { + // instance local locality + for (hostPort <- instanceLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -223,7 +223,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostPortList += index } - // host locality (includes hyper local) + // host locality (includes instance local) for (hostPort <- hostLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -233,7 +233,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostList += index } - // rack locality (includes hyper local and host local) + // rack locality (includes instance local and host local) for (rackLocalHostPort <- rackLocalLocations) { // DEBUG Code Utils.checkHostPort(rackLocalHostPort) @@ -247,7 +247,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe allPendingTasks += index } - // Return the pending tasks list for a given host port (hyper local), or an empty list if + // Return the pending tasks list for a given host port (instance local), or an empty list if // there is no map entry for that host private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { // DEBUG Code @@ -269,7 +269,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) } - // Number of pending tasks for a given host Port (which would be hyper local) + // Number of pending tasks for a given host Port (which would be instance local) def numPendingTasksForHostPort(hostPort: String): Int = { getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) } @@ -352,9 +352,9 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - val hyperLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) - if (hyperLocalTask != None) { - return hyperLocalTask + val instanceLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (instanceLocalTask != None) { + return instanceLocalTask } val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) @@ -387,7 +387,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return findSpeculativeTask(hostPort, locality) } - private def isHyperLocalLocation(task: Task[_], hostPort: String): Boolean = { + private def isInstanceLocalLocation(task: Task[_], hostPort: String): Boolean = { Utils.checkHostPort(hostPort) val locs = task.preferredLocations @@ -443,7 +443,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val taskLocality = - if (isHyperLocalLocation(task, hostPort)) TaskLocality.HYPER_LOCAL else + if (isInstanceLocalLocation(task, hostPort)) TaskLocality.INSTANCE_LOCAL else if (isHostLocalLocation(task, hostPort)) TaskLocality.HOST_LOCAL else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else TaskLocality.ANY @@ -608,8 +608,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking hyper local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no hyper local node for the task) + // Note: NOT checking instance local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no instance local node for the task) for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) From c047f0e3adae59d7e388a1d42d940c3cd5714f82 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 26 Apr 2013 13:28:21 +0800 Subject: [PATCH 524/696] filter out Spark streaming block RDD and sort RDDInfo with id --- .../scala/spark/storage/StorageUtils.scala | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index dec47a9d4113b..8f52168c241b3 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -4,9 +4,9 @@ import spark.{Utils, SparkContext} import BlockManagerMasterActor.BlockStatus private[spark] -case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, +case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, blocks: Map[String, BlockStatus]) { - + def memUsed(blockPrefix: String = "") = { blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.memSize). reduceOption(_+_).getOrElse(0l) @@ -22,35 +22,40 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) { + numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) + extends Ordered[RDDInfo] { override def toString = { import Utils.memoryBytesToString "RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id, storageLevel.toString, numCachedPartitions, numPartitions, memoryBytesToString(memSize), memoryBytesToString(diskSize)) } + + override def compare(that: RDDInfo) = { + this.id - that.id + } } /* Helper methods for storage-related objects */ private[spark] object StorageUtils { - /* Given the current storage status of the BlockManager, returns information for each RDD */ - def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], + /* Given the current storage status of the BlockManager, returns information for each RDD */ + def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], sc: SparkContext) : Array[RDDInfo] = { - rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) + rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) } - /* Given a list of BlockStatus objets, returns information for each RDD */ - def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], + /* Given a list of BlockStatus objets, returns information for each RDD */ + def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], sc: SparkContext) : Array[RDDInfo] = { // Group by rddId, ignore the partition name - val groupedRddBlocks = infos.groupBy { case(k, v) => + val groupedRddBlocks = infos.filterKeys(_.startsWith("rdd_")).groupBy { case(k, v) => k.substring(0,k.lastIndexOf('_')) }.mapValues(_.values.toArray) // For each RDD, generate an RDDInfo object - groupedRddBlocks.map { case(rddKey, rddBlocks) => + val rddInfos = groupedRddBlocks.map { case(rddKey, rddBlocks) => // Add up memory and disk sizes val memSize = rddBlocks.map(_.memSize).reduce(_ + _) @@ -65,10 +70,14 @@ object StorageUtils { RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize) }.toArray + + scala.util.Sorting.quickSort(rddInfos) + + rddInfos } - /* Removes all BlockStatus object that are not part of a block prefix */ - def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus], + /* Removes all BlockStatus object that are not part of a block prefix */ + def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus], prefix: String) : Array[StorageStatus] = { storageStatusList.map { status => From 1b5aaeadc72ad5197c00897c41f670ea241d0235 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 2 May 2013 07:30:06 +0530 Subject: [PATCH 525/696] Integrate review comments 2 --- .../scheduler/cluster/ClusterScheduler.scala | 78 +++++++++---------- .../scheduler/cluster/TaskSetManager.scala | 74 +++++++++--------- .../scheduler/local/LocalScheduler.scala | 2 +- 3 files changed, 77 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 49fc449e8608d..cf4483f1442b1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -32,28 +32,28 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val TASK_REVIVAL_INTERVAL = System.getProperty("spark.tasks.revive.interval", "0").toLong /* - This property controls how aggressive we should be to modulate waiting for host local task scheduling. - To elaborate, currently there is a time limit (3 sec def) to ensure that spark attempts to wait for host locality of tasks before + This property controls how aggressive we should be to modulate waiting for node local task scheduling. + To elaborate, currently there is a time limit (3 sec def) to ensure that spark attempts to wait for node locality of tasks before scheduling on other nodes. We have modified this in yarn branch such that offers to task set happen in prioritized order : - host-local, rack-local and then others - But once all available host local (and no pref) tasks are scheduled, instead of waiting for 3 sec before + node-local, rack-local and then others + But once all available node local (and no pref) tasks are scheduled, instead of waiting for 3 sec before scheduling to other nodes (which degrades performance for time sensitive tasks and on larger clusters), we can modulate that : to also allow rack local nodes or any node. The default is still set to HOST - so that previous behavior is maintained. This is to allow tuning the tension between pulling rdd data off node and scheduling computation asap. TODO: rename property ? The value is one of - - HOST_LOCAL (default, no change w.r.t current behavior), + - NODE_LOCAL (default, no change w.r.t current behavior), - RACK_LOCAL and - ANY Note that this property makes more sense when used in conjugation with spark.tasks.revive.interval > 0 : else it is not very effective. Additional Note: For non trivial clusters, there is a 4x - 5x reduction in running time (in some of our experiments) based on whether - it is left at default HOST_LOCAL, RACK_LOCAL (if cluster is configured to be rack aware) or ANY. + it is left at default NODE_LOCAL, RACK_LOCAL (if cluster is configured to be rack aware) or ANY. If cluster is rack aware, then setting it to RACK_LOCAL gives best tradeoff and a 3x - 4x performance improvement while minimizing IO impact. Also, it brings down the variance in running time drastically. */ - val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "HOST_LOCAL")) + val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "NODE_LOCAL")) val activeTaskSets = new HashMap[String, TaskSetManager] var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -73,7 +73,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val activeExecutorIds = new HashSet[String] // TODO: We might want to remove this and merge it with execId datastructures - but later. - // Which hosts in the cluster are alive (contains hostPort's) - used for instance local and node local task locality. + // Which hosts in the cluster are alive (contains hostPort's) - used for process local and node local task locality. private val hostPortsAlive = new HashSet[String] private val hostToAliveHostPorts = new HashMap[String, HashSet[String]] @@ -217,9 +217,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - // merge availableCpus into hostToAvailableCpus block ? + // merge availableCpus into nodeToAvailableCpus block ? val availableCpus = offers.map(o => o.cores).toArray - val hostToAvailableCpus = { + val nodeToAvailableCpus = { val map = new HashMap[String, Int]() for (offer <- offers) { val hostPort = offer.hostPort @@ -239,9 +239,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { - // Split offers based on host local, rack local and off-rack tasks. - val instanceLocalOffers = new HashMap[String, ArrayBuffer[Int]]() - val hostLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + // Split offers based on node local, rack local and off-rack tasks. + val processLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + val nodeLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val otherOffers = new HashMap[String, ArrayBuffer[Int]]() @@ -250,29 +250,29 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // DEBUG code Utils.checkHostPort(hostPort) - val numInstanceLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) - if (numInstanceLocalTasks > 0){ - val list = instanceLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) - for (j <- 0 until numInstanceLocalTasks) list += i + val numProcessLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) + if (numProcessLocalTasks > 0){ + val list = processLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) + for (j <- 0 until numProcessLocalTasks) list += i } val host = Utils.parseHostPort(hostPort)._1 - val numHostLocalTasks = math.max(0, - // Remove instance local tasks (which are also host local btw !) from this - math.min(manager.numPendingTasksForHost(hostPort) - numInstanceLocalTasks, hostToAvailableCpus(host))) - if (numHostLocalTasks > 0){ - val list = hostLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) - for (j <- 0 until numHostLocalTasks) list += i + val numNodeLocalTasks = math.max(0, + // Remove process local tasks (which are also host local btw !) from this + math.min(manager.numPendingTasksForHost(hostPort) - numProcessLocalTasks, nodeToAvailableCpus(host))) + if (numNodeLocalTasks > 0){ + val list = nodeLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) + for (j <- 0 until numNodeLocalTasks) list += i } val numRackLocalTasks = math.max(0, - // Remove host local tasks (which are also rack local btw !) from this - math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numInstanceLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) + // Remove node local tasks (which are also rack local btw !) from this + math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numProcessLocalTasks - numNodeLocalTasks, nodeToAvailableCpus(host))) if (numRackLocalTasks > 0){ val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numRackLocalTasks) list += i } - if (numHostLocalTasks <= 0 && numRackLocalTasks <= 0){ + if (numNodeLocalTasks <= 0 && numRackLocalTasks <= 0){ // add to others list - spread even this across cluster. val list = otherOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) list += i @@ -280,19 +280,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } val offersPriorityList = new ArrayBuffer[Int]( - instanceLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) + processLocalOffers.size + nodeLocalOffers.size + rackLocalOffers.size + otherOffers.size) - // First instance local, then host local, then rack, then others + // First process local, then host local, then rack, then others - // numHostLocalOffers contains count of both instance local and host offers. - val numHostLocalOffers = { - val instanceLocalPriorityList = ClusterScheduler.prioritizeContainers(instanceLocalOffers) - offersPriorityList ++= instanceLocalPriorityList + // numNodeLocalOffers contains count of both process local and host offers. + val numNodeLocalOffers = { + val processLocalPriorityList = ClusterScheduler.prioritizeContainers(processLocalOffers) + offersPriorityList ++= processLocalPriorityList - val hostLocalPriorityList = ClusterScheduler.prioritizeContainers(hostLocalOffers) - offersPriorityList ++= hostLocalPriorityList + val nodeLocalPriorityList = ClusterScheduler.prioritizeContainers(nodeLocalOffers) + offersPriorityList ++= nodeLocalPriorityList - instanceLocalPriorityList.size + hostLocalPriorityList.size + processLocalPriorityList.size + nodeLocalPriorityList.size } val numRackLocalOffers = { val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) @@ -303,8 +303,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var lastLoop = false val lastLoopIndex = TASK_SCHEDULING_AGGRESSION match { - case TaskLocality.HOST_LOCAL => numHostLocalOffers - case TaskLocality.RACK_LOCAL => numRackLocalOffers + numHostLocalOffers + case TaskLocality.NODE_LOCAL => numNodeLocalOffers + case TaskLocality.RACK_LOCAL => numRackLocalOffers + numNodeLocalOffers case TaskLocality.ANY => offersPriorityList.size } @@ -343,8 +343,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // prevent more looping launchedTask = false } else if (!lastLoop && !launchedTask) { - // Do this only if TASK_SCHEDULING_AGGRESSION != HOST_LOCAL - if (TASK_SCHEDULING_AGGRESSION != TaskLocality.HOST_LOCAL) { + // Do this only if TASK_SCHEDULING_AGGRESSION != NODE_LOCAL + if (TASK_SCHEDULING_AGGRESSION != TaskLocality.NODE_LOCAL) { // fudge launchedTask to ensure we loop once more launchedTask = true // dont loop anymore diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 5f3faaa5c3173..ff4790e4cb068 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -13,21 +13,21 @@ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer -private[spark] object TaskLocality extends Enumeration("INSTANCE_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { +private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { - // instance local is expected to be used ONLY within tasksetmanager for now. - val INSTANCE_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value + // process local is expected to be used ONLY within tasksetmanager for now. + val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { // Must not be the constraint. - assert (constraint != TaskLocality.INSTANCE_LOCAL) + assert (constraint != TaskLocality.PROCESS_LOCAL) constraint match { - case TaskLocality.HOST_LOCAL => condition == TaskLocality.HOST_LOCAL - case TaskLocality.RACK_LOCAL => condition == TaskLocality.HOST_LOCAL || condition == TaskLocality.RACK_LOCAL + case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL + case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL // For anything else, allow case _ => true } @@ -37,15 +37,15 @@ private[spark] object TaskLocality extends Enumeration("INSTANCE_LOCAL", "HOST_L // better way to do this ? try { val retval = TaskLocality.withName(str) - // Must not specify INSTANCE_LOCAL ! - assert (retval != TaskLocality.INSTANCE_LOCAL) + // Must not specify PROCESS_LOCAL ! + assert (retval != TaskLocality.PROCESS_LOCAL) retval } catch { case nEx: NoSuchElementException => { - logWarning("Invalid task locality specified '" + str + "', defaulting to HOST_LOCAL"); + logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); // default to preserve earlier behavior - HOST_LOCAL + NODE_LOCAL } } } @@ -84,7 +84,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node (instance local to container). These collections are actually + // List of pending tasks for each node (process local to container). These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put @@ -142,12 +142,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } // Note that it follows the hierarchy. - // if we search for HOST_LOCAL, the output will include INSTANCE_LOCAL and - // if we search for RACK_LOCAL, it will include INSTANCE_LOCAL & HOST_LOCAL + // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and + // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - if (TaskLocality.INSTANCE_LOCAL == taskLocality) { + if (TaskLocality.PROCESS_LOCAL == taskLocality) { // straight forward comparison ! Special case it. val retval = new HashSet[String]() scheduler.synchronized { @@ -162,7 +162,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } val taskPreferredLocations = - if (TaskLocality.HOST_LOCAL == taskLocality) { + if (TaskLocality.NODE_LOCAL == taskLocality) { _taskPreferredLocations } else { assert (TaskLocality.RACK_LOCAL == taskLocality) @@ -203,19 +203,19 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe private def addPendingTask(index: Int) { // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val instanceLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.INSTANCE_LOCAL) - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) + val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. - assert (instanceLocalLocations.size == 0) + assert (processLocalLocations.size == 0) assert (hostLocalLocations.size == 0) pendingTasksWithNoPrefs += index } else { - // instance local locality - for (hostPort <- instanceLocalLocations) { + // process local locality + for (hostPort <- processLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -223,7 +223,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostPortList += index } - // host locality (includes instance local) + // host locality (includes process local) for (hostPort <- hostLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -233,7 +233,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostList += index } - // rack locality (includes instance local and host local) + // rack locality (includes process local and host local) for (rackLocalHostPort <- rackLocalLocations) { // DEBUG Code Utils.checkHostPort(rackLocalHostPort) @@ -247,7 +247,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe allPendingTasks += index } - // Return the pending tasks list for a given host port (instance local), or an empty list if + // Return the pending tasks list for a given host port (process local), or an empty list if // there is no map entry for that host private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { // DEBUG Code @@ -269,7 +269,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) } - // Number of pending tasks for a given host Port (which would be instance local) + // Number of pending tasks for a given host Port (which would be process local) def numPendingTasksForHostPort(hostPort: String): Int = { getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) } @@ -305,13 +305,13 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // task must have a preference for this host/rack/no preferred locations at all. private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - assert (TaskLocality.isAllowed(locality, TaskLocality.HOST_LOCAL)) + assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set if (speculatableTasks.size > 0) { val localTask = speculatableTasks.find { index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) val attemptLocs = taskAttempts(index).map(_.hostPort) (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) } @@ -352,9 +352,9 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - val instanceLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) - if (instanceLocalTask != None) { - return instanceLocalTask + val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (processLocalTask != None) { + return processLocalTask } val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) @@ -387,7 +387,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return findSpeculativeTask(hostPort, locality) } - private def isInstanceLocalLocation(task: Task[_], hostPort: String): Boolean = { + private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = { Utils.checkHostPort(hostPort) val locs = task.preferredLocations @@ -433,7 +433,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val locality = if (overrideLocality != null) overrideLocality else { // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.HOST_LOCAL else TaskLocality.ANY + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY } findTask(hostPort, locality) match { @@ -443,8 +443,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val taskLocality = - if (isInstanceLocalLocation(task, hostPort)) TaskLocality.INSTANCE_LOCAL else - if (isHostLocalLocation(task, hostPort)) TaskLocality.HOST_LOCAL else + if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else + if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else TaskLocality.ANY val prefStr = taskLocality.toString @@ -456,7 +456,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) - if (TaskLocality.HOST_LOCAL == taskLocality) { + if (TaskLocality.NODE_LOCAL == taskLocality) { lastPreferredLaunchTime = time } // Serialize and return the task @@ -608,11 +608,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking instance local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no instance local node for the task) + // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no process local node for the task) for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) if (newLocs.isEmpty) { pendingTasksWithNoPrefs += index } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index f060a940a9269..53dd6fbe13278 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -53,7 +53,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon def runTask(task: Task[_], idInJob: Int, attemptId: Int) { logInfo("Running " + task) - val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.HOST_LOCAL) + val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { From dfde9ce9dde0a151d42f7aecb826b40a4c98b459 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 2 May 2013 07:41:33 +0530 Subject: [PATCH 526/696] comment out debug versions of checkHost, etc from Utils - which were used to test --- core/src/main/scala/spark/Utils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 0e348f8189b89..c1495d531714a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -335,6 +335,7 @@ private object Utils extends Logging { retval } +/* // Used by DEBUG code : remove when all testing done private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$") def checkHost(host: String, message: String = "") { @@ -363,8 +364,8 @@ private object Utils extends Logging { // temp code for debug System.exit(-1) } +*/ -/* // Once testing is complete in various modes, replace with this ? def checkHost(host: String, message: String = "") {} def checkHostPort(hostPort: String, message: String = "") {} @@ -373,7 +374,6 @@ private object Utils extends Logging { def logErrorWithStack(msg: String) { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } } -*/ def getUserNameFromEnvironment(): String = { SparkHadoopUtil.getUserNameFromEnvironment From 98df9d28536f5208530488a316df9401e16490bd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 20:17:09 -0700 Subject: [PATCH 527/696] Added removeRdd function in BlockManager. --- core/src/main/scala/spark/RDD.scala | 15 +++-------- core/src/main/scala/spark/SparkContext.scala | 8 +++--- .../spark/storage/BlockManagerMaster.scala | 16 ++++++++++++ .../scala/spark/storage/BlockManagerUI.scala | 4 +-- .../spark/storage/BlockManagerSuite.scala | 25 +++++++++++++++++++ 5 files changed, 51 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index c77f9915c021f..fd14ef17f1ce6 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -107,7 +107,7 @@ abstract class RDD[T: ClassManifest]( // ======================================================================= /** A unique ID for this RDD (within its SparkContext). */ - val id = sc.newRddId() + val id: Int = sc.newRddId() /** A friendly name for this RDD */ var name: String = null @@ -120,7 +120,8 @@ abstract class RDD[T: ClassManifest]( /** * Set this RDD's storage level to persist its values across operations after the first time - * it is computed. Can only be called once on each RDD. + * it is computed. This can only be used to assign a new storage level if the RDD does not + * have a storage level set yet.. */ def persist(newLevel: StorageLevel): RDD[T] = { // TODO: Handle changes of StorageLevel @@ -143,15 +144,7 @@ abstract class RDD[T: ClassManifest]( /** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */ def unpersist(): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - val rddBlockPrefix = "rdd_" + id + "_" - // Get the list of blocks in block manager, and remove ones that are part of this RDD. - // The runtime complexity is linear to the number of blocks persisted in the cluster. - // It could be expensive if the cluster is large and has a lot of blocks persisted. - sc.getExecutorStorageStatus().flatMap(_.blocks).foreach { case(blockId, status) => - if (blockId.startsWith(rddBlockPrefix)) { - sc.env.blockManager.master.removeBlock(blockId) - } - } + sc.env.blockManager.master.removeRdd(id) sc.persistentRdds.remove(id) storageLevel = StorageLevel.NONE this diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index d7d450d9580b8..2ae4ad8659ff6 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -508,18 +508,18 @@ class SparkContext( * Return information about what RDDs are cached, if they are in mem or on disk, how much space * they take, etc. */ - def getRDDStorageInfo(): Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus(), this) + def getRDDStorageInfo: Array[RDDInfo] = { + StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } - def getStageInfo(): Map[Stage,StageInfo] = { + def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos } /** * Return information about blocks stored in all of the slaves */ - def getExecutorStorageStatus(): Array[StorageStatus] = { + def getExecutorStorageStatus: Array[StorageStatus] = { env.blockManager.master.getStorageStatus } diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 6fae62d373ba9..ac26c16867045 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -15,6 +15,7 @@ import akka.util.duration._ import spark.{Logging, SparkException, Utils} + private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt @@ -87,6 +88,21 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi askDriverWithReply(RemoveBlock(blockId)) } + /** + * Remove all blocks belonging to the given RDD. + */ + def removeRdd(rddId: Int) { + val rddBlockPrefix = "rdd_" + rddId + "_" + // Get the list of blocks in block manager, and remove ones that are part of this RDD. + // The runtime complexity is linear to the number of blocks persisted in the cluster. + // It could be expensive if the cluster is large and has a lot of blocks persisted. + getStorageStatus.flatMap(_.blocks).foreach { case(blockId, status) => + if (blockId.startsWith(rddBlockPrefix)) { + removeBlock(blockId) + } + } + } + /** * Return the memory status for each block manager, in the form of a map from * the block manager's id to two long values. The first value is the maximum diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index c9e4519efe889..07da5720440cd 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -45,7 +45,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, path("") { completeWith { // Request the current storage status from the Master - val storageStatusList = sc.getExecutorStorageStatus() + val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) @@ -60,7 +60,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, parameter("id") { id => completeWith { val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus() + val storageStatusList = sc.getExecutorStorageStatus val filteredStorageStatusList = StorageUtils. filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 5a11a4483bb1c..9fe0de665c09e 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -207,6 +207,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("removing rdd") { + store = new BlockManager("", actorSystem, master, serializer, 2000) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + // Putting a1, a2 and a3 in memory. + store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY) + store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY) + master.removeRdd(0) + + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("rdd_0_0") should be (None) + master.getLocations("rdd_0_0") should have size 0 + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("rdd_0_1") should be (None) + master.getLocations("rdd_0_1") should have size 0 + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("nonrddblock") should not be (None) + master.getLocations("nonrddblock") should have size (1) + } + } + test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) store = new BlockManager("", actorSystem, master, serializer, 2000) From 4a318774088f829fe54c3ef0b5f565a845631b4e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 20:31:54 -0700 Subject: [PATCH 528/696] Added the unpersist api to JavaRDD. --- core/src/main/scala/spark/api/java/JavaRDD.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index e29f1e58990b8..eb81ed64cd62c 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -14,12 +14,18 @@ JavaRDDLike[T, JavaRDD[T]] { /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): JavaRDD[T] = wrapRDD(rdd.cache()) - /** + /** * Set this RDD's storage level to persist its values across operations after the first time - * it is computed. Can only be called once on each RDD. + * it is computed. This can only be used to assign a new storage level if the RDD does not + * have a storage level set yet.. */ def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel)) + /** + * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. + */ + def unpersist(): JavaRDD[T] = wrapRDD(rdd.unpersist()) + // Transformations (return a new RDD) /** @@ -31,7 +37,7 @@ JavaRDDLike[T, JavaRDD[T]] { * Return a new RDD containing the distinct elements in this RDD. */ def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions)) - + /** * Return a new RDD containing only the elements that satisfy a predicate. */ @@ -54,7 +60,7 @@ JavaRDDLike[T, JavaRDD[T]] { */ def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) - + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). @@ -63,7 +69,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** * Return an RDD with the elements from `this` that are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ From c847dd3da29483fede326cb9821b0d33f735137e Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 19 Mar 2013 15:08:22 -0700 Subject: [PATCH 529/696] Don't accept generated temp directory names that can't be created successfully. --- core/src/main/scala/spark/storage/DiskStore.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c9553d2e0fa9e..215c25132b8a4 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -168,8 +168,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) localDir = new File(rootDir, "spark-local-" + localDirId) if (!localDir.exists) { - localDir.mkdirs() - foundLocalDir = true + foundLocalDir = localDir.mkdirs() } } catch { case e: Exception => From 11589c39d9f75e9757ba1717c5202f77d30031b2 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Fri, 3 May 2013 12:23:30 +0530 Subject: [PATCH 530/696] Fix ZippedRDD as part Matei's suggestion --- core/src/main/scala/spark/rdd/ZippedRDD.scala | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index e80250a99bdf0..51573fe68a701 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,6 +1,6 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} @@ -49,9 +49,20 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( override def getPreferredLocations(s: Partition): Seq[String] = { val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - // TODO: becomes complicated - intersect on hostPort if available, else fallback to host (removing intersected hostPort's). - // Since I am not very sure about this RDD, leaving it to others to comment better ! - rdd1.preferredLocations(partition1).intersect(rdd2.preferredLocations(partition2)) + val pref1 = rdd1.preferredLocations(partition1) + val pref2 = rdd2.preferredLocations(partition2) + + // both partitions are instance local. + val instanceLocalLocations = pref1.intersect(pref2) + + // remove locations which are already handled via instanceLocalLocations, and intersect where both partitions are node local. + val nodeLocalPref1 = pref1.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val nodeLocalPref2 = pref2.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val nodeLocalLocations = nodeLocalPref1.intersect(nodeLocalPref2) + + + // Can have mix of instance local (hostPort) and node local (host) locations as preference ! + instanceLocalLocations ++ nodeLocalLocations } override def clearDependencies() { From 2bc895a829caa459e032e12e1d117994dd510a5c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 May 2013 01:02:16 -0700 Subject: [PATCH 531/696] Updated according to Matei's code review comment. --- .../src/main/scala/spark/ShuffleFetcher.scala | 2 +- core/src/main/scala/spark/SparkEnv.scala | 10 +++-- .../main/scala/spark/rdd/CoGroupedRDD.scala | 3 +- .../main/scala/spark/rdd/ShuffledRDD.scala | 5 +-- .../main/scala/spark/rdd/SubtractedRDD.scala | 4 +- .../spark/scheduler/ShuffleMapTask.scala | 5 +-- .../scala/spark/serializer/Serializer.scala | 42 ----------------- .../spark/serializer/SerializerManager.scala | 45 +++++++++++++++++++ .../main/scala/spark/storage/DiskStore.scala | 36 ++++++++------- .../spark/storage/ShuffleBlockManager.scala | 34 +++++++------- 10 files changed, 98 insertions(+), 88 deletions(-) create mode 100644 core/src/main/scala/spark/serializer/SerializerManager.scala diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 49addc0c1075f..9513a001263f1 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -10,7 +10,7 @@ private[spark] abstract class ShuffleFetcher { * @return An iterator over the elements of the fetched shuffle outputs. */ def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, - serializer: Serializer = Serializer.default): Iterator[(K,V)] + serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[(K,V)] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 8ba52245fa507..2fa97cd829ad9 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -7,7 +7,7 @@ import spark.broadcast.BroadcastManager import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager -import spark.serializer.Serializer +import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils @@ -21,6 +21,7 @@ import spark.util.AkkaUtils class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, + val serializerManager: SerializerManager, val serializer: Serializer, val closureSerializer: Serializer, val cacheManager: CacheManager, @@ -92,10 +93,12 @@ object SparkEnv extends Logging { Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } - val serializer = Serializer.setDefault( + val serializerManager = new SerializerManager + + val serializer = serializerManager.setDefault( System.getProperty("spark.serializer", "spark.JavaSerializer")) - val closureSerializer = Serializer.get( + val closureSerializer = serializerManager.get( System.getProperty("spark.closure.serializer", "spark.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): ActorRef = { @@ -155,6 +158,7 @@ object SparkEnv extends Logging { new SparkEnv( executorId, actorSystem, + serializerManager, serializer, closureSerializer, cacheManager, diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 9e996e9958b0e..7599ba1a0224b 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -8,7 +8,6 @@ import scala.collection.mutable.ArrayBuffer import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} -import spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -114,7 +113,7 @@ class CoGroupedRDD[K]( } } - val ser = Serializer.get(serializerClass) + val ser = SparkEnv.get.serializerManager.get(serializerClass) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 8175e23eff264..c7d1926b83d82 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -2,7 +2,6 @@ package spark.rdd import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} import spark.SparkContext._ -import spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -32,7 +31,7 @@ class ShuffledRDD[K, V]( override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[K, V]( - shuffledId, split.index, context.taskMetrics, Serializer.get(serializerClass)) + SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics, + SparkEnv.get.serializerManager.get(serializerClass)) } } diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index f60c35c38ec31..8a9efc5da21f2 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -11,7 +11,7 @@ import spark.Partition import spark.SparkEnv import spark.ShuffleDependency import spark.OneToOneDependency -import spark.serializer.Serializer + /** * An optimized version of cogroup for set difference/subtraction. @@ -68,7 +68,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val serializer = Serializer.get(serializerClass) + val serializer = SparkEnv.get.serializerManager.get(serializerClass) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 124d2d7e26129..f097213ab5aef 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -14,7 +14,6 @@ import com.ning.compress.lzf.LZFOutputStream import spark._ import spark.executor.ShuffleWriteMetrics -import spark.serializer.Serializer import spark.storage._ import spark.util.{TimeStampedHashMap, MetadataCleaner} @@ -139,12 +138,12 @@ private[spark] class ShuffleMapTask( metrics = Some(taskContext.taskMetrics) val blockManager = SparkEnv.get.blockManager - var shuffle: ShuffleBlockManager#Shuffle = null + var shuffle: ShuffleBlocks = null var buckets: ShuffleWriterGroup = null try { // Obtain all the block writers for shuffle blocks. - val ser = Serializer.get(dep.serializerClass) + val ser = SparkEnv.get.serializerManager.get(dep.serializerClass) shuffle = blockManager.shuffleBlockManager.forShuffle(dep.shuffleId, numOutputSplits, ser) buckets = shuffle.acquireWriters(partition) diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index 77b1a1a434cb3..2ad73b711de74 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -2,7 +2,6 @@ package spark.serializer import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentHashMap import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream @@ -19,47 +18,6 @@ trait Serializer { } -/** - * A singleton object that can be used to fetch serializer objects based on the serializer - * class name. If a previous instance of the serializer object has been created, the get - * method returns that instead of creating a new one. - */ -object Serializer { - - private val serializers = new ConcurrentHashMap[String, Serializer] - private var _default: Serializer = _ - - def default = _default - - def setDefault(clsName: String): Serializer = { - _default = get(clsName) - _default - } - - def get(clsName: String): Serializer = { - if (clsName == null) { - default - } else { - var serializer = serializers.get(clsName) - if (serializer != null) { - // If the serializer has been created previously, reuse that. - serializer - } else this.synchronized { - // Otherwise, create a new one. But make sure no other thread has attempted - // to create another new one at the same time. - serializer = serializers.get(clsName) - if (serializer == null) { - val clsLoader = Thread.currentThread.getContextClassLoader - serializer = Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] - serializers.put(clsName, serializer) - } - serializer - } - } - } -} - - /** * An instance of a serializer, for use by one thread at a time. */ diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/spark/serializer/SerializerManager.scala new file mode 100644 index 0000000000000..60b2aac797a13 --- /dev/null +++ b/core/src/main/scala/spark/serializer/SerializerManager.scala @@ -0,0 +1,45 @@ +package spark.serializer + +import java.util.concurrent.ConcurrentHashMap + + +/** + * A service that returns a serializer object given the serializer's class name. If a previous + * instance of the serializer object has been created, the get method returns that instead of + * creating a new one. + */ +private[spark] class SerializerManager { + + private val serializers = new ConcurrentHashMap[String, Serializer] + private var _default: Serializer = _ + + def default = _default + + def setDefault(clsName: String): Serializer = { + _default = get(clsName) + _default + } + + def get(clsName: String): Serializer = { + if (clsName == null) { + default + } else { + var serializer = serializers.get(clsName) + if (serializer != null) { + // If the serializer has been created previously, reuse that. + serializer + } else this.synchronized { + // Otherwise, create a new one. But make sure no other thread has attempted + // to create another new one at the same time. + serializer = serializers.get(clsName) + if (serializer == null) { + val clsLoader = Thread.currentThread.getContextClassLoader + serializer = + Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] + serializers.put(clsName, serializer) + } + serializer + } + } + } +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 4cddcc86fc3ad..498bc9eeb6a9f 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -2,6 +2,7 @@ package spark.storage import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} import java.nio.ByteBuffer +import java.nio.channels.FileChannel import java.nio.channels.FileChannel.MapMode import java.util.{Random, Date} import java.text.SimpleDateFormat @@ -26,14 +27,16 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private val f: File = createFile(blockId /*, allowAppendExisting */) - private var repositionableStream: FastBufferedOutputStream = null + // The file channel, used for repositioning / truncating the file. + private var channel: FileChannel = null private var bs: OutputStream = null private var objOut: SerializationStream = null - private var validLength = 0L + private var lastValidPosition = 0L override def open(): DiskBlockObjectWriter = { - repositionableStream = new FastBufferedOutputStream(new FileOutputStream(f)) - bs = blockManager.wrapForCompression(blockId, repositionableStream) + val fos = new FileOutputStream(f, true) + channel = fos.getChannel() + bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos)) objOut = serializer.newInstance().serializeStream(bs) this } @@ -41,9 +44,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def close() { objOut.close() bs.close() - objOut = null + channel = null bs = null - repositionableStream = null + objOut = null // Invoke the close callback handler. super.close() } @@ -54,25 +57,23 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Return the number of bytes written for this commit. override def commit(): Long = { bs.flush() - validLength = repositionableStream.position() - validLength + val prevPos = lastValidPosition + lastValidPosition = channel.position() + lastValidPosition - prevPos } override def revertPartialWrites() { - // Flush the outstanding writes and delete the file. - objOut.close() - bs.close() - objOut = null - bs = null - repositionableStream = null - f.delete() + // Discard current writes. We do this by flushing the outstanding writes and + // truncate the file to the last valid position. + bs.flush() + channel.truncate(lastValidPosition) } override def write(value: Any) { objOut.writeObject(value) } - override def size(): Long = validLength + override def size(): Long = lastValidPosition } val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @@ -86,7 +87,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() - def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int): BlockObjectWriter = { + def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int) + : BlockObjectWriter = { new DiskBlockObjectWriter(blockId, serializer, bufferSize) } diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 1903df0817580..49eabfb0d2150 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -8,26 +8,30 @@ class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter]) private[spark] -class ShuffleBlockManager(blockManager: BlockManager) { +trait ShuffleBlocks { + def acquireWriters(mapId: Int): ShuffleWriterGroup + def releaseWriters(group: ShuffleWriterGroup) +} - def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): Shuffle = { - new Shuffle(shuffleId, numBuckets, serializer) - } - class Shuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer) { +private[spark] +class ShuffleBlockManager(blockManager: BlockManager) { - // Get a group of writers for a map task. - def acquireWriters(mapId: Int): ShuffleWriterGroup = { - val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 - val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => - val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) - blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() + def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): ShuffleBlocks = { + new ShuffleBlocks { + // Get a group of writers for a map task. + override def acquireWriters(mapId: Int): ShuffleWriterGroup = { + val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => + val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) + blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() + } + new ShuffleWriterGroup(mapId, writers) } - new ShuffleWriterGroup(mapId, writers) - } - def releaseWriters(group: ShuffleWriterGroup) = { - // Nothing really to release here. + override def releaseWriters(group: ShuffleWriterGroup) = { + // Nothing really to release here. + } } } } From 6fae936088d2a50606ba5082cee4a3c3a98a2b01 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 May 2013 22:30:06 -0700 Subject: [PATCH 532/696] applications (aka drivers) send their webUI address to master when registering so it can be displayed in the master web ui --- .../spark/deploy/ApplicationDescription.scala | 4 +++- .../scala/spark/deploy/client/TestClient.scala | 2 +- .../spark/deploy/master/ApplicationInfo.scala | 8 +++++++- .../main/scala/spark/deploy/master/Master.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 3 ++- .../scala/spark/storage/BlockManagerUI.scala | 17 +++++++++-------- .../spark/deploy/master/app_details.scala.html | 1 + 7 files changed, 24 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 6659e53b25f37..bb9e7b3bba6f8 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -5,7 +5,9 @@ private[spark] class ApplicationDescription( val cores: Int, val memoryPerSlave: Int, val command: Command, - val sparkHome: String) + val sparkHome: String, + val appUIHost: String, + val appUIPort: Int) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index ad92532b5849a..e4ab01dd2a2a2 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -25,7 +25,7 @@ private[spark] object TestClient { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home") + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "localhost", 0) val listener = new TestListener val client = new Client(actorSystem, url, desc, listener) client.start() diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3591a9407237a..3ee1b60351054 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -10,7 +10,9 @@ private[spark] class ApplicationInfo( val id: String, val desc: ApplicationDescription, val submitDate: Date, - val driver: ActorRef) + val driver: ActorRef, + val appUIHost: String, + val appUIPort: Int) { var state = ApplicationState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] @@ -60,4 +62,8 @@ private[spark] class ApplicationInfo( System.currentTimeMillis() - startTime } } + + + def appUIAddress = "http://" + this.appUIHost + ":" + this.appUIPort + } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 160afe5239d98..9f2d3da495bd4 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -244,7 +244,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver) + val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUIHost, desc.appUIPort) apps += app idToApp(app.id) = app actorToApp(driver) = app diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 0b8922d139490..5d7d1feb7456a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -31,7 +31,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) - val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome) + val appDesc = + new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.host, sc.ui.port) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 07da5720440cd..13158e426245d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -20,19 +20,20 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val STATIC_RESOURCE_DIR = "spark/deploy/static" implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val host = Utils.localHostName() + val port = if (System.getProperty("spark.ui.port") != null) { + System.getProperty("spark.ui.port").toInt + } else { + // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which + // random port it bound to, so we have to try to find a local one by creating a socket. + Utils.findFreePort() + } /** Start a HTTP server to run the Web interface */ def start() { try { - val port = if (System.getProperty("spark.ui.port") != null) { - System.getProperty("spark.ui.port").toInt - } else { - // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which - // random port it bound to, so we have to try to find a local one by creating a socket. - Utils.findFreePort() - } AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer") - logInfo("Started BlockManager web UI at http://%s:%d".format(Utils.localHostName(), port)) + logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 301a7e212495d..02086b476fe6c 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -22,6 +22,7 @@
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • +
  • Application Detail UI
  • From bb8a434f9db177d764d169a7273c66ed01c066c1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 3 May 2013 15:14:02 -0700 Subject: [PATCH 533/696] Add zipPartitions to Java API. --- .../scala/spark/api/java/JavaRDDLike.scala | 15 +++++++++++ .../api/java/function/FlatMapFunction2.scala | 11 ++++++++ core/src/test/scala/spark/JavaAPISuite.java | 26 +++++++++++++++++++ 3 files changed, 52 insertions(+) create mode 100644 core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index d884529d7a6f5..9b74d1226f1d9 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -182,6 +182,21 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classManifest))(classManifest, other.classManifest) } + /** + * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by + * applying a function to the zipped partitions. Assumes that all the RDDs have the + * *same number of partitions*, but does *not* require them to have the same number + * of elements in each partition. + */ + def zipPartitions[U, V]( + f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V], + other: JavaRDDLike[U, _]): JavaRDD[V] = { + def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator( + f.apply(asJavaIterator(x), asJavaIterator(y)).iterator()) + JavaRDD.fromRDD( + rdd.zipPartitions(fn, other.rdd)(other.classManifest, f.elementType()))(f.elementType()) + } + // Actions (launch a job to return a value to the user program) /** diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala new file mode 100644 index 0000000000000..6044043add316 --- /dev/null +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala @@ -0,0 +1,11 @@ +package spark.api.java.function + +/** + * A function that takes two inputs and returns zero or more output records. + */ +abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Iterable[C]] { + @throws(classOf[Exception]) + def call(a: A, b:B) : java.lang.Iterable[C] + + def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]] +} diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index d3dcd3bbebe7e..93bb69b41c790 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -632,6 +632,32 @@ public Double call(Integer x) { zipped.count(); } + @Test + public void zipPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); + JavaRDD rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); + FlatMapFunction2, Iterator, Integer> sizesFn = + new FlatMapFunction2, Iterator, Integer>() { + @Override + public Iterable call(Iterator i, Iterator s) { + int sizeI = 0; + int sizeS = 0; + while (i.hasNext()) { + sizeI += 1; + i.next(); + } + while (s.hasNext()) { + sizeS += 1; + s.next(); + } + return Arrays.asList(sizeI, sizeS); + } + }; + + JavaRDD sizes = rdd1.zipPartitions(sizesFn, rdd2); + Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); + } + @Test public void accumulators() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); From 2274ad0786b758e3170e96815e9693ea27635f06 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 3 May 2013 16:29:36 -0700 Subject: [PATCH 534/696] Fix flaky test by changing catch and adding sleep --- core/src/test/scala/spark/DistributedSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index ab3e197035712..a13c88cfb4bd5 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -261,9 +261,9 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter val data = sc.parallelize(Seq(true, false, false, false), 4) data.persist(StorageLevel.MEMORY_ONLY_2) data.count - assert(sc.persistentRdds.isEmpty == false) + assert(sc.persistentRdds.isEmpty === false) data.unpersist() - assert(sc.persistentRdds.isEmpty == true) + assert(sc.persistentRdds.isEmpty === true) failAfter(Span(3000, Millis)) { try { @@ -271,12 +271,12 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter Thread.sleep(200) } } catch { - case e: Exception => + case _ => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } } - assert(sc.getRDDStorageInfo.isEmpty == true) + assert(sc.getRDDStorageInfo.isEmpty === true) } } From edb57c8331738403d66c15ed99996e8bfb0488f7 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sat, 4 May 2013 19:47:45 +0530 Subject: [PATCH 535/696] Add support for instance local in getPreferredLocations of ZippedPartitionsBaseRDD. Add comments to both ZippedPartitionsBaseRDD and ZippedRDD to better describe the potential problem with the approach --- .../scala/spark/rdd/ZippedPartitionsRDD.scala | 28 +++++++++++++++++-- core/src/main/scala/spark/rdd/ZippedRDD.scala | 20 ++++++++----- 2 files changed, 38 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index fc3f29ffcda3e..dd9f3c2680974 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -1,6 +1,6 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class ZippedPartitionsPartition( @@ -38,9 +38,31 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( } override def getPreferredLocations(s: Partition): Seq[String] = { + // Note that as number of rdd's increase and/or number of slaves in cluster increase, the computed preferredLocations below + // become diminishingly small : so we might need to look at alternate strategies to alleviate this. + // If there are no (or very small number of preferred locations), we will end up transferred the blocks to 'any' node in the + // cluster - paying with n/w and cache cost. + // Maybe pick a node which figures max amount of time ? + // Choose node which is hosting 'larger' of some subset of blocks ? + // Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible) val splits = s.asInstanceOf[ZippedPartitionsPartition].partitions - val preferredLocations = rdds.zip(splits).map(x => x._1.preferredLocations(x._2)) - preferredLocations.reduce((x, y) => x.intersect(y)) + val rddSplitZip = rdds.zip(splits) + + // exact match. + val exactMatchPreferredLocations = rddSplitZip.map(x => x._1.preferredLocations(x._2)) + val exactMatchLocations = exactMatchPreferredLocations.reduce((x, y) => x.intersect(y)) + + // Remove exact match and then do host local match. + val otherNodePreferredLocations = rddSplitZip.map(x => { + x._1.preferredLocations(x._2).map(hostPort => { + val host = Utils.parseHostPort(hostPort)._1 + + if (exactMatchLocations.contains(host)) null else host + }).filter(_ != null) + }) + val otherNodeLocalLocations = otherNodePreferredLocations.reduce((x, y) => x.intersect(y)) + + otherNodeLocalLocations ++ exactMatchLocations } override def clearDependencies() { diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 51573fe68a701..f728e93d2452e 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -48,21 +48,27 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( } override def getPreferredLocations(s: Partition): Seq[String] = { + // Note that as number of slaves in cluster increase, the computed preferredLocations can become small : so we might need + // to look at alternate strategies to alleviate this. (If there are no (or very small number of preferred locations), we + // will end up transferred the blocks to 'any' node in the cluster - paying with n/w and cache cost. + // Maybe pick one or the other ? (so that atleast one block is local ?). + // Choose node which is hosting 'larger' of the blocks ? + // Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible) val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions val pref1 = rdd1.preferredLocations(partition1) val pref2 = rdd2.preferredLocations(partition2) - // both partitions are instance local. - val instanceLocalLocations = pref1.intersect(pref2) + // exact match - instance local and host local. + val exactMatchLocations = pref1.intersect(pref2) - // remove locations which are already handled via instanceLocalLocations, and intersect where both partitions are node local. - val nodeLocalPref1 = pref1.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) - val nodeLocalPref2 = pref2.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) - val nodeLocalLocations = nodeLocalPref1.intersect(nodeLocalPref2) + // remove locations which are already handled via exactMatchLocations, and intersect where both partitions are node local. + val otherNodeLocalPref1 = pref1.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val otherNodeLocalPref2 = pref2.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val otherNodeLocalLocations = otherNodeLocalPref1.intersect(otherNodeLocalPref2) // Can have mix of instance local (hostPort) and node local (host) locations as preference ! - instanceLocalLocations ++ nodeLocalLocations + exactMatchLocations ++ otherNodeLocalLocations } override def clearDependencies() { From 02e8cfa61792f296555c7ed16613a91d895181a1 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Sat, 4 May 2013 12:31:30 -0500 Subject: [PATCH 536/696] HBase example --- .../main/scala/spark/examples/HBaseTest.scala | 34 +++++++++++++++++++ project/SparkBuild.scala | 6 +++- 2 files changed, 39 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/spark/examples/HBaseTest.scala diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala new file mode 100644 index 0000000000000..90ff64b483ef0 --- /dev/null +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -0,0 +1,34 @@ +package spark.examples + +import spark._ +import spark.rdd.NewHadoopRDD +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, HColumnDescriptor} +import org.apache.hadoop.hbase.client.HBaseAdmin +import org.apache.hadoop.hbase.mapreduce.TableInputFormat + +object HBaseTest { + def main(args: Array[String]) { + val sc = new SparkContext(args(0), "HBaseTest", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + + val conf = HBaseConfiguration.create() + conf.set(TableInputFormat.INPUT_TABLE, args(1)) + + // Initialize hBase tables if necessary + val admin = new HBaseAdmin(conf) + if(!admin.isTableAvailable(args(1))) { + val colDesc = new HColumnDescriptor(args(2)) + val tableDesc = new HTableDescriptor(args(1)) + tableDesc.addFamily(colDesc) + admin.createTable(tableDesc) + } + + val hBaseRDD = new NewHadoopRDD(sc, classOf[TableInputFormat], + classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], + classOf[org.apache.hadoop.hbase.client.Result], conf) + + hBaseRDD.count() + + System.exit(0) + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 190d72343587e..6f5607d31c71e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -200,7 +200,11 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") + resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), + libraryDependencies ++= Seq( + "com.twitter" % "algebird-core_2.9.2" % "0.1.11", + "org.apache.hbase" % "hbase" % "0.94.6" + ) ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From 9290f16430f92c66d4ec3b1ec76e491ae7cf26dc Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Sat, 4 May 2013 12:39:14 -0500 Subject: [PATCH 537/696] Remove unnecessary column family config --- examples/src/main/scala/spark/examples/HBaseTest.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index 90ff64b483ef0..37aedde302457 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -2,7 +2,7 @@ package spark.examples import spark._ import spark.rdd.NewHadoopRDD -import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, HColumnDescriptor} +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} import org.apache.hadoop.hbase.client.HBaseAdmin import org.apache.hadoop.hbase.mapreduce.TableInputFormat @@ -14,12 +14,10 @@ object HBaseTest { val conf = HBaseConfiguration.create() conf.set(TableInputFormat.INPUT_TABLE, args(1)) - // Initialize hBase tables if necessary + // Initialize hBase table if necessary val admin = new HBaseAdmin(conf) if(!admin.isTableAvailable(args(1))) { - val colDesc = new HColumnDescriptor(args(2)) val tableDesc = new HTableDescriptor(args(1)) - tableDesc.addFamily(colDesc) admin.createTable(tableDesc) } From 7cff7e789723b646d1692fc71ef99f89a862bdc6 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Sat, 4 May 2013 14:56:55 -0500 Subject: [PATCH 538/696] Fix indents and mention other configuration options --- examples/src/main/scala/spark/examples/HBaseTest.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index 37aedde302457..d94b25828d017 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -12,6 +12,9 @@ object HBaseTest { System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val conf = HBaseConfiguration.create() + + // Other options for configuring scan behavior are available. More information available at + // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html conf.set(TableInputFormat.INPUT_TABLE, args(1)) // Initialize hBase table if necessary @@ -22,8 +25,8 @@ object HBaseTest { } val hBaseRDD = new NewHadoopRDD(sc, classOf[TableInputFormat], - classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], - classOf[org.apache.hadoop.hbase.client.Result], conf) + classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], + classOf[org.apache.hadoop.hbase.client.Result], conf) hBaseRDD.count() From d48e9fde01cec2a7db794edf4cbe66c2228531aa Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 4 May 2013 12:36:47 -0700 Subject: [PATCH 539/696] Fix SPARK-629: weird number of cores in job details page. --- .../scala/spark/deploy/ApplicationDescription.scala | 2 +- core/src/main/scala/spark/deploy/JsonProtocol.scala | 4 ++-- .../scala/spark/deploy/master/ApplicationInfo.scala | 2 +- .../twirl/spark/deploy/master/app_details.scala.html | 11 ++++------- 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 6659e53b25f37..b6b9f9bf9d8ab 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -2,7 +2,7 @@ package spark.deploy private[spark] class ApplicationDescription( val name: String, - val cores: Int, + val maxCores: Int, /* Integer.MAX_VALUE denotes an unlimited number of cores */ val memoryPerSlave: Int, val command: Command, val sparkHome: String) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 71a641a9efea2..ea832101d2177 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -26,7 +26,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { "starttime" -> JsNumber(obj.startTime), "id" -> JsString(obj.id), "name" -> JsString(obj.desc.name), - "cores" -> JsNumber(obj.desc.cores), + "cores" -> JsNumber(obj.desc.maxCores), "user" -> JsString(obj.desc.user), "memoryperslave" -> JsNumber(obj.desc.memoryPerSlave), "submitdate" -> JsString(obj.submitDate.toString)) @@ -35,7 +35,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] { def write(obj: ApplicationDescription) = JsObject( "name" -> JsString(obj.name), - "cores" -> JsNumber(obj.cores), + "cores" -> JsNumber(obj.maxCores), "memoryperslave" -> JsNumber(obj.memoryPerSlave), "user" -> JsString(obj.user) ) diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3591a9407237a..70e5caab6653d 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -37,7 +37,7 @@ private[spark] class ApplicationInfo( coresGranted -= exec.cores } - def coresLeft: Int = desc.cores - coresGranted + def coresLeft: Int = desc.maxCores - coresGranted private var _retryCount = 0 diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 301a7e212495d..66147e213f79d 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -9,15 +9,12 @@
  • ID: @app.id
  • Description: @app.desc.name
  • User: @app.desc.user
  • -
  • Cores: - @app.desc.cores - (@app.coresGranted Granted - @if(app.desc.cores == Integer.MAX_VALUE) { - +
  • Cores: + @if(app.desc.maxCores == Integer.MAX_VALUE) { + Unlimited (@app.coresGranted granted) } else { - , @app.coresLeft + @app.desc.maxCores (@app.coresGranted granted, @app.coresLeft left) } - )
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • From c0688451a6a91f596d9c596383026ddbdcbb8bb0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 4 May 2013 12:37:04 -0700 Subject: [PATCH 540/696] Fix wrong closing tags in web UI HTML. --- core/src/main/twirl/spark/deploy/master/executor_row.scala.html | 2 +- core/src/main/twirl/spark/deploy/master/worker_row.scala.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html index d2d80fad48992..21e72c7aabf7c 100644 --- a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html @@ -3,7 +3,7 @@
    @executor.id - @executor.worker.id + @executor.worker.id @executor.cores @executor.memory
    - @worker.id + @worker.id @{worker.host}:@{worker.port} @worker.state
    +If no master URL is specified, the spark shell defaults to "local". + For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. ### Deploying Code on a Cluster From b8e46b6074e5ecc1ae4ed22ea32983597c14b683 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 16 May 2013 01:52:40 -0700 Subject: [PATCH 577/696] Abort job if result exceeds Akka frame size; add test. --- core/src/main/scala/spark/TaskEndReason.scala | 2 ++ core/src/main/scala/spark/executor/Executor.scala | 3 ++- .../spark/scheduler/cluster/TaskSetManager.scala | 6 ++++++ core/src/test/scala/spark/DistributedSuite.scala | 13 +++++++++++++ 4 files changed, 23 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index 420c54bc9a0d4..c5da4535627cf 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -20,3 +20,5 @@ case class FetchFailed(bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, re private[spark] case class ExceptionFailure(exception: Throwable) extends TaskEndReason private[spark] case class OtherFailure(message: String) extends TaskEndReason + +private[spark] case class TaskResultTooBigFailure() extends TaskEndReason \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 718f0ff5bcf8d..9ec4eb6e88c44 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -115,7 +115,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) if (serializedResult.limit >= (akkaFrameSize - 1024)) { - throw new SparkException("Result for " + taskId + " exceeded Akka frame size") + context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(TaskResultTooBigFailure())) + return } context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) logInfo("Finished task ID " + taskId) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 27e713e2c43dd..df7f0eafffef1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -492,6 +492,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe sched.taskSetFinished(this) return + case taskResultTooBig: TaskResultTooBigFailure => + logInfo("Loss was due to task %s result exceeding Akka frame size;" + + "aborting job".format(tid)) + abort("Task %s result exceeded Akka frame size".format(tid)) + return + case ef: ExceptionFailure => val key = ef.exception.toString val now = System.currentTimeMillis diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4df3bb5b6793b..9f58999cbe6af 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -277,6 +277,19 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } } + + test("job should fail if TaskResult exceeds Akka frame size") { + // We must use local-cluster mode since results are returned differently + // when running under LocalScheduler: + sc = new SparkContext("local-cluster[1,1,512]", "test") + val akkaFrameSize = + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)} + val exception = intercept[SparkException] { + rdd.reduce((x, y) => x) + } + exception.getMessage should endWith("result exceeded Akka frame size") + } } object DistributedSuite { From 87540a7b386837d177a6d356ad1f5ef2c1ad6ea5 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 16 May 2013 15:27:58 +0530 Subject: [PATCH 578/696] Fix running on yarn documentation --- docs/running-on-yarn.md | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c8cf8ffc35451..41c0b235dd9ad 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -11,14 +11,32 @@ Ex: mvn -Phadoop2-yarn clean install # Building spark core consolidated jar. -Currently, only sbt can buid a consolidated jar which contains the entire spark code - which is required for launching jars on yarn. -To do this via sbt - though (right now) is a manual process of enabling it in project/SparkBuild.scala. +We need a consolidated spark core jar (which bundles all the required dependencies) to run Spark jobs on a yarn cluster. +This can be built either through sbt or via maven. + +- Building spark assembled jar via sbt. +It is a manual process of enabling it in project/SparkBuild.scala. Please comment out the HADOOP_VERSION, HADOOP_MAJOR_VERSION and HADOOP_YARN variables before the line 'For Hadoop 2 YARN support' Next, uncomment the subsequent 3 variable declaration lines (for these three variables) which enable hadoop yarn support. -Currnetly, it is a TODO to add support for maven assembly. +Assembly of the jar Ex: +./sbt/sbt clean assembly + +The assembled jar would typically be something like : +./streaming/target/spark-streaming-.jar + + +- Building spark assembled jar via sbt. +Use the hadoop2-yarn profile and execute the package target. + +Something like this. Ex: +$ mvn -Phadoop2-yarn clean package -DskipTests=true + + +This will build the shaded (consolidated) jar. Typically something like : +./repl-bin/target/spark-repl-bin--shaded-hadoop2-yarn.jar # Preparations @@ -62,6 +80,6 @@ The above starts a YARN Client programs which periodically polls the Application # Important Notes - When your application instantiates a Spark context it must use a special "standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "standalone" as an argument to your program, as shown in the example above. -- YARN does not support requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. +- We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. - Currently, we have not yet integrated with hadoop security. If --user is present, the hadoop_user specified will be used to run the tasks on the cluster. If unspecified, current user will be used (which should be valid in cluster). Once hadoop security support is added, and if hadoop cluster is enabled with security, additional restrictions would apply via delegation tokens passed. From feddd2530ddfac7a01b03c9113b29945ec0e9a82 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 16 May 2013 17:49:14 +0530 Subject: [PATCH 579/696] Filter out nulls - prevent NPE --- core/src/main/scala/spark/SparkContext.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 2ae4ad8659ff6..15a75c7e932ce 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -537,6 +537,8 @@ class SparkContext( * filesystems), or an HTTP, HTTPS or FTP URI. */ def addJar(path: String) { + // weird - debug why this is happening. + if (null == path) return val uri = new URI(path) val key = uri.getScheme match { case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) From f16c781709f9e108d9fe8ac052fb55146ce8a14f Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 16 May 2013 17:50:22 +0530 Subject: [PATCH 580/696] Fix documentation to use yarn-standalone as master --- docs/running-on-yarn.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 41c0b235dd9ad..2e46ff0ed1e08 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -69,7 +69,7 @@ For example: SPARK_JAR=./core/target/spark-core-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \ --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar \ --class spark.examples.SparkPi \ - --args standalone \ + --args yarn-standalone \ --num-workers 3 \ --master-memory 4g \ --worker-memory 2g \ @@ -79,7 +79,7 @@ The above starts a YARN Client programs which periodically polls the Application # Important Notes -- When your application instantiates a Spark context it must use a special "standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "standalone" as an argument to your program, as shown in the example above. +- When your application instantiates a Spark context it must use a special "yarn-standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "yarn-standalone" as an argument to your program, as shown in the example above. - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. - Currently, we have not yet integrated with hadoop security. If --user is present, the hadoop_user specified will be used to run the tasks on the cluster. If unspecified, current user will be used (which should be valid in cluster). Once hadoop security support is added, and if hadoop cluster is enabled with security, additional restrictions would apply via delegation tokens passed. From f0881f8d4812dcee955aa303d7a4b76c58b75a61 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Fri, 17 May 2013 01:58:50 +0530 Subject: [PATCH 581/696] Hope this does not turn into a bike shed change --- core/src/main/scala/spark/SparkContext.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 15a75c7e932ce..736b5485b7d9d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -537,8 +537,10 @@ class SparkContext( * filesystems), or an HTTP, HTTPS or FTP URI. */ def addJar(path: String) { - // weird - debug why this is happening. - if (null == path) return + if (null == path) { + logInfo("null specified as parameter to addJar") + return + } val uri = new URI(path) val key = uri.getScheme match { case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) From c6e2770bfe940a4f4f26f75c9ba228faea7316f0 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 17 May 2013 05:10:38 +0800 Subject: [PATCH 582/696] Fix ClusterScheduler bug to avoid allocating tasks to same slave --- .../scheduler/cluster/ClusterScheduler.scala | 48 +++++++++++-------- .../scala/spark/scheduler/cluster/Pool.scala | 20 ++++---- .../spark/scheduler/cluster/Schedulable.scala | 3 +- .../scheduler/cluster/TaskSetManager.scala | 8 +++- .../scheduler/ClusterSchedulerSuite.scala | 46 +++++++++++------- 5 files changed, 75 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 1a300c9e8cbe7..4caafcc1d34d0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -164,27 +164,35 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray - for (i <- 0 until offers.size) { - var launchedTask = true - val execId = offers(i).executorId - val host = offers(i).hostname - while (availableCpus(i) > 0 && launchedTask) { + var launchedTask = false + val sortedLeafSchedulable = rootPool.getSortedLeafSchedulable() + for (schedulable <- sortedLeafSchedulable) + { + logDebug("parentName:%s,name:%s,runningTasks:%s".format(schedulable.parent.name,schedulable.name,schedulable.runningTasks)) + } + for (schedulable <- sortedLeafSchedulable) { + do { launchedTask = false - rootPool.receiveOffer(execId,host,availableCpus(i)) match { - case Some(task) => - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = task.taskSetId - taskSetTaskIds(task.taskSetId) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - - case None => {} - } - } + for (i <- 0 until offers.size) { + var launchedTask = true + val execId = offers(i).executorId + val host = offers(i).hostname + schedulable.slaveOffer(execId,host,availableCpus(i)) match { + case Some(task) => + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = task.taskSetId + taskSetTaskIds(task.taskSetId) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + + case None => {} + } + } + } while(launchedTask) } if (tasks.size > 0) { hasLaunchedTask = true diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index d5482f71add04..ae603e7dd96d6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -75,19 +75,17 @@ private[spark] class Pool( return shouldRevive } - override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + return None + } + + override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { + var leafSchedulableQueue = new ArrayBuffer[Schedulable] val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) - for (manager <- sortedSchedulableQueue) { - logInfo("parentName:%s,schedulableName:%s,minShares:%d,weight:%d,runningTasks:%d".format( - manager.parent.name, manager.name, manager.minShare, manager.weight, manager.runningTasks)) + for (schedulable <- sortedSchedulableQueue) { + leafSchedulableQueue ++= schedulable.getSortedLeafSchedulable() } - for (manager <- sortedSchedulableQueue) { - val task = manager.receiveOffer(execId, host, availableCpus) - if (task != None) { - return task - } - } - return None + return leafSchedulableQueue } override def increaseRunningTasks(taskNum: Int) { diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 54e8ae95f9f37..c620588e14972 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -21,6 +21,7 @@ private[spark] trait Schedulable { def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String): Unit - def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] + def slaveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] def checkSpeculatableTasks(): Boolean + def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index baaaa41a37f6b..80edbe77a13ad 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -198,7 +198,7 @@ private[spark] class TaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -398,6 +398,12 @@ private[spark] class TaskSetManager( //nothing } + override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { + var leafSchedulableQueue = new ArrayBuffer[Schedulable] + leafSchedulableQueue += this + return leafSchedulableQueue + } + override def executorLost(execId: String, hostname: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) val newHostsAlive = sched.hostsAlive diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 2eda48196be74..8426be7575a51 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -6,6 +6,7 @@ import org.scalatest.BeforeAndAfter import spark._ import spark.scheduler._ import spark.scheduler.cluster._ +import scala.collection.mutable.ArrayBuffer import java.util.Properties @@ -25,34 +26,34 @@ class DummyTaskSetManager( var numTasks = initNumTasks var tasksFinished = 0 - def increaseRunningTasks(taskNum: Int) { + override def increaseRunningTasks(taskNum: Int) { runningTasks += taskNum if (parent != null) { parent.increaseRunningTasks(taskNum) } } - def decreaseRunningTasks(taskNum: Int) { + override def decreaseRunningTasks(taskNum: Int) { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) } } - def addSchedulable(schedulable: Schedulable) { + override def addSchedulable(schedulable: Schedulable) { } - def removeSchedulable(schedulable: Schedulable) { + override def removeSchedulable(schedulable: Schedulable) { } - def getSchedulableByName(name: String): Schedulable = { + override def getSchedulableByName(name: String): Schedulable = { return null } - def executorLost(executorId: String, host: String): Unit = { + override def executorLost(executorId: String, host: String): Unit = { } - def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] = { + override def slaveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] = { if (tasksFinished + runningTasks < numTasks) { increaseRunningTasks(1) return Some(new TaskDescription(0, stageId.toString, execId, "task 0:0", null)) @@ -60,10 +61,16 @@ class DummyTaskSetManager( return None } - def checkSpeculatableTasks(): Boolean = { + override def checkSpeculatableTasks(): Boolean = { return true } + override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { + var leafSchedulableQueue = new ArrayBuffer[Schedulable] + leafSchedulableQueue += this + return leafSchedulableQueue + } + def taskFinished() { decreaseRunningTasks(1) tasksFinished +=1 @@ -80,16 +87,21 @@ class DummyTaskSetManager( class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { - def receiveOffer(rootPool: Pool) : Option[TaskDescription] = { - rootPool.receiveOffer("execId_1", "hostname_1", 1) + def resourceOffer(rootPool: Pool): Int = { + val taskSetQueue = rootPool.getSortedLeafSchedulable() + for (taskSet <- taskSetQueue) + { + taskSet.slaveOffer("execId_1", "hostname_1", 1) match { + case Some(task) => + return task.taskSetId.toInt + case None => {} + } + } + -1 } def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { - receiveOffer(rootPool) match { - case Some(task) => - assert(task.taskSetId.toInt === expectedTaskSetId) - case _ => - } + assert(resourceOffer(rootPool) === expectedTaskSetId) } test("FIFO Scheduler Test") { @@ -105,9 +117,9 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { schedulableBuilder.addTaskSetManager(taskSetManager2, null) checkTaskSetId(rootPool, 0) - receiveOffer(rootPool) + resourceOffer(rootPool) checkTaskSetId(rootPool, 1) - receiveOffer(rootPool) + resourceOffer(rootPool) taskSetManager1.abort() checkTaskSetId(rootPool, 2) } From f742435f18f65e1dbf6235dd49f93b10f22cfe4b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 14:31:03 -0700 Subject: [PATCH 583/696] Removed the duplicated netty dependency in SBT build file. --- project/SparkBuild.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 234b021c93d07..0ea23b446fd31 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -142,7 +142,6 @@ object SparkBuild extends Build { ), libraryDependencies ++= Seq( - "io.netty" % "netty" % "3.5.3.Final", "com.google.guava" % "guava" % "11.0.1", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, From 61cf17623835007114ee69394999faaba8a46206 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 14:31:26 -0700 Subject: [PATCH 584/696] Added dependency on netty-all in Maven. --- core/pom.xml | 4 ++++ pom.xml | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 57a95328c397b..d8687bf9911b3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -87,6 +87,10 @@ org.apache.mesos mesos + + io.netty + netty-all + log4j log4j diff --git a/pom.xml b/pom.xml index d7cdc591cf487..eda18fdd12d33 100644 --- a/pom.xml +++ b/pom.xml @@ -256,6 +256,11 @@ mesos ${mesos.version} + + io.netty + netty-all + 4.0.0.Beta2 + org.apache.derby derby From 43644a293f5faec088530cf3a84d3680f2a103af Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 14:31:38 -0700 Subject: [PATCH 585/696] Only check for repl classes if the user is running the repl. Otherwise, check for core classes in run. This fixed the problem that core tests depend on whether repl module is compiled or not. --- run | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/run b/run index c744bbd3dc86c..c0065c53f1709 100755 --- a/run +++ b/run @@ -102,12 +102,18 @@ STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" # Exit if the user hasn't compiled Spark -if [ ! -e "$REPL_DIR/target" ]; then - echo "Failed to find Spark classes in $REPL_DIR/target" >&2 +if [ ! -e "$CORE_DIR/target" ]; then + echo "Failed to find Spark classes in $CORE_DIR/target" >&2 echo "You need to compile Spark before running this program" >&2 exit 1 fi +if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then + echo "Failed to find Spark classes in $REPL_DIR/target" >&2 + echo "You need to compile Spark repl module before running this program" >&2 + exit 1 +fi + # Build up classpath CLASSPATH="$SPARK_CLASSPATH" CLASSPATH="$CLASSPATH:$FWDIR/conf" From 3b3300383a6ccb9d8b62243c7814eb6c2e1ab313 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 16:51:28 -0700 Subject: [PATCH 586/696] Updated Scala version in docs generation ruby script. --- docs/_plugins/copy_api_dirs.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index d77e53963c2f7..c10ae595de02f 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -18,7 +18,7 @@ # Copy over the scaladoc from each project into the docs directory. # This directory will be copied over to _site when `jekyll` command is run. projects.each do |project_name| - source = "../" + project_name + "/target/scala-2.9.2/api" + source = "../" + project_name + "/target/scala-2.9.3/api" dest = "api/" + project_name puts "echo making directory " + dest From dc146406aefa4285d2a2a5d9d45f2ef883e9ef73 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 17:07:14 -0700 Subject: [PATCH 587/696] Updated Scala version in docs generation ruby script. --- docs/_plugins/copy_api_dirs.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index d77e53963c2f7..c10ae595de02f 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -18,7 +18,7 @@ # Copy over the scaladoc from each project into the docs directory. # This directory will be copied over to _site when `jekyll` command is run. projects.each do |project_name| - source = "../" + project_name + "/target/scala-2.9.2/api" + source = "../" + project_name + "/target/scala-2.9.3/api" dest = "api/" + project_name puts "echo making directory " + dest From da2642bead2eaf15bfcc28520858cf212d5975a4 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Fri, 17 May 2013 06:58:46 +0530 Subject: [PATCH 588/696] Fix example jar name --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 2e46ff0ed1e08..3946100247bb6 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -25,7 +25,7 @@ Assembly of the jar Ex: ./sbt/sbt clean assembly The assembled jar would typically be something like : -./streaming/target/spark-streaming-.jar +./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar - Building spark assembled jar via sbt. From d19753b9c78857acae441dce3133fbb6c5855f95 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 18 May 2013 06:45:19 +0800 Subject: [PATCH 589/696] expose TaskSetManager type to resourceOffer function in ClusterScheduler --- .../scheduler/cluster/ClusterScheduler.scala | 14 +-- .../scala/spark/scheduler/cluster/Pool.scala | 12 +- .../spark/scheduler/cluster/Schedulable.scala | 3 +- .../scheduler/cluster/TaskDescription.scala | 1 - .../scheduler/cluster/TaskSetManager.scala | 12 +- .../scheduler/ClusterSchedulerSuite.scala | 112 +++++++++++------- 6 files changed, 84 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 4caafcc1d34d0..e6399a3547255 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -165,24 +165,24 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray var launchedTask = false - val sortedLeafSchedulable = rootPool.getSortedLeafSchedulable() - for (schedulable <- sortedLeafSchedulable) + val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() + for (manager <- sortedTaskSetQueue) { - logDebug("parentName:%s,name:%s,runningTasks:%s".format(schedulable.parent.name,schedulable.name,schedulable.runningTasks)) + logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } - for (schedulable <- sortedLeafSchedulable) { + for (manager <- sortedTaskSetQueue) { do { launchedTask = false for (i <- 0 until offers.size) { var launchedTask = true val execId = offers(i).executorId val host = offers(i).hostname - schedulable.slaveOffer(execId,host,availableCpus(i)) match { + manager.slaveOffer(execId,host,availableCpus(i)) match { case Some(task) => tasks(i) += task val tid = task.taskId - taskIdToTaskSetId(tid) = task.taskSetId - taskSetTaskIds(task.taskSetId) += tid + taskIdToTaskSetId(tid) = manager.taskSet.id + taskSetTaskIds(manager.taskSet.id) += tid taskIdToExecutorId(tid) = execId activeExecutorIds += execId executorsByHost(host) += execId diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index ae603e7dd96d6..4dc15f413c660 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -75,17 +75,13 @@ private[spark] class Pool( return shouldRevive } - override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { - return None - } - - override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { - var leafSchedulableQueue = new ArrayBuffer[Schedulable] + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { - leafSchedulableQueue ++= schedulable.getSortedLeafSchedulable() + sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue() } - return leafSchedulableQueue + return sortedTaskSetQueue } override def increaseRunningTasks(taskNum: Int) { diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index c620588e14972..6bb7525b49735 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -21,7 +21,6 @@ private[spark] trait Schedulable { def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String): Unit - def slaveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] def checkSpeculatableTasks(): Boolean - def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] + def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index cdd004c94bae0..b41e951be99f8 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -5,7 +5,6 @@ import spark.util.SerializableBuffer private[spark] class TaskDescription( val taskId: Long, - val taskSetId: String, val executorId: String, val name: String, _serializedTask: ByteBuffer) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 80edbe77a13ad..b9d2dbf487454 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -198,7 +198,7 @@ private[spark] class TaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -234,7 +234,7 @@ private[spark] class TaskSetManager( logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId, taskSet.id, execId, taskName, serializedTask)) + return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) } case _ => } @@ -398,10 +398,10 @@ private[spark] class TaskSetManager( //nothing } - override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { - var leafSchedulableQueue = new ArrayBuffer[Schedulable] - leafSchedulableQueue += this - return leafSchedulableQueue + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue } override def executorLost(execId: String, hostname: String) { diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8426be7575a51..956cc7421c5aa 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -13,18 +13,20 @@ import java.util.Properties class DummyTaskSetManager( initPriority: Int, initStageId: Int, - initNumTasks: Int) - extends Schedulable { - - var parent: Schedulable = null - var weight = 1 - var minShare = 2 - var runningTasks = 0 - var priority = initPriority - var stageId = initStageId - var name = "TaskSet_"+stageId - var numTasks = initNumTasks - var tasksFinished = 0 + initNumTasks: Int, + clusterScheduler: ClusterScheduler, + taskSet: TaskSet) + extends TaskSetManager(clusterScheduler,taskSet) { + + parent = null + weight = 1 + minShare = 2 + runningTasks = 0 + priority = initPriority + stageId = initStageId + name = "TaskSet_"+stageId + override val numTasks = initNumTasks + tasksFinished = 0 override def increaseRunningTasks(taskNum: Int) { runningTasks += taskNum @@ -41,11 +43,11 @@ class DummyTaskSetManager( } override def addSchedulable(schedulable: Schedulable) { - } - + } + override def removeSchedulable(schedulable: Schedulable) { } - + override def getSchedulableByName(name: String): Schedulable = { return null } @@ -65,11 +67,11 @@ class DummyTaskSetManager( return true } - override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { - var leafSchedulableQueue = new ArrayBuffer[Schedulable] - leafSchedulableQueue += this - return leafSchedulableQueue - } +// override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { +// var leafSchedulableQueue = new ArrayBuffer[Schedulable] +// leafSchedulableQueue += this +// return leafSchedulableQueue +// } def taskFinished() { decreaseRunningTasks(1) @@ -85,10 +87,28 @@ class DummyTaskSetManager( } } +class DummyTask(stageId: Int) extends Task[Int](stageId) +{ + def run(attemptId: Long): Int = { + return 0 + } +} + class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { - + + val sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + tasks += task + + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int): DummyTaskSetManager = { + new DummyTaskSetManager(priority, stage, numTasks, clusterScheduler, taskSet) + } + def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedLeafSchedulable() + val taskSetQueue = rootPool.getSortedTaskSetQueue() for (taskSet <- taskSetQueue) { taskSet.slaveOffer("execId_1", "hostname_1", 1) match { @@ -109,13 +129,13 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - val taskSetManager0 = new DummyTaskSetManager(0, 0, 2) - val taskSetManager1 = new DummyTaskSetManager(0, 1, 2) - val taskSetManager2 = new DummyTaskSetManager(0, 2, 2) + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2) schedulableBuilder.addTaskSetManager(taskSetManager0, null) schedulableBuilder.addTaskSetManager(taskSetManager1, null) schedulableBuilder.addTaskSetManager(taskSetManager2, null) - + checkTaskSetId(rootPool, 0) resourceOffer(rootPool) checkTaskSetId(rootPool, 1) @@ -130,7 +150,7 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) val schedulableBuilder = new FairSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - + assert(rootPool.getSchedulableByName("default") != null) assert(rootPool.getSchedulableByName("1") != null) assert(rootPool.getSchedulableByName("2") != null) @@ -146,16 +166,16 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { properties1.setProperty("spark.scheduler.cluster.fair.pool","1") val properties2 = new Properties() properties2.setProperty("spark.scheduler.cluster.fair.pool","2") - - val taskSetManager10 = new DummyTaskSetManager(1, 0, 1) - val taskSetManager11 = new DummyTaskSetManager(1, 1, 1) - val taskSetManager12 = new DummyTaskSetManager(1, 2, 2) + + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2) schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - - val taskSetManager23 = new DummyTaskSetManager(2, 3, 2) - val taskSetManager24 = new DummyTaskSetManager(2, 4, 2) + + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2) schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) @@ -190,27 +210,27 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) pool1.addSchedulable(pool10) pool1.addSchedulable(pool11) - - val taskSetManager000 = new DummyTaskSetManager(0, 0, 5) - val taskSetManager001 = new DummyTaskSetManager(0, 1, 5) + + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5) pool00.addSchedulable(taskSetManager000) pool00.addSchedulable(taskSetManager001) - - val taskSetManager010 = new DummyTaskSetManager(1, 2, 5) - val taskSetManager011 = new DummyTaskSetManager(1, 3, 5) + + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5) pool01.addSchedulable(taskSetManager010) pool01.addSchedulable(taskSetManager011) - - val taskSetManager100 = new DummyTaskSetManager(2, 4, 5) - val taskSetManager101 = new DummyTaskSetManager(2, 5, 5) + + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5) pool10.addSchedulable(taskSetManager100) pool10.addSchedulable(taskSetManager101) - val taskSetManager110 = new DummyTaskSetManager(3, 6, 5) - val taskSetManager111 = new DummyTaskSetManager(3, 7, 5) + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5) pool11.addSchedulable(taskSetManager110) pool11.addSchedulable(taskSetManager111) - + checkTaskSetId(rootPool, 0) checkTaskSetId(rootPool, 4) checkTaskSetId(rootPool, 6) From 0eab7a78b90e2593075c479282f631a5a20e77a9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 17 May 2013 18:05:46 -0700 Subject: [PATCH 590/696] Fixed a couple typos and formating problems in the YARN documentation. --- docs/running-on-yarn.md | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 3946100247bb6..66fb8d73e80f3 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -14,29 +14,31 @@ Ex: mvn -Phadoop2-yarn clean install We need a consolidated spark core jar (which bundles all the required dependencies) to run Spark jobs on a yarn cluster. This can be built either through sbt or via maven. -- Building spark assembled jar via sbt. -It is a manual process of enabling it in project/SparkBuild.scala. +- Building spark assembled jar via sbt. + It is a manual process of enabling it in project/SparkBuild.scala. Please comment out the HADOOP_VERSION, HADOOP_MAJOR_VERSION and HADOOP_YARN variables before the line 'For Hadoop 2 YARN support' Next, uncomment the subsequent 3 variable declaration lines (for these three variables) which enable hadoop yarn support. -Assembly of the jar Ex: -./sbt/sbt clean assembly +Assembly of the jar Ex: + + ./sbt/sbt clean assembly The assembled jar would typically be something like : -./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar +`./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar` -- Building spark assembled jar via sbt. -Use the hadoop2-yarn profile and execute the package target. +- Building spark assembled jar via Maven. + Use the hadoop2-yarn profile and execute the package target. Something like this. Ex: -$ mvn -Phadoop2-yarn clean package -DskipTests=true + + mvn -Phadoop2-yarn clean package -DskipTests=true This will build the shaded (consolidated) jar. Typically something like : -./repl-bin/target/spark-repl-bin--shaded-hadoop2-yarn.jar +`./repl-bin/target/spark-repl-bin--shaded-hadoop2-yarn.jar` # Preparations From 8d78c5f89f25d013c997c03587193f3d87a268b0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 17 May 2013 18:51:35 -0700 Subject: [PATCH 591/696] Changed the logging level from info to warning when addJar(null) is called. --- core/src/main/scala/spark/SparkContext.scala | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 736b5485b7d9d..69b4c5d20d3eb 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -538,16 +538,17 @@ class SparkContext( */ def addJar(path: String) { if (null == path) { - logInfo("null specified as parameter to addJar") - return - } - val uri = new URI(path) - val key = uri.getScheme match { - case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) - case _ => path + logWarning("null specified as parameter to addJar", + new SparkException("null specified as parameter to addJar")) + } else { + val uri = new URI(path) + val key = uri.getScheme match { + case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) + case _ => path + } + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } /** From e7982c798efccd523165d0e347c7912ba14fcdd7 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sat, 18 May 2013 16:11:29 -0700 Subject: [PATCH 592/696] Exclude old versions of Netty from Maven-based build --- pom.xml | 6 ++++++ streaming/pom.xml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/pom.xml b/pom.xml index eda18fdd12d33..6ee64d07c26ec 100644 --- a/pom.xml +++ b/pom.xml @@ -565,6 +565,12 @@ org.apache.avro avro-ipc 1.7.1.cloudera.2 + + + org.jboss.netty + netty + + diff --git a/streaming/pom.xml b/streaming/pom.xml index 08ff3e2ae12f4..4dc9a19d51fc1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -41,6 +41,12 @@ org.apache.flume flume-ng-sdk 1.2.0 + + + org.jboss.netty + netty + + com.github.sgroschupf From ecd6d75c6a88232c40070baed3dd67bdf77f0c69 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 21 May 2013 06:49:23 +0800 Subject: [PATCH 593/696] fix bug of unit tests --- .../scheduler/cluster/ClusterScheduler.scala | 6 +- .../scheduler/ClusterSchedulerSuite.scala | 72 ++++++++++--------- 2 files changed, 43 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 9547f4f6dd188..053d4b8e4ab5f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -352,7 +352,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHostPort(hostPort) += execId availableCpus(i) -= 1 launchedTask = true - + case None => {} } } @@ -373,7 +373,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } while (launchedTask) } - + if (tasks.size > 0) { hasLaunchedTask = true } @@ -522,7 +522,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) hostPortsAlive -= hostPort hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(hostPort)._1, new HashSet[String]).remove(hostPort) } - + val execs = executorsByHostPort.getOrElse(hostPort, new HashSet) execs -= executorId if (execs.isEmpty) { diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 7af749fb5cc34..a39418b71618f 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -67,12 +67,6 @@ class DummyTaskSetManager( return true } -// override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { -// var leafSchedulableQueue = new ArrayBuffer[Schedulable] -// leafSchedulableQueue += this -// return leafSchedulableQueue -// } - def taskFinished() { decreaseRunningTasks(1) tasksFinished +=1 @@ -94,17 +88,10 @@ class DummyTask(stageId: Int) extends Task[Int](stageId) } } -class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { - - val sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new DummyTask(0) - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - tasks += task +class ClusterSchedulerSuite extends FunSuite with LocalSparkContext { - def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int): DummyTaskSetManager = { - new DummyTaskSetManager(priority, stage, numTasks, clusterScheduler, taskSet) + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): DummyTaskSetManager = { + new DummyTaskSetManager(priority, stage, numTasks, cs , taskSet) } def resourceOffer(rootPool: Pool): Int = { @@ -125,13 +112,20 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { } test("FIFO Scheduler Test") { + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2) + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, clusterScheduler, taskSet) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, clusterScheduler, taskSet) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager0, null) schedulableBuilder.addTaskSetManager(taskSetManager1, null) schedulableBuilder.addTaskSetManager(taskSetManager2, null) @@ -145,6 +139,13 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { } test("Fair Scheduler Test") { + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() System.setProperty("spark.fairscheduler.allocation.file", xmlPath) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) @@ -167,15 +168,15 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val properties2 = new Properties() properties2.setProperty("spark.scheduler.cluster.fair.pool","2") - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2) + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2) + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, clusterScheduler, taskSet) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) @@ -195,6 +196,13 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { } test("Nested Pool Test") { + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) @@ -211,23 +219,23 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { pool1.addSchedulable(pool10) pool1.addSchedulable(pool11) - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5) + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, clusterScheduler, taskSet) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, clusterScheduler, taskSet) pool00.addSchedulable(taskSetManager000) pool00.addSchedulable(taskSetManager001) - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5) + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, clusterScheduler, taskSet) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, clusterScheduler, taskSet) pool01.addSchedulable(taskSetManager010) pool01.addSchedulable(taskSetManager011) - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5) + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, clusterScheduler, taskSet) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, clusterScheduler, taskSet) pool10.addSchedulable(taskSetManager100) pool10.addSchedulable(taskSetManager101) - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5) + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, clusterScheduler, taskSet) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, clusterScheduler, taskSet) pool11.addSchedulable(taskSetManager110) pool11.addSchedulable(taskSetManager111) From 3217d486f7fdd590250f2efee567e4779e130d34 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Mon, 20 May 2013 19:41:38 -0500 Subject: [PATCH 594/696] Add hBase dependency to examples POM --- examples/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/examples/pom.xml b/examples/pom.xml index c42d2bcdb9ed5..0fbb5a3d5d1a1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -67,6 +67,11 @@ hadoop-core provided + + org.apache.hbase + hbase + 0.94.6 + @@ -105,6 +110,11 @@ hadoop-client provided + + org.apache.hbase + hbase + 0.94.6 + From 786c97b87c9d3074796e2d931635d9c6f72b9704 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 27 Feb 2013 10:06:05 -0800 Subject: [PATCH 595/696] DistributedSuite: remove dead test code --- core/src/test/scala/spark/DistributedSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 068bb6ca4f420..0866fb47b3950 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -222,7 +222,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter sc = new SparkContext(clusterUrl, "test") val data = sc.parallelize(Seq(true, true), 2) assert(data.count === 2) // force executors to start - val masterId = SparkEnv.get.blockManager.blockManagerId assert(data.map(markNodeIfIdentity).collect.size === 2) assert(data.map(failOnMarkedIdentity).collect.size === 2) } From f350f14084dd04a2ea77e92e35b3cf415ef72202 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 5 Feb 2013 17:08:55 -0800 Subject: [PATCH 596/696] Use ARRAY_SAMPLE_SIZE constant instead of 100.0 --- core/src/main/scala/spark/SizeEstimator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala index d4e115725031d..f8a4c4e4891f5 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -198,7 +198,7 @@ private[spark] object SizeEstimator extends Logging { val elem = JArray.get(array, index) size += SizeEstimator.estimate(elem, state.visited) } - state.size += ((length / 100.0) * size).toLong + state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong } } } From bd3ea8f2a66de5ddc12dc1b2273e675d0abb8393 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 24 May 2013 14:26:19 +0800 Subject: [PATCH 597/696] fix CheckpointRDD getPreferredLocations java.io.FileNotFoundException --- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 43ee39c993a3c..377b1bdbe002f 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -43,7 +43,7 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri checkpointData.get.cpFile = Some(checkpointPath) override def getPreferredLocations(split: Partition): Seq[String] = { - val status = fs.getFileStatus(new Path(checkpointPath)) + val status = fs.getFileStatus(new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index))) val locations = fs.getFileBlockLocations(status, 0, status.getLen) locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost") } From cda2b150412314c47c2c24883111bfc441c3a3a2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 May 2013 13:05:06 -0700 Subject: [PATCH 598/696] Use ec2-metadata in start-slave.sh. PR #419 applied the same change, but only to start-master.sh, so some workers were still starting their web UI's using internal addresses. This should finally fix SPARK-613. --- bin/start-slave.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 616c76e4ee6e0..26b5b9d462037 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -6,7 +6,8 @@ bin=`cd "$bin"; pwd` # Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI if [ "$SPARK_PUBLIC_DNS" = "" ]; then # If we appear to be running on EC2, use the public address by default: - if [[ `hostname` == *ec2.internal ]]; then + # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname + if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname` fi fi From 6ea085169d8ba2d09ca9236273d65238b8411f04 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 May 2013 14:08:37 -0700 Subject: [PATCH 599/696] Fixed the bug that shuffle serializer is ignored by the new shuffle block iterators for local blocks. Also added a unit test for that. --- .../spark/storage/BlockFetcherIterator.scala | 2 +- core/src/test/scala/spark/ShuffleSuite.scala | 21 ++++++++++++++++--- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 43f835237ca32..88eed0d8c8dfd 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -163,7 +163,7 @@ object BlockFetcherIterator { // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight for (id <- localBlockIds) { - getLocal(id) match { + getLocalFromDisk(id, serializer) match { case Some(iter) => { // Pass 0 as size since it's not in flight results.put(new FetchResult(id, 0, () => iter)) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 2b2a90defa4e9..fdee7ca384605 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -99,7 +99,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val sums = pairs.reduceByKey(_+_, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } - + test("reduceByKey with partitioner") { sc = new SparkContext("local", "test") val p = new Partitioner() { @@ -272,7 +272,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } // partitionBy so we have a narrow dependency val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency + // more partitions/no partitioner so a shuffle dependency val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val c = a.subtract(b) assert(c.collect().toSet === Set((1, "a"), (3, "c"))) @@ -298,18 +298,33 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } // partitionBy so we have a narrow dependency val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency + // more partitions/no partitioner so a shuffle dependency val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val c = a.subtractByKey(b) assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.partitioner.get === p) } + test("shuffle serializer") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[1,2,512]", "test") + val a = sc.parallelize(1 to 10, 2) + val b = a.map { x => + (x, new ShuffleSuite.NonJavaSerializableClass(x * 2)) + } + // If the Kryo serializer is not used correctly, the shuffle would fail because the + // default Java serializer cannot handle the non serializable class. + val c = new ShuffledRDD(b, new HashPartitioner(3), classOf[spark.KryoSerializer].getName) + assert(c.count === 10) + } } object ShuffleSuite { + def mergeCombineException(x: Int, y: Int): Int = { throw new SparkException("Exception for map-side combine.") x + y } + + class NonJavaSerializableClass(val value: Int) } From 26962c9340ac92b11d43e87200e699471d0b6330 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 May 2013 16:39:33 -0700 Subject: [PATCH 600/696] Automatically configure Netty port. This makes unit tests using local-cluster pass. Previously they were failing because Netty was trying to bind to the same port for all processes. Pair programmed with @shivaram. --- .../java/spark/network/netty/FileServer.java | 68 ++++++++++++++----- .../spark/network/netty/ShuffleSender.scala | 23 +++---- .../spark/storage/BlockFetcherIterator.scala | 3 +- .../scala/spark/storage/BlockManager.scala | 13 ++-- .../scala/spark/storage/BlockManagerId.scala | 32 ++++++--- .../main/scala/spark/storage/DiskStore.scala | 52 ++++---------- .../scala/spark/MapOutputTrackerSuite.scala | 28 ++++---- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 14 ++-- .../spark/storage/BlockManagerSuite.scala | 6 +- 10 files changed, 129 insertions(+), 112 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index 647b26bf8a155..dd3f12561cb65 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -1,51 +1,83 @@ package spark.network.netty; +import java.net.InetSocketAddress; + import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.ChannelOption; import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioServerSocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Server that accept the path of a file an echo back its content. */ class FileServer { + private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private ServerBootstrap bootstrap = null; - private Channel channel = null; - private PathResolver pResolver; + private ChannelFuture channelFuture = null; + private int port = 0; + private Thread blockingThread = null; - public FileServer(PathResolver pResolver) { - this.pResolver = pResolver; - } + public FileServer(PathResolver pResolver, int port) { + InetSocketAddress addr = new InetSocketAddress(port); - public void run(int port) { // Configure the server. bootstrap = new ServerBootstrap(); - try { - bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) + bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) .channel(OioServerSocketChannel.class) .option(ChannelOption.SO_BACKLOG, 100) .option(ChannelOption.SO_RCVBUF, 1500) .childHandler(new FileServerChannelInitializer(pResolver)); - // Start the server. - channel = bootstrap.bind(port).sync().channel(); - channel.closeFuture().sync(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally{ + // Start the server. + channelFuture = bootstrap.bind(addr); + this.port = addr.getPort(); + } + + /** + * Start the file server asynchronously in a new thread. + */ + public void start() { + try { + blockingThread = new Thread() { + public void run() { + try { + Channel channel = channelFuture.sync().channel(); + channel.closeFuture().sync(); + } catch (InterruptedException e) { + LOG.error("File server start got interrupted", e); + } + } + }; + blockingThread.setDaemon(true); + blockingThread.start(); + } finally { bootstrap.shutdown(); } } + public int getPort() { + return port; + } + public void stop() { - if (channel!=null) { - channel.close(); + if (blockingThread != null) { + blockingThread.stop(); + blockingThread = null; + } + if (channelFuture != null) { + channelFuture.channel().closeFuture(); + channelFuture = null; } if (bootstrap != null) { bootstrap.shutdown(); + bootstrap = null; } } } diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala index dc87fefc56794..d6fa4b1e80744 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -5,23 +5,22 @@ import java.io.File import spark.Logging -private[spark] class ShuffleSender(val port: Int, val pResolver: PathResolver) extends Logging { - val server = new FileServer(pResolver) +private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { - Runtime.getRuntime().addShutdownHook( - new Thread() { - override def run() { - server.stop() - } - } - ) + val server = new FileServer(pResolver, portIn) + server.start() - def start() { - server.run(port) + def stop() { + server.stop() } + + def port: Int = server.getPort() } +/** + * An application for testing the shuffle sender as a standalone program. + */ private[spark] object ShuffleSender { def main(args: Array[String]) { @@ -50,7 +49,5 @@ private[spark] object ShuffleSender { } } val sender = new ShuffleSender(port, pResovler) - - sender.start() } } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 88eed0d8c8dfd..95308c728273c 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -272,8 +272,7 @@ object BlockFetcherIterator { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host)) - val cmId = new ConnectionManagerId( - req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) val cpier = new ShuffleCopier cpier.getBlocks(cmId, req.blocks, putResult) logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 40d608628e9c2..d35c43f194bcd 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -94,11 +94,16 @@ private[spark] class BlockManager( private[storage] val diskStore: DiskStore = new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) + // If we use Netty for shuffle, start a new Netty-based shuffle sender service. + private val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean + private val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt + private val nettyPort = if (useNetty) diskStore.startShuffleBlockSender(nettyPortConfig) else 0 + val connectionManager = new ConnectionManager(0) implicit val futureExecContext = connectionManager.futureExecContext val blockManagerId = BlockManagerId( - executorId, connectionManager.id.host, connectionManager.id.port) + executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) @@ -266,7 +271,6 @@ private[spark] class BlockManager( } } - /** * Get locations of an array of blocks. */ @@ -274,7 +278,7 @@ private[spark] class BlockManager( val startTimeMs = System.currentTimeMillis val locations = master.getLocations(blockIds).toArray logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) - return locations + locations } /** @@ -971,8 +975,7 @@ private[spark] object BlockManager extends Logging { assert (env != null || blockManagerMaster != null) val locationBlockIds: Seq[Seq[BlockManagerId]] = if (env != null) { - val blockManager = env.blockManager - blockManager.getLocationBlockIds(blockIds) + env.blockManager.getLocationBlockIds(blockIds) } else { blockManagerMaster.getLocations(blockIds) } diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index f4a2181490f9f..1e557d6148465 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -7,18 +7,19 @@ import spark.Utils /** * This class represent an unique identifier for a BlockManager. * The first 2 constructors of this class is made private to ensure that - * BlockManagerId objects can be created only using the factory method in - * [[spark.storage.BlockManager$]]. This allows de-duplication of ID objects. + * BlockManagerId objects can be created only using the apply method in + * the companion object. This allows de-duplication of ID objects. * Also, constructor parameters are private to ensure that parameters cannot * be modified from outside this class. */ private[spark] class BlockManagerId private ( private var executorId_ : String, private var host_ : String, - private var port_ : Int + private var port_ : Int, + private var nettyPort_ : Int ) extends Externalizable { - private def this() = this(null, null, 0) // For deserialization only + private def this() = this(null, null, 0, 0) // For deserialization only def executorId: String = executorId_ @@ -39,28 +40,32 @@ private[spark] class BlockManagerId private ( def port: Int = port_ + def nettyPort: Int = nettyPort_ + override def writeExternal(out: ObjectOutput) { out.writeUTF(executorId_) out.writeUTF(host_) out.writeInt(port_) + out.writeInt(nettyPort_) } override def readExternal(in: ObjectInput) { executorId_ = in.readUTF() host_ = in.readUTF() port_ = in.readInt() + nettyPort_ = in.readInt() } @throws(classOf[IOException]) private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) - override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, host, port) + override def toString = "BlockManagerId(%s, %s, %d, %d)".format(executorId, host, port, nettyPort) - override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port + override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port + nettyPort override def equals(that: Any) = that match { case id: BlockManagerId => - executorId == id.executorId && port == id.port && host == id.host + executorId == id.executorId && port == id.port && host == id.host && nettyPort == id.nettyPort case _ => false } @@ -69,8 +74,17 @@ private[spark] class BlockManagerId private ( private[spark] object BlockManagerId { - def apply(execId: String, host: String, port: Int) = - getCachedBlockManagerId(new BlockManagerId(execId, host, port)) + /** + * Returns a [[spark.storage.BlockManagerId]] for the given configuraiton. + * + * @param execId ID of the executor. + * @param host Host name of the block manager. + * @param port Port of the block manager. + * @param nettyPort Optional port for the Netty-based shuffle sender. + * @return A new [[spark.storage.BlockManagerId]]. + */ + def apply(execId: String, host: String, port: Int, nettyPort: Int) = + getCachedBlockManagerId(new BlockManagerId(execId, host, port, nettyPort)) def apply(in: ObjectInput) = { val obj = new BlockManagerId() diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 933eeaa2162c6..57d4dafefc56a 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -82,22 +82,15 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val MAX_DIR_CREATION_ATTEMPTS: Int = 10 val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt - var shuffleSender : Thread = null - val thisInstance = this + var shuffleSender : ShuffleSender = null // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid // having really large inodes at the top level. val localDirs = createLocalDirs() val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) - val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean - addShutdownHook() - if(useNetty){ - startShuffleBlockSender() - } - def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { new DiskBlockObjectWriter(blockId, serializer, bufferSize) @@ -274,8 +267,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) localDirs.foreach { localDir => if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) } - if (useNetty && shuffleSender != null) + if (shuffleSender != null) { shuffleSender.stop + } } catch { case t: Throwable => logError("Exception while deleting local spark dirs", t) } @@ -283,39 +277,17 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) }) } - private def startShuffleBlockSender() { - try { - val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt - - val pResolver = new PathResolver { - override def getAbsolutePath(blockId: String): String = { - if (!blockId.startsWith("shuffle_")) { - return null - } - thisInstance.getFile(blockId).getAbsolutePath() - } - } - shuffleSender = new Thread { - override def run() = { - val sender = new ShuffleSender(port, pResolver) - logInfo("Created ShuffleSender binding to port : "+ port) - sender.start - } - } - shuffleSender.setDaemon(true) - shuffleSender.start - - } catch { - case interrupted: InterruptedException => - logInfo("Runner thread for ShuffleBlockSender interrupted") - - case e: Exception => { - logError("Error running ShuffleBlockSender ", e) - if (shuffleSender != null) { - shuffleSender.stop - shuffleSender = null + private[storage] def startShuffleBlockSender(port: Int): Int = { + val pResolver = new PathResolver { + override def getAbsolutePath(blockId: String): String = { + if (!blockId.startsWith("shuffle_")) { + return null } + DiskStore.this.getFile(blockId).getAbsolutePath() } } + shuffleSender = new ShuffleSender(port, pResolver) + logInfo("Created ShuffleSender binding to port : "+ shuffleSender.port) + shuffleSender.port } } diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index b5cedc0b688c2..6e585e1c3a540 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -8,7 +8,7 @@ import spark.storage.BlockManagerId import spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { - + test("compressSize") { assert(MapOutputTracker.compressSize(0L) === 0) assert(MapOutputTracker.compressSize(1L) === 1) @@ -45,13 +45,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0), Array(compressedSize10000, compressedSize1000))) val statuses = tracker.getServerStatuses(10, 0) - assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), - (BlockManagerId("b", "hostB", 1000), size10000))) + assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000, 0), size1000), + (BlockManagerId("b", "hostB", 1000, 0), size10000))) tracker.stop() } @@ -64,14 +64,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000, compressedSize1000, compressedSize1000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0), Array(compressedSize10000, compressedSize1000, compressedSize1000))) // As if we had two simulatenous fetch failures - tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) - tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) // The remaining reduce task might try to grab the output despite the shuffle failure; // this should cause it to fail, and the scheduler will ignore the failure due to the @@ -88,12 +88,12 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTracker() masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") - + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() slaveTracker.trackerActor = slaveSystem.actorFor( "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") - + masterTracker.registerShuffle(10, 1) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) @@ -102,13 +102,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize1000 = MapOutputTracker.compressSize(1000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000))) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) - masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index fdee7ca384605..58c834c735d0a 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -326,5 +326,5 @@ object ShuffleSuite { x + y } - class NonJavaSerializableClass(val value: Int) + class NonJavaSerializableClass(val value: Int) extends Serializable } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 16554eac6e684..30e6fef9505c7 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -44,7 +44,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.generation = mapOutputTracker.getGeneration) - taskSets += taskSet + taskSets += taskSet } override def setListener(listener: TaskSchedulerListener) = {} override def defaultParallelism() = 2 @@ -164,7 +164,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } } } - + /** Sends the rdd to the scheduler for scheduling. */ private def submit( rdd: RDD[_], @@ -174,7 +174,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont listener: JobListener = listener) { runEvent(JobSubmitted(rdd, func, partitions, allowLocal, null, listener)) } - + /** Sends TaskSetFailed to the scheduler. */ private def failed(taskSet: TaskSet, message: String) { runEvent(TaskSetFailed(taskSet, message)) @@ -209,11 +209,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener)) assert(results === Map(0 -> 42)) } - + test("run trivial job w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) - submit(finalRdd, Array(0)) + submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -250,7 +250,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) } - + test("run trivial shuffle with fetch failure") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) @@ -398,6 +398,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) private def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) + BlockManagerId("exec-" + host, host, 12345, 0) } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 71d1f0bcc8395..bff2475686a47 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -99,9 +99,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("BlockManagerId object caching") { - val id1 = BlockManagerId("e1", "XXX", 1) - val id2 = BlockManagerId("e1", "XXX", 1) // this should return the same object as id1 - val id3 = BlockManagerId("e1", "XXX", 2) // this should return a different object + val id1 = BlockManagerId("e1", "XXX", 1, 0) + val id2 = BlockManagerId("e1", "XXX", 1, 0) // this should return the same object as id1 + val id3 = BlockManagerId("e1", "XXX", 2, 0) // this should return a different object assert(id2 === id1, "id2 is not same as id1") assert(id2.eq(id1), "id2 is not the same object as id1") assert(id3 != id1, "id3 is same as id1") From 6bbbe012877115eab084fea09baf677abaf52f2b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 May 2013 16:51:45 -0700 Subject: [PATCH 601/696] Fixed a stupid mistake that NonJavaSerializableClass was made Java serializable. --- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 58c834c735d0a..fdee7ca384605 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -326,5 +326,5 @@ object ShuffleSuite { x + y } - class NonJavaSerializableClass(val value: Int) extends Serializable + class NonJavaSerializableClass(val value: Int) } From a674d67c0aebb940e3b816e2307206115baec175 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 28 May 2013 16:24:05 -0500 Subject: [PATCH 602/696] Fix start-slave not passing instance number to spark-daemon. --- bin/start-slave.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 26b5b9d462037..dfcbc6981bae1 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -12,4 +12,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@" +"$bin"/spark-daemon.sh start spark.deploy.worker.Worker 1 "$@" From 4fe1fbdd51f781157138ffd35da5834366379688 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 28 May 2013 16:26:32 -0500 Subject: [PATCH 603/696] Remove unused addIfNoPort. --- core/src/main/scala/spark/Utils.scala | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c1495d531714a..84626df553a38 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -403,17 +403,6 @@ private object Utils extends Logging { hostPortParseResults.get(hostPort) } - def addIfNoPort(hostPort: String, port: Int): String = { - if (port <= 0) throw new IllegalArgumentException("Invalid port specified " + port) - - // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now. - // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 - val indx: Int = hostPort.lastIndexOf(':') - if (-1 != indx) return hostPort - - hostPort + ":" + port - } - private[spark] val daemonThreadFactory: ThreadFactory = new ThreadFactoryBuilder().setDaemon(true).build() From fbc1ab346867d5c81dc59e4c8d85aeda2f516ce2 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 28 May 2013 16:27:16 -0700 Subject: [PATCH 604/696] Couple of Netty fixes a. Fix the port number by reading it from the bound channel b. Fix the shutdown sequence to make sure we actually block on the channel c. Fix the unit test to use two JVMs. --- .../java/spark/network/netty/FileServer.java | 45 ++++++++++--------- core/src/test/scala/spark/ShuffleSuite.scala | 14 +++++- 2 files changed, 37 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index dd3f12561cb65..dd3a557ae5952 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -37,29 +37,33 @@ public FileServer(PathResolver pResolver, int port) { .childHandler(new FileServerChannelInitializer(pResolver)); // Start the server. channelFuture = bootstrap.bind(addr); - this.port = addr.getPort(); + try { + // Get the address we bound to. + InetSocketAddress boundAddress = + ((InetSocketAddress) channelFuture.sync().channel().localAddress()); + this.port = boundAddress.getPort(); + } catch (InterruptedException ie) { + this.port = 0; + } } /** * Start the file server asynchronously in a new thread. */ public void start() { - try { - blockingThread = new Thread() { - public void run() { - try { - Channel channel = channelFuture.sync().channel(); - channel.closeFuture().sync(); - } catch (InterruptedException e) { - LOG.error("File server start got interrupted", e); - } + blockingThread = new Thread() { + public void run() { + try { + channelFuture.channel().closeFuture().sync(); + LOG.info("FileServer exiting"); + } catch (InterruptedException e) { + LOG.error("File server start got interrupted", e); } - }; - blockingThread.setDaemon(true); - blockingThread.start(); - } finally { - bootstrap.shutdown(); - } + // NOTE: bootstrap is shutdown in stop() + } + }; + blockingThread.setDaemon(true); + blockingThread.start(); } public int getPort() { @@ -67,17 +71,16 @@ public int getPort() { } public void stop() { - if (blockingThread != null) { - blockingThread.stop(); - blockingThread = null; - } + // Close the bound channel. if (channelFuture != null) { - channelFuture.channel().closeFuture(); + channelFuture.channel().close(); channelFuture = null; } + // Shutdown bootstrap. if (bootstrap != null) { bootstrap.shutdown(); bootstrap = null; } + // TODO: Shutdown all accepted channels as well ? } } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index fdee7ca384605..a4fe14b9ae4dd 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -305,9 +305,20 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(c.partitioner.get === p) } + test("shuffle local cluster") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[2,1,512]", "test") + val a = sc.parallelize(1 to 10, 2) + val b = a.map { + x => (x, x * 2) + } + val c = new ShuffledRDD(b, new HashPartitioner(3)) + assert(c.count === 10) + } + test("shuffle serializer") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[1,2,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test") val a = sc.parallelize(1 to 10, 2) val b = a.map { x => (x, new ShuffleSuite.NonJavaSerializableClass(x * 2)) @@ -317,6 +328,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val c = new ShuffledRDD(b, new HashPartitioner(3), classOf[spark.KryoSerializer].getName) assert(c.count === 10) } + } object ShuffleSuite { From b79b10a6d60a7f1f199e6bddd1243a05c57526ad Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 29 May 2013 00:52:55 -0700 Subject: [PATCH 605/696] Flush serializer to fix zero-size kryo blocks bug. Also convert the local-cluster test case to check for non-zero block sizes --- .../main/scala/spark/storage/DiskStore.scala | 2 ++ core/src/test/scala/spark/ShuffleSuite.scala | 22 ++++++++++++++----- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 57d4dafefc56a..1829c2f92e2ea 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -59,6 +59,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Flush the partial writes, and set valid length to be the length of the entire file. // Return the number of bytes written for this commit. override def commit(): Long = { + // NOTE: Flush the serializer first and then the compressed/buffered output stream + objOut.flush() bs.flush() val prevPos = lastValidPosition lastValidPosition = channel.position() diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index a4fe14b9ae4dd..271f4a4e44177 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -305,15 +305,27 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(c.partitioner.get === p) } - test("shuffle local cluster") { - // Use a local cluster with 2 processes to make sure there are both local and remote blocks + test("shuffle non-zero block size") { sc = new SparkContext("local-cluster[2,1,512]", "test") + val NUM_BLOCKS = 3 + val a = sc.parallelize(1 to 10, 2) - val b = a.map { - x => (x, x * 2) + val b = a.map { x => + (x, new ShuffleSuite.NonJavaSerializableClass(x * 2)) } - val c = new ShuffledRDD(b, new HashPartitioner(3)) + // If the Kryo serializer is not used correctly, the shuffle would fail because the + // default Java serializer cannot handle the non serializable class. + val c = new ShuffledRDD(b, new HashPartitioner(NUM_BLOCKS), + classOf[spark.KryoSerializer].getName) + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + assert(c.count === 10) + + // All blocks must have non-zero size + (0 until NUM_BLOCKS).foreach { id => + val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) + assert(statuses.forall(s => s._2 > 0)) + } } test("shuffle serializer") { From 618c8cae1ee5dede98824823e00f7863571c0e57 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 29 May 2013 13:09:58 -0700 Subject: [PATCH 606/696] Skip fetching zero-sized blocks in OIO. Also unify splitLocalRemoteBlocks for netty/nio and add a test case --- .../spark/storage/BlockFetcherIterator.scala | 61 ++++--------------- core/src/test/scala/spark/ShuffleSuite.scala | 27 ++++++++ 2 files changed, 39 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 95308c728273c..1d69d658f7d8a 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -124,6 +124,7 @@ object BlockFetcherIterator { protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. + val originalTotalBlocks = _totalBlocks val remoteRequests = new ArrayBuffer[FetchRequest] for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { @@ -140,8 +141,15 @@ object BlockFetcherIterator { var curBlocks = new ArrayBuffer[(String, Long)] while (iterator.hasNext) { val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size + // Skip empty blocks + if (size > 0) { + curBlocks += ((blockId, size)) + curRequestSize += size + } else if (size == 0) { + _totalBlocks -= 1 + } else { + throw new BlockException(blockId, "Negative block size " + size) + } if (curRequestSize >= minRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) @@ -155,6 +163,8 @@ object BlockFetcherIterator { } } } + logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + + originalTotalBlocks + " blocks") remoteRequests } @@ -278,53 +288,6 @@ object BlockFetcherIterator { logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) } - override protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = _totalBlocks; - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - if (size > 0) { - curBlocks += ((blockId, size)) - curRequestSize += size - } else if (size == 0) { - //here we changes the totalBlocks - _totalBlocks -= 1 - } else { - throw new BlockException(blockId, "Negative block size " + size) - } - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + - originalTotalBlocks + " blocks") - remoteRequests - } - private var copiers: List[_ <: Thread] = null override def initialize() { diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index fdee7ca384605..4e50ae2ca9821 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -317,6 +317,33 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val c = new ShuffledRDD(b, new HashPartitioner(3), classOf[spark.KryoSerializer].getName) assert(c.count === 10) } + + test("zero sized blocks") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[2,1,512]", "test") + + // 10 partitions from 4 keys + val NUM_BLOCKS = 10 + val a = sc.parallelize(1 to 4, NUM_BLOCKS) + val b = a.map(x => (x, x*2)) + + // NOTE: The default Java serializer doesn't create zero-sized blocks. + // So, use Kryo + val c = new ShuffledRDD(b, new HashPartitioner(10), classOf[spark.KryoSerializer].getName) + + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + assert(c.count === 4) + + val blockSizes = (0 until NUM_BLOCKS).flatMap { id => + val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) + statuses.map(x => x._2) + } + val nonEmptyBlocks = blockSizes.filter(x => x > 0) + + // We should have at most 4 non-zero sized partitions + assert(nonEmptyBlocks.size <= 4) + } + } object ShuffleSuite { From 19fd6d54c012bd9f73620e9b817f4975de162277 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 29 May 2013 17:29:34 -0700 Subject: [PATCH 607/696] Also flush serializer in revertPartialWrites --- core/src/main/scala/spark/storage/DiskStore.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 1829c2f92e2ea..c7281200e7e00 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -70,6 +70,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def revertPartialWrites() { // Discard current writes. We do this by flushing the outstanding writes and // truncate the file to the last valid position. + objOut.flush() bs.flush() channel.truncate(lastValidPosition) } From ecceb101d3019ef511c42a8a8a3bb0e46520ffef Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 30 May 2013 10:43:01 +0800 Subject: [PATCH 608/696] implement FIFO and fair scheduler for spark local mode --- .../scheduler/cluster/ClusterScheduler.scala | 2 +- .../cluster/ClusterTaskSetManager.scala | 734 ++++++++++++++++++ .../scheduler/cluster/TaskSetManager.scala | 733 +---------------- .../scheduler/local/LocalScheduler.scala | 386 +++++++-- .../scheduler/ClusterSchedulerSuite.scala | 2 +- 5 files changed, 1057 insertions(+), 800 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 053d4b8e4ab5f..3a0c29b27fecc 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -177,7 +177,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val tasks = taskSet.tasks logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") this.synchronized { - val manager = new TaskSetManager(this, taskSet) + val manager = new ClusterTaskSetManager(this, taskSet) activeTaskSets(taskSet.id) = manager schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) taskSetTaskIds(taskSet.id) = new HashSet[Long]() diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala new file mode 100644 index 0000000000000..ec4041ab869e6 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -0,0 +1,734 @@ +package spark.scheduler.cluster + +import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.math.max +import scala.math.min + +import spark._ +import spark.scheduler._ +import spark.TaskState.TaskState +import java.nio.ByteBuffer + +private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { + + // process local is expected to be used ONLY within tasksetmanager for now. + val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value + + type TaskLocality = Value + + def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { + + // Must not be the constraint. + assert (constraint != TaskLocality.PROCESS_LOCAL) + + constraint match { + case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL + case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL + // For anything else, allow + case _ => true + } + } + + def parse(str: String): TaskLocality = { + // better way to do this ? + try { + val retval = TaskLocality.withName(str) + // Must not specify PROCESS_LOCAL ! + assert (retval != TaskLocality.PROCESS_LOCAL) + + retval + } catch { + case nEx: NoSuchElementException => { + logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); + // default to preserve earlier behavior + NODE_LOCAL + } + } + } +} + +/** + * Schedules the tasks within a single TaskSet in the ClusterScheduler. + */ +private[spark] class ClusterTaskSetManager( + sched: ClusterScheduler, + val taskSet: TaskSet) + extends TaskSetManager + with Logging { + + // Maximum time to wait to run a task in a preferred location (in ms) + val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong + + // CPUs to request per task + val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble + + // Maximum times a task is allowed to fail before failing the job + val MAX_TASK_FAILURES = 4 + + // Quantile of tasks at which to start speculation + val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble + + // Serializer for closures and tasks. + val ser = SparkEnv.get.closureSerializer.newInstance() + + val tasks = taskSet.tasks + val numTasks = tasks.length + val copiesRunning = new Array[Int](numTasks) + val finished = new Array[Boolean](numTasks) + val numFailures = new Array[Int](numTasks) + val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) + var tasksFinished = 0 + + var weight = 1 + var minShare = 0 + var runningTasks = 0 + var priority = taskSet.priority + var stageId = taskSet.stageId + var name = "TaskSet_"+taskSet.stageId.toString + var parent:Schedulable = null + + // Last time when we launched a preferred task (for delay scheduling) + var lastPreferredLaunchTime = System.currentTimeMillis + + // List of pending tasks for each node (process local to container). These collections are actually + // treated as stacks, in which new tasks are added to the end of the + // ArrayBuffer and removed from the end. This makes it faster to detect + // tasks that repeatedly fail because whenever a task failed, it is put + // back at the head of the stack. They are also only cleaned up lazily; + // when a task is launched, it remains in all the pending lists except + // the one that it was launched from, but gets removed from them later. + private val pendingTasksForHostPort = new HashMap[String, ArrayBuffer[Int]] + + // List of pending tasks for each node. + // Essentially, similar to pendingTasksForHostPort, except at host level + private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] + + // List of pending tasks for each node based on rack locality. + // Essentially, similar to pendingTasksForHost, except at rack level + private val pendingRackLocalTasksForHost = new HashMap[String, ArrayBuffer[Int]] + + // List containing pending tasks with no locality preferences + val pendingTasksWithNoPrefs = new ArrayBuffer[Int] + + // List containing all pending tasks (also used as a stack, as above) + val allPendingTasks = new ArrayBuffer[Int] + + // Tasks that can be speculated. Since these will be a small fraction of total + // tasks, we'll just hold them in a HashSet. + val speculatableTasks = new HashSet[Int] + + // Task index, start and finish time for each task attempt (indexed by task ID) + val taskInfos = new HashMap[Long, TaskInfo] + + // Did the job fail? + var failed = false + var causeOfFailure = "" + + // How frequently to reprint duplicate exceptions in full, in milliseconds + val EXCEPTION_PRINT_INTERVAL = + System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong + // Map of recent exceptions (identified by string representation and + // top stack frame) to duplicate count (how many times the same + // exception has appeared) and time the full exception was + // printed. This should ideally be an LRU map that can drop old + // exceptions automatically. + val recentExceptions = HashMap[String, (Int, Long)]() + + // Figure out the current map output tracker generation and set it on all tasks + val generation = sched.mapOutputTracker.getGeneration + logDebug("Generation for " + taskSet.id + ": " + generation) + for (t <- tasks) { + t.generation = generation + } + + // 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. + for (i <- (0 until numTasks).reverse) { + addPendingTask(i) + } + + // Note that it follows the hierarchy. + // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and + // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL + private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, + taskLocality: TaskLocality.TaskLocality): HashSet[String] = { + + if (TaskLocality.PROCESS_LOCAL == taskLocality) { + // straight forward comparison ! Special case it. + val retval = new HashSet[String]() + scheduler.synchronized { + for (location <- _taskPreferredLocations) { + if (scheduler.isExecutorAliveOnHostPort(location)) { + retval += location + } + } + } + + return retval + } + + val taskPreferredLocations = + if (TaskLocality.NODE_LOCAL == taskLocality) { + _taskPreferredLocations + } else { + assert (TaskLocality.RACK_LOCAL == taskLocality) + // Expand set to include all 'seen' rack local hosts. + // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // Best case effort, and maybe sort of kludge for now ... rework it later ? + val hosts = new HashSet[String] + _taskPreferredLocations.foreach(h => { + val rackOpt = scheduler.getRackForHost(h) + if (rackOpt.isDefined) { + val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) + if (hostsOpt.isDefined) { + hosts ++= hostsOpt.get + } + } + + // Ensure that irrespective of what scheduler says, host is always added ! + hosts += h + }) + + hosts + } + + val retval = new HashSet[String] + scheduler.synchronized { + for (prefLocation <- taskPreferredLocations) { + val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1) + if (aliveLocationsOpt.isDefined) { + retval ++= aliveLocationsOpt.get + } + } + } + + retval + } + + // Add a task to all the pending-task lists that it should be on. + private def addPendingTask(index: Int) { + // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate + // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. + val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) + val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + + if (rackLocalLocations.size == 0) { + // Current impl ensures this. + assert (processLocalLocations.size == 0) + assert (hostLocalLocations.size == 0) + pendingTasksWithNoPrefs += index + } else { + + // process local locality + for (hostPort <- processLocalLocations) { + // DEBUG Code + Utils.checkHostPort(hostPort) + + val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer()) + hostPortList += index + } + + // host locality (includes process local) + for (hostPort <- hostLocalLocations) { + // DEBUG Code + Utils.checkHostPort(hostPort) + + val host = Utils.parseHostPort(hostPort)._1 + val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) + hostList += index + } + + // rack locality (includes process local and host local) + for (rackLocalHostPort <- rackLocalLocations) { + // DEBUG Code + Utils.checkHostPort(rackLocalHostPort) + + val rackLocalHost = Utils.parseHostPort(rackLocalHostPort)._1 + val list = pendingRackLocalTasksForHost.getOrElseUpdate(rackLocalHost, ArrayBuffer()) + list += index + } + } + + allPendingTasks += index + } + + // Return the pending tasks list for a given host port (process local), or an empty list if + // there is no map entry for that host + private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { + // DEBUG Code + Utils.checkHostPort(hostPort) + pendingTasksForHostPort.getOrElse(hostPort, ArrayBuffer()) + } + + // Return the pending tasks list for a given host, or an empty list if + // there is no map entry for that host + private def getPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { + val host = Utils.parseHostPort(hostPort)._1 + pendingTasksForHost.getOrElse(host, ArrayBuffer()) + } + + // Return the pending tasks (rack level) list for a given host, or an empty list if + // there is no map entry for that host + private def getRackLocalPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { + val host = Utils.parseHostPort(hostPort)._1 + pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) + } + + // Number of pending tasks for a given host Port (which would be process local) + def numPendingTasksForHostPort(hostPort: String): Int = { + getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + // Number of pending tasks for a given host (which would be data local) + def numPendingTasksForHost(hostPort: String): Int = { + getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + // Number of pending rack local tasks for a given host + def numRackLocalPendingTasksForHost(hostPort: String): Int = { + getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + + // Dequeue a pending task from the given list and return its index. + // Return None if the list is empty. + // This method also cleans up any tasks in the list that have already + // been launched, since we want that to happen lazily. + private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { + while (!list.isEmpty) { + val index = list.last + list.trimEnd(1) + if (copiesRunning(index) == 0 && !finished(index)) { + return Some(index) + } + } + return None + } + + // Return a speculative task for a given host if any are available. The task should not have an + // attempt running on this host, in case the host is slow. In addition, if locality is set, the + // task must have a preference for this host/rack/no preferred locations at all. + private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + + assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) + speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set + + if (speculatableTasks.size > 0) { + val localTask = speculatableTasks.find { + index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) + } + + if (localTask != None) { + speculatableTasks -= localTask.get + return localTask + } + + // check for rack locality + if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + val rackTask = speculatableTasks.find { + index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + locations.contains(hostPort) && !attemptLocs.contains(hostPort) + } + + if (rackTask != None) { + speculatableTasks -= rackTask.get + return rackTask + } + } + + // Any task ... + if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { + // Check for attemptLocs also ? + val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) + if (nonLocalTask != None) { + speculatableTasks -= nonLocalTask.get + return nonLocalTask + } + } + } + return None + } + + // Dequeue a pending task for a given node and return its index. + // If localOnly is set to false, allow non-local tasks as well. + private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (processLocalTask != None) { + return processLocalTask + } + + val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) + if (localTask != None) { + return localTask + } + + if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort)) + if (rackLocalTask != None) { + return rackLocalTask + } + } + + // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. + // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). + val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) + if (noPrefTask != None) { + return noPrefTask + } + + if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { + val nonLocalTask = findTaskFromList(allPendingTasks) + if (nonLocalTask != None) { + return nonLocalTask + } + } + + // Finally, if all else has failed, find a speculative task + return findSpeculativeTask(hostPort, locality) + } + + private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = { + Utils.checkHostPort(hostPort) + + val locs = task.preferredLocations + + locs.contains(hostPort) + } + + private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = { + val locs = task.preferredLocations + + // If no preference, consider it as host local + if (locs.isEmpty) return true + + val host = Utils.parseHostPort(hostPort)._1 + locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined + } + + // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). + // This is true if either the task has preferred locations and this host is one, or it has + // no preferred locations (in which we still count the launch as preferred). + private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { + + val locs = task.preferredLocations + + val preferredRacks = new HashSet[String]() + for (preferredHost <- locs) { + val rack = sched.getRackForHost(preferredHost) + if (None != rack) preferredRacks += rack.get + } + + if (preferredRacks.isEmpty) return false + + val hostRack = sched.getRackForHost(hostPort) + + return None != hostRack && preferredRacks.contains(hostRack.get) + } + + // Respond to an offer of a single slave from the scheduler by finding a task + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + + if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { + // If explicitly specified, use that + val locality = if (overrideLocality != null) overrideLocality else { + // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... + val time = System.currentTimeMillis + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY + } + + findTask(hostPort, locality) match { + case Some(index) => { + // Found a task; do some bookkeeping and return a Mesos task for it + val task = tasks(index) + val taskId = sched.newTaskId() + // Figure out whether this should count as a preferred launch + val taskLocality = + if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else + if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else + if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else + TaskLocality.ANY + val prefStr = taskLocality.toString + logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( + taskSet.id, index, taskId, execId, hostPort, prefStr)) + // Do various bookkeeping + copiesRunning(index) += 1 + val time = System.currentTimeMillis + val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) + taskInfos(taskId) = info + taskAttempts(index) = info :: taskAttempts(index) + if (TaskLocality.NODE_LOCAL == taskLocality) { + lastPreferredLaunchTime = time + } + // Serialize and return the task + val startTime = System.currentTimeMillis + val serializedTask = Task.serializeWithDependencies( + task, sched.sc.addedFiles, sched.sc.addedJars, ser) + val timeTaken = System.currentTimeMillis - startTime + increaseRunningTasks(1) + logInfo("Serialized task %s:%d as %d bytes in %d ms".format( + taskSet.id, index, serializedTask.limit, timeTaken)) + val taskName = "task %s:%d".format(taskSet.id, index) + return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) + } + case _ => + } + } + return None + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + state match { + case TaskState.FINISHED => + taskFinished(tid, state, serializedData) + case TaskState.LOST => + taskLost(tid, state, serializedData) + case TaskState.FAILED => + taskLost(tid, state, serializedData) + case TaskState.KILLED => + taskLost(tid, state, serializedData) + case _ => + } + } + + def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + if (info.failed) { + // We might get two task-lost messages for the same task in coarse-grained Mesos mode, + // or even from Mesos itself when acks get delayed. + return + } + val index = info.index + info.markSuccessful() + decreaseRunningTasks(1) + if (!finished(index)) { + tasksFinished += 1 + logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( + tid, info.duration, tasksFinished, numTasks)) + // Deserialize task result and pass it to the scheduler + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + // Mark finished and stop if we've finished all the tasks + finished(index) = true + if (tasksFinished == numTasks) { + sched.taskSetFinished(this) + } + } else { + logInfo("Ignoring task-finished event for TID " + tid + + " because task " + index + " is already finished") + } + } + + def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + if (info.failed) { + // We might get two task-lost messages for the same task in coarse-grained Mesos mode, + // or even from Mesos itself when acks get delayed. + return + } + val index = info.index + info.markFailed() + decreaseRunningTasks(1) + if (!finished(index)) { + logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) + copiesRunning(index) -= 1 + // Check if the problem is a map output fetch failure. In that case, this + // task will never succeed on any node, so tell the scheduler about it. + if (serializedData != null && serializedData.limit() > 0) { + val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader) + reason match { + case fetchFailed: FetchFailed => + logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) + sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) + finished(index) = true + tasksFinished += 1 + sched.taskSetFinished(this) + decreaseRunningTasks(runningTasks) + return + + case ef: ExceptionFailure => + val key = ef.description + val now = System.currentTimeMillis + val (printFull, dupCount) = { + if (recentExceptions.contains(key)) { + val (dupCount, printTime) = recentExceptions(key) + if (now - printTime > EXCEPTION_PRINT_INTERVAL) { + recentExceptions(key) = (0, now) + (true, 0) + } else { + recentExceptions(key) = (dupCount + 1, printTime) + (false, dupCount + 1) + } + } else { + recentExceptions(key) = (0, now) + (true, 0) + } + } + if (printFull) { + val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s\n%s".format( + ef.className, ef.description, locs.mkString("\n"))) + } else { + logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) + } + + case _ => {} + } + } + // On non-fetch failures, re-enqueue the task as pending for a max number of retries + addPendingTask(index) + // Count failed attempts only on FAILED and LOST state (not on KILLED) + if (state == TaskState.FAILED || state == TaskState.LOST) { + numFailures(index) += 1 + if (numFailures(index) > MAX_TASK_FAILURES) { + logError("Task %s:%d failed more than %d times; aborting job".format( + taskSet.id, index, MAX_TASK_FAILURES)) + abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) + } + } + } else { + logInfo("Ignoring task-lost event for TID " + tid + + " because task " + index + " is already finished") + } + } + + def error(message: String) { + // Save the error message + abort("Error: " + message) + } + + def abort(message: String) { + failed = true + causeOfFailure = message + // TODO: Kill running tasks if we were not terminated due to a Mesos error + sched.listener.taskSetFailed(taskSet, message) + decreaseRunningTasks(runningTasks) + sched.taskSetFinished(this) + } + + override def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + override def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + override def getSchedulableByName(name: String): Schedulable = { + return null + } + + override def addSchedulable(schedulable:Schedulable) { + //nothing + } + + override def removeSchedulable(schedulable:Schedulable) { + //nothing + } + + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue + } + + override def executorLost(execId: String, hostPort: String) { + logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) + + // If some task has preferred locations only on hostname, and there are no more executors there, + // put it in the no-prefs list to avoid the wait from delay scheduling + + // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to + // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. + // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no process local node for the task) + for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { + // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + if (newLocs.isEmpty) { + pendingTasksWithNoPrefs += index + } + } + + // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage + if (tasks(0).isInstanceOf[ShuffleMapTask]) { + for ((tid, info) <- taskInfos if info.executorId == execId) { + val index = taskInfos(tid).index + if (finished(index)) { + finished(index) = false + copiesRunning(index) -= 1 + tasksFinished -= 1 + addPendingTask(index) + // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our + // stage finishes when a total of tasks.size tasks finish. + sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null) + } + } + } + // Also re-enqueue any tasks that were running on the node + for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { + taskLost(tid, TaskState.KILLED, null) + } + } + + /** + * Check for tasks to be speculated and return true if there are any. This is called periodically + * by the ClusterScheduler. + * + * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that + * we don't scan the whole task set. It might also help to make this sorted by launch time. + */ + override def checkSpeculatableTasks(): Boolean = { + // Can't speculate if we only have one task, or if all tasks have finished. + if (numTasks == 1 || tasksFinished == numTasks) { + return false + } + var foundTasks = false + val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt + logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) + if (tasksFinished >= minFinishedForSpeculation) { + val time = System.currentTimeMillis() + val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray + Arrays.sort(durations) + val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1)) + val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100) + // TODO: Threshold should also look at standard deviation of task durations and have a lower + // bound based on that. + logDebug("Task length threshold for speculation: " + threshold) + for ((tid, info) <- taskInfos) { + val index = info.index + if (!finished(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && + !speculatableTasks.contains(index)) { + logInfo( + "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( + taskSet.id, index, info.hostPort, threshold)) + speculatableTasks += index + foundTasks = true + } + } + } + return foundTasks + } + + override def hasPendingTasks(): Boolean = { + numTasks > 0 && tasksFinished < numTasks + } +} diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 1c403ef3234fd..2b5a74d4e5481 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -1,734 +1,17 @@ package spark.scheduler.cluster -import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} - import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.math.max -import scala.math.min - -import spark._ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer - -private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { - - // process local is expected to be used ONLY within tasksetmanager for now. - val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value - - type TaskLocality = Value - - def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { - - // Must not be the constraint. - assert (constraint != TaskLocality.PROCESS_LOCAL) - - constraint match { - case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL - case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL - // For anything else, allow - case _ => true - } - } - - def parse(str: String): TaskLocality = { - // better way to do this ? - try { - val retval = TaskLocality.withName(str) - // Must not specify PROCESS_LOCAL ! - assert (retval != TaskLocality.PROCESS_LOCAL) - - retval - } catch { - case nEx: NoSuchElementException => { - logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); - // default to preserve earlier behavior - NODE_LOCAL - } - } - } -} - /** - * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class TaskSetManager( - sched: ClusterScheduler, - val taskSet: TaskSet) - extends Schedulable - with Logging { - - // Maximum time to wait to run a task in a preferred location (in ms) - val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong - - // CPUs to request per task - val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble - - // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = 4 - - // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble - - // Serializer for closures and tasks. - val ser = SparkEnv.get.closureSerializer.newInstance() - - val tasks = taskSet.tasks - val numTasks = tasks.length - val copiesRunning = new Array[Int](numTasks) - val finished = new Array[Boolean](numTasks) - val numFailures = new Array[Int](numTasks) - val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) - var tasksFinished = 0 - - var weight = 1 - var minShare = 0 - var runningTasks = 0 - var priority = taskSet.priority - var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null - - // Last time when we launched a preferred task (for delay scheduling) - var lastPreferredLaunchTime = System.currentTimeMillis - - // List of pending tasks for each node (process local to container). These collections are actually - // treated as stacks, in which new tasks are added to the end of the - // ArrayBuffer and removed from the end. This makes it faster to detect - // tasks that repeatedly fail because whenever a task failed, it is put - // back at the head of the stack. They are also only cleaned up lazily; - // when a task is launched, it remains in all the pending lists except - // the one that it was launched from, but gets removed from them later. - private val pendingTasksForHostPort = new HashMap[String, ArrayBuffer[Int]] - - // List of pending tasks for each node. - // Essentially, similar to pendingTasksForHostPort, except at host level - private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] - - // List of pending tasks for each node based on rack locality. - // Essentially, similar to pendingTasksForHost, except at rack level - private val pendingRackLocalTasksForHost = new HashMap[String, ArrayBuffer[Int]] - - // List containing pending tasks with no locality preferences - val pendingTasksWithNoPrefs = new ArrayBuffer[Int] - - // List containing all pending tasks (also used as a stack, as above) - val allPendingTasks = new ArrayBuffer[Int] - - // Tasks that can be speculated. Since these will be a small fraction of total - // tasks, we'll just hold them in a HashSet. - val speculatableTasks = new HashSet[Int] - - // Task index, start and finish time for each task attempt (indexed by task ID) - val taskInfos = new HashMap[Long, TaskInfo] - - // Did the job fail? - var failed = false - var causeOfFailure = "" - - // How frequently to reprint duplicate exceptions in full, in milliseconds - val EXCEPTION_PRINT_INTERVAL = - System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong - // Map of recent exceptions (identified by string representation and - // top stack frame) to duplicate count (how many times the same - // exception has appeared) and time the full exception was - // printed. This should ideally be an LRU map that can drop old - // exceptions automatically. - val recentExceptions = HashMap[String, (Int, Long)]() - - // Figure out the current map output tracker generation and set it on all tasks - val generation = sched.mapOutputTracker.getGeneration - logDebug("Generation for " + taskSet.id + ": " + generation) - for (t <- tasks) { - t.generation = generation - } - - // 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. - for (i <- (0 until numTasks).reverse) { - addPendingTask(i) - } - - // Note that it follows the hierarchy. - // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and - // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL - private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, - taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - - if (TaskLocality.PROCESS_LOCAL == taskLocality) { - // straight forward comparison ! Special case it. - val retval = new HashSet[String]() - scheduler.synchronized { - for (location <- _taskPreferredLocations) { - if (scheduler.isExecutorAliveOnHostPort(location)) { - retval += location - } - } - } - - return retval - } - - val taskPreferredLocations = - if (TaskLocality.NODE_LOCAL == taskLocality) { - _taskPreferredLocations - } else { - assert (TaskLocality.RACK_LOCAL == taskLocality) - // Expand set to include all 'seen' rack local hosts. - // This works since container allocation/management happens within master - so any rack locality information is updated in msater. - // Best case effort, and maybe sort of kludge for now ... rework it later ? - val hosts = new HashSet[String] - _taskPreferredLocations.foreach(h => { - val rackOpt = scheduler.getRackForHost(h) - if (rackOpt.isDefined) { - val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) - if (hostsOpt.isDefined) { - hosts ++= hostsOpt.get - } - } - - // Ensure that irrespective of what scheduler says, host is always added ! - hosts += h - }) - - hosts - } - - val retval = new HashSet[String] - scheduler.synchronized { - for (prefLocation <- taskPreferredLocations) { - val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1) - if (aliveLocationsOpt.isDefined) { - retval ++= aliveLocationsOpt.get - } - } - } - - retval - } - - // Add a task to all the pending-task lists that it should be on. - private def addPendingTask(index: Int) { - // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate - // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - - if (rackLocalLocations.size == 0) { - // Current impl ensures this. - assert (processLocalLocations.size == 0) - assert (hostLocalLocations.size == 0) - pendingTasksWithNoPrefs += index - } else { - - // process local locality - for (hostPort <- processLocalLocations) { - // DEBUG Code - Utils.checkHostPort(hostPort) - - val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer()) - hostPortList += index - } - - // host locality (includes process local) - for (hostPort <- hostLocalLocations) { - // DEBUG Code - Utils.checkHostPort(hostPort) - - val host = Utils.parseHostPort(hostPort)._1 - val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) - hostList += index - } - - // rack locality (includes process local and host local) - for (rackLocalHostPort <- rackLocalLocations) { - // DEBUG Code - Utils.checkHostPort(rackLocalHostPort) - - val rackLocalHost = Utils.parseHostPort(rackLocalHostPort)._1 - val list = pendingRackLocalTasksForHost.getOrElseUpdate(rackLocalHost, ArrayBuffer()) - list += index - } - } - - allPendingTasks += index - } - - // Return the pending tasks list for a given host port (process local), or an empty list if - // there is no map entry for that host - private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { - // DEBUG Code - Utils.checkHostPort(hostPort) - pendingTasksForHostPort.getOrElse(hostPort, ArrayBuffer()) - } - - // Return the pending tasks list for a given host, or an empty list if - // there is no map entry for that host - private def getPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { - val host = Utils.parseHostPort(hostPort)._1 - pendingTasksForHost.getOrElse(host, ArrayBuffer()) - } - - // Return the pending tasks (rack level) list for a given host, or an empty list if - // there is no map entry for that host - private def getRackLocalPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { - val host = Utils.parseHostPort(hostPort)._1 - pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) - } - - // Number of pending tasks for a given host Port (which would be process local) - def numPendingTasksForHostPort(hostPort: String): Int = { - getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) - } - - // Number of pending tasks for a given host (which would be data local) - def numPendingTasksForHost(hostPort: String): Int = { - getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) - } - - // Number of pending rack local tasks for a given host - def numRackLocalPendingTasksForHost(hostPort: String): Int = { - getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) - } - - - // Dequeue a pending task from the given list and return its index. - // Return None if the list is empty. - // This method also cleans up any tasks in the list that have already - // been launched, since we want that to happen lazily. - private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { - while (!list.isEmpty) { - val index = list.last - list.trimEnd(1) - if (copiesRunning(index) == 0 && !finished(index)) { - return Some(index) - } - } - return None - } - - // Return a speculative task for a given host if any are available. The task should not have an - // attempt running on this host, in case the host is slow. In addition, if locality is set, the - // task must have a preference for this host/rack/no preferred locations at all. - private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - - assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) - speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set - - if (speculatableTasks.size > 0) { - val localTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) - } - - if (localTask != None) { - speculatableTasks -= localTask.get - return localTask - } - - // check for rack locality - if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - locations.contains(hostPort) && !attemptLocs.contains(hostPort) - } - - if (rackTask != None) { - speculatableTasks -= rackTask.get - return rackTask - } - } - - // Any task ... - if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - // Check for attemptLocs also ? - val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) - if (nonLocalTask != None) { - speculatableTasks -= nonLocalTask.get - return nonLocalTask - } - } - } - return None - } - - // Dequeue a pending task for a given node and return its index. - // If localOnly is set to false, allow non-local tasks as well. - private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) - if (processLocalTask != None) { - return processLocalTask - } - - val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) - if (localTask != None) { - return localTask - } - - if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort)) - if (rackLocalTask != None) { - return rackLocalTask - } - } - - // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. - // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). - val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) - if (noPrefTask != None) { - return noPrefTask - } - - if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - val nonLocalTask = findTaskFromList(allPendingTasks) - if (nonLocalTask != None) { - return nonLocalTask - } - } - - // Finally, if all else has failed, find a speculative task - return findSpeculativeTask(hostPort, locality) - } - - private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = { - Utils.checkHostPort(hostPort) - - val locs = task.preferredLocations - - locs.contains(hostPort) - } - - private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = { - val locs = task.preferredLocations - - // If no preference, consider it as host local - if (locs.isEmpty) return true - - val host = Utils.parseHostPort(hostPort)._1 - locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined - } - - // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). - // This is true if either the task has preferred locations and this host is one, or it has - // no preferred locations (in which we still count the launch as preferred). - private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { - - val locs = task.preferredLocations - - val preferredRacks = new HashSet[String]() - for (preferredHost <- locs) { - val rack = sched.getRackForHost(preferredHost) - if (None != rack) preferredRacks += rack.get - } - - if (preferredRacks.isEmpty) return false - - val hostRack = sched.getRackForHost(hostPort) - - return None != hostRack && preferredRacks.contains(hostRack.get) - } - - // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { - - if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { - // If explicitly specified, use that - val locality = if (overrideLocality != null) overrideLocality else { - // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... - val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY - } - - findTask(hostPort, locality) match { - case Some(index) => { - // Found a task; do some bookkeeping and return a Mesos task for it - val task = tasks(index) - val taskId = sched.newTaskId() - // Figure out whether this should count as a preferred launch - val taskLocality = - if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else - if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else - if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else - TaskLocality.ANY - val prefStr = taskLocality.toString - logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( - taskSet.id, index, taskId, execId, hostPort, prefStr)) - // Do various bookkeeping - copiesRunning(index) += 1 - val time = System.currentTimeMillis - val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) - taskInfos(taskId) = info - taskAttempts(index) = info :: taskAttempts(index) - if (TaskLocality.NODE_LOCAL == taskLocality) { - lastPreferredLaunchTime = time - } - // Serialize and return the task - val startTime = System.currentTimeMillis - val serializedTask = Task.serializeWithDependencies( - task, sched.sc.addedFiles, sched.sc.addedJars, ser) - val timeTaken = System.currentTimeMillis - startTime - increaseRunningTasks(1) - logInfo("Serialized task %s:%d as %d bytes in %d ms".format( - taskSet.id, index, serializedTask.limit, timeTaken)) - val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) - } - case _ => - } - } - return None - } - - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - state match { - case TaskState.FINISHED => - taskFinished(tid, state, serializedData) - case TaskState.LOST => - taskLost(tid, state, serializedData) - case TaskState.FAILED => - taskLost(tid, state, serializedData) - case TaskState.KILLED => - taskLost(tid, state, serializedData) - case _ => - } - } - - def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. - return - } - val index = info.index - info.markSuccessful() - decreaseRunningTasks(1) - if (!finished(index)) { - tasksFinished += 1 - logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( - tid, info.duration, tasksFinished, numTasks)) - // Deserialize task result and pass it to the scheduler - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) - // Mark finished and stop if we've finished all the tasks - finished(index) = true - if (tasksFinished == numTasks) { - sched.taskSetFinished(this) - } - } else { - logInfo("Ignoring task-finished event for TID " + tid + - " because task " + index + " is already finished") - } - } - - def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. - return - } - val index = info.index - info.markFailed() - decreaseRunningTasks(1) - if (!finished(index)) { - logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) - copiesRunning(index) -= 1 - // Check if the problem is a map output fetch failure. In that case, this - // task will never succeed on any node, so tell the scheduler about it. - if (serializedData != null && serializedData.limit() > 0) { - val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader) - reason match { - case fetchFailed: FetchFailed => - logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) - finished(index) = true - tasksFinished += 1 - sched.taskSetFinished(this) - decreaseRunningTasks(runningTasks) - return - - case ef: ExceptionFailure => - val key = ef.description - val now = System.currentTimeMillis - val (printFull, dupCount) = { - if (recentExceptions.contains(key)) { - val (dupCount, printTime) = recentExceptions(key) - if (now - printTime > EXCEPTION_PRINT_INTERVAL) { - recentExceptions(key) = (0, now) - (true, 0) - } else { - recentExceptions(key) = (dupCount + 1, printTime) - (false, dupCount + 1) - } - } else { - recentExceptions(key) = (0, now) - (true, 0) - } - } - if (printFull) { - val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s\n%s".format( - ef.className, ef.description, locs.mkString("\n"))) - } else { - logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) - } - - case _ => {} - } - } - // On non-fetch failures, re-enqueue the task as pending for a max number of retries - addPendingTask(index) - // Count failed attempts only on FAILED and LOST state (not on KILLED) - if (state == TaskState.FAILED || state == TaskState.LOST) { - numFailures(index) += 1 - if (numFailures(index) > MAX_TASK_FAILURES) { - logError("Task %s:%d failed more than %d times; aborting job".format( - taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) - } - } - } else { - logInfo("Ignoring task-lost event for TID " + tid + - " because task " + index + " is already finished") - } - } - - def error(message: String) { - // Save the error message - abort("Error: " + message) - } - - def abort(message: String) { - failed = true - causeOfFailure = message - // TODO: Kill running tasks if we were not terminated due to a Mesos error - sched.listener.taskSetFailed(taskSet, message) - decreaseRunningTasks(runningTasks) - sched.taskSetFinished(this) - } - - override def increaseRunningTasks(taskNum: Int) { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - override def decreaseRunningTasks(taskNum: Int) { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed - override def getSchedulableByName(name: String): Schedulable = { - return null - } - - override def addSchedulable(schedulable:Schedulable) { - //nothing - } - - override def removeSchedulable(schedulable:Schedulable) { - //nothing - } - - override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - sortedTaskSetQueue += this - return sortedTaskSetQueue - } - - override def executorLost(execId: String, hostPort: String) { - logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) - - // If some task has preferred locations only on hostname, and there are no more executors there, - // put it in the no-prefs list to avoid the wait from delay scheduling - - // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to - // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no process local node for the task) - for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { - // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - if (newLocs.isEmpty) { - pendingTasksWithNoPrefs += index - } - } - - // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage - if (tasks(0).isInstanceOf[ShuffleMapTask]) { - for ((tid, info) <- taskInfos if info.executorId == execId) { - val index = taskInfos(tid).index - if (finished(index)) { - finished(index) = false - copiesRunning(index) -= 1 - tasksFinished -= 1 - addPendingTask(index) - // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our - // stage finishes when a total of tasks.size tasks finish. - sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null) - } - } - } - // Also re-enqueue any tasks that were running on the node - for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { - taskLost(tid, TaskState.KILLED, null) - } - } - - /** - * Check for tasks to be speculated and return true if there are any. This is called periodically - * by the ClusterScheduler. - * - * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that - * we don't scan the whole task set. It might also help to make this sorted by launch time. - */ - override def checkSpeculatableTasks(): Boolean = { - // Can't speculate if we only have one task, or if all tasks have finished. - if (numTasks == 1 || tasksFinished == numTasks) { - return false - } - var foundTasks = false - val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt - logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) - if (tasksFinished >= minFinishedForSpeculation) { - val time = System.currentTimeMillis() - val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray - Arrays.sort(durations) - val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1)) - val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100) - // TODO: Threshold should also look at standard deviation of task durations and have a lower - // bound based on that. - logDebug("Task length threshold for speculation: " + threshold) - for ((tid, info) <- taskInfos) { - val index = info.index - if (!finished(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && - !speculatableTasks.contains(index)) { - logInfo( - "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( - taskSet.id, index, info.hostPort, threshold)) - speculatableTasks += index - foundTasks = true - } - } - } - return foundTasks - } - - override def hasPendingTasks(): Boolean = { - numTasks > 0 && tasksFinished < numTasks - } +private[spark] trait TaskSetManager extends Schedulable { + def taskSet: TaskSet + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + def numPendingTasksForHostPort(hostPort: String): Int + def numRackLocalPendingTasksForHost(hostPort :String): Int + def numPendingTasksForHost(hostPort: String): Int + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) + def error(message: String) } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 37a67f9b1b977..664dc9e886e2c 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -2,19 +2,215 @@ package spark.scheduler.local import java.io.File import java.util.concurrent.atomic.AtomicInteger +import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet import spark._ +import spark.TaskState.TaskState import spark.executor.ExecutorURLClassLoader import spark.scheduler._ -import spark.scheduler.cluster.{TaskLocality, TaskInfo} +import spark.scheduler.cluster._ +import akka.actor._ /** * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally * the scheduler also allows each task to fail up to maxFailures times, which is useful for * testing fault recovery. */ -private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkContext) + +private[spark] case class LocalReviveOffers() +private[spark] case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) + +private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { + def receive = { + case LocalReviveOffers => + logInfo("LocalReviveOffers") + launchTask(localScheduler.resourceOffer(freeCores)) + case LocalStatusUpdate(taskId, state, serializeData) => + logInfo("LocalStatusUpdate") + freeCores += 1 + localScheduler.statusUpdate(taskId, state, serializeData) + launchTask(localScheduler.resourceOffer(freeCores)) + } + + def launchTask(tasks : Seq[TaskDescription]) { + for (task <- tasks) + { + freeCores -= 1 + localScheduler.threadPool.submit(new Runnable { + def run() { + localScheduler.runTask(task.taskId,task.serializedTask) + } + }) + } + } +} + +private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { + var parent: Schedulable = null + var weight: Int = 1 + var minShare: Int = 0 + var runningTasks: Int = 0 + var priority: Int = taskSet.priority + var stageId: Int = taskSet.stageId + var name: String = "TaskSet_"+taskSet.stageId.toString + + + var failCount = new Array[Int](taskSet.tasks.size) + val taskInfos = new HashMap[Long, TaskInfo] + val numTasks = taskSet.tasks.size + var numFinished = 0 + val ser = SparkEnv.get.closureSerializer.newInstance() + val copiesRunning = new Array[Int](numTasks) + val finished = new Array[Boolean](numTasks) + val numFailures = new Array[Int](numTasks) + + def increaseRunningTasks(taskNum: Int): Unit = { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int): Unit = { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + def addSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def removeSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def getSchedulableByName(name: String): Schedulable = { + return null + } + + def executorLost(executorId: String, host: String): Unit = { + //nothing + } + + def checkSpeculatableTasks(): Boolean = { + return true + } + + def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue + } + + def hasPendingTasks(): Boolean = { + return true + } + + def findTask(): Option[Int] = { + for (i <- 0 to numTasks-1) { + if (copiesRunning(i) == 0 && !finished(i)) { + return Some(i) + } + } + return None + } + + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + Thread.currentThread().setContextClassLoader(sched.classLoader) + SparkEnv.set(sched.env) + logInfo("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) + if (availableCpus > 0 && numFinished < numTasks) { + findTask() match { + case Some(index) => + logInfo(taskSet.tasks(index).toString) + val taskId = sched.attemptId.getAndIncrement() + val task = taskSet.tasks(index) + logInfo("taskId:%d,task:%s".format(index,task)) + val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + taskInfos(taskId) = info + val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") + val taskName = "task %s:%d".format(taskSet.id, index) + copiesRunning(index) += 1 + increaseRunningTasks(1) + return Some(new TaskDescription(taskId, null, taskName, bytes)) + case None => {} + } + } + return None + } + + def numPendingTasksForHostPort(hostPort: String): Int = { + return 0 + } + + def numRackLocalPendingTasksForHost(hostPort :String): Int = { + return 0 + } + + def numPendingTasksForHost(hostPort: String): Int = { + return 0 + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + state match { + case TaskState.FINISHED => + taskEnded(tid, state, serializedData) + case TaskState.FAILED => + taskFailed(tid, state, serializedData) + case _ => {} + } + } + + def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markSuccessful() + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics) + numFinished += 1 + decreaseRunningTasks(1) + finished(index) = true + if (numFinished == numTasks) { + sched.taskSetFinished(this) + } + } + + def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markFailed() + decreaseRunningTasks(1) + val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + if (!finished(index)) { + copiesRunning(index) -= 1 + numFailures(index) += 1 + val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) + if (numFailures(index) > 4) { + val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) + //logError(errorMessage) + //sched.listener.taskEnded(task, reason, null, null, info, null) + sched.listener.taskSetFailed(taskSet, errorMessage) + sched.taskSetFinished(this) + decreaseRunningTasks(runningTasks) + } + } + } + + def error(message: String) { + } +} + +private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext) extends TaskScheduler with Logging { @@ -30,90 +226,126 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon val classLoader = new ExecutorURLClassLoader(Array(), Thread.currentThread.getContextClassLoader) + var schedulableBuilder: SchedulableBuilder = null + var rootPool: Pool = null + val activeTaskSets = new HashMap[String, TaskSetManager] + val taskIdToTaskSetId = new HashMap[Long, String] + val taskSetTaskIds = new HashMap[String, HashSet[Long]] + + var localActor: ActorRef = null // TODO: Need to take into account stage priority in scheduling - override def start() { } + override def start() { + //default scheduler is FIFO + val schedulingMode = System.getProperty("spark.cluster.schedulingmode", "FIFO") + //temporarily set rootPool name to empty + rootPool = new Pool("", SchedulingMode.withName(schedulingMode), 0, 0) + schedulableBuilder = { + schedulingMode match { + case "FIFO" => + new FIFOSchedulableBuilder(rootPool) + case "FAIR" => + new FairSchedulableBuilder(rootPool) + } + } + schedulableBuilder.buildPools() + + //val properties = new ArrayBuffer[(String, String)] + localActor = env.actorSystem.actorOf( + Props(new LocalActor(this, threads)), "Test") + } override def setListener(listener: TaskSchedulerListener) { this.listener = listener } override def submitTasks(taskSet: TaskSet) { - val tasks = taskSet.tasks - val failCount = new Array[Int](tasks.size) + var manager = new LocalTaskSetManager(this, taskSet) + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) + activeTaskSets(taskSet.id) = manager + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + localActor ! LocalReviveOffers + } - def submitTask(task: Task[_], idInJob: Int) { - val myAttemptId = attemptId.getAndIncrement() - threadPool.submit(new Runnable { - def run() { - runTask(task, idInJob, myAttemptId) - } - }) + def resourceOffer(freeCores: Int): Seq[TaskDescription] = { + var freeCpuCores = freeCores + val tasks = new ArrayBuffer[TaskDescription](freeCores) + val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() + for (manager <- sortedTaskSetQueue) { + logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } - def runTask(task: Task[_], idInJob: Int, attemptId: Int) { - logInfo("Running " + task) - val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) - // Set the Spark execution environment for the worker thread - SparkEnv.set(env) - try { - Accumulators.clear() - Thread.currentThread().setContextClassLoader(classLoader) - - // Serialize and deserialize the task so that accumulators are changed to thread-local ones; - // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. - val ser = SparkEnv.get.closureSerializer.newInstance() - val bytes = Task.serializeWithDependencies(task, sc.addedFiles, sc.addedJars, ser) - logInfo("Size of task " + idInJob + " is " + bytes.limit + " bytes") - val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) - updateDependencies(taskFiles, taskJars) // Download any files added with addFile - val deserStart = System.currentTimeMillis() - val deserializedTask = ser.deserialize[Task[_]]( - taskBytes, Thread.currentThread.getContextClassLoader) - val deserTime = System.currentTimeMillis() - deserStart - - // Run it - val result: Any = deserializedTask.run(attemptId) - - // Serialize and deserialize the result to emulate what the Mesos - // executor does. This is useful to catch serialization errors early - // on in development (so when users move their local Spark programs - // to the cluster, they don't get surprised by serialization errors). - val serResult = ser.serialize(result) - deserializedTask.metrics.get.resultSize = serResult.limit() - val resultToReturn = ser.deserialize[Any](serResult) - val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( - ser.serialize(Accumulators.values)) - logInfo("Finished " + task) - info.markSuccessful() - deserializedTask.metrics.get.executorRunTime = info.duration.toInt //close enough - deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt - - // If the threadpool has not already been shutdown, notify DAGScheduler - if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, deserializedTask.metrics.getOrElse(null)) - } catch { - case t: Throwable => { - logError("Exception in task " + idInJob, t) - failCount.synchronized { - failCount(idInJob) += 1 - if (failCount(idInJob) <= maxFailures) { - submitTask(task, idInJob) - } else { - // TODO: Do something nicer here to return all the way to the user - if (!Thread.currentThread().isInterrupted) { - val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) - listener.taskEnded(task, failure, null, null, info, null) - } - } + var launchTask = false + for (manager <- sortedTaskSetQueue) { + do { + launchTask = false + logInfo("freeCores is" + freeCpuCores) + manager.slaveOffer(null,null,freeCpuCores) match { + case Some(task) => + tasks += task + taskIdToTaskSetId(task.taskId) = manager.taskSet.id + taskSetTaskIds(manager.taskSet.id) += task.taskId + freeCpuCores -= 1 + launchTask = true + case None => {} } - } - } + } while(launchTask) } + return tasks + } - for ((task, i) <- tasks.zipWithIndex) { - submitTask(task, i) - } + def taskSetFinished(manager: TaskSetManager) { + activeTaskSets -= manager.taskSet.id + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds -= manager.taskSet.id + } + + def runTask(taskId: Long, bytes: ByteBuffer) { + logInfo("Running " + taskId) + val info = new TaskInfo(taskId, 0 , System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + // Set the Spark execution environment for the worker thread + SparkEnv.set(env) + val ser = SparkEnv.get.closureSerializer.newInstance() + try { + Accumulators.clear() + Thread.currentThread().setContextClassLoader(classLoader) + + // Serialize and deserialize the task so that accumulators are changed to thread-local ones; + // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. + val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) + updateDependencies(taskFiles, taskJars) // Download any files added with addFile + val deserStart = System.currentTimeMillis() + val deserializedTask = ser.deserialize[Task[_]]( + taskBytes, Thread.currentThread.getContextClassLoader) + val deserTime = System.currentTimeMillis() - deserStart + + // Run it + val result: Any = deserializedTask.run(taskId) + + // Serialize and deserialize the result to emulate what the Mesos + // executor does. This is useful to catch serialization errors early + // on in development (so when users move their local Spark programs + // to the cluster, they don't get surprised by serialization errors). + val serResult = ser.serialize(result) + deserializedTask.metrics.get.resultSize = serResult.limit() + val resultToReturn = ser.deserialize[Any](serResult) + val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( + ser.serialize(Accumulators.values)) + logInfo("Finished " + taskId) + deserializedTask.metrics.get.executorRunTime = deserTime.toInt//info.duration.toInt //close enough + deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt + + val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) + val serializedResult = ser.serialize(taskResult) + localActor ! LocalStatusUpdate(taskId, TaskState.FINISHED, serializedResult) + } catch { + case t: Throwable => { + val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure)) + } + } } /** @@ -128,6 +360,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) currentFiles(name) = timestamp } + for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) @@ -143,7 +376,14 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } } - override def stop() { + def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) + { + val taskSetId = taskIdToTaskSetId(taskId) + val taskSetManager = activeTaskSets(taskSetId) + taskSetManager.statusUpdate(taskId, state, serializedData) + } + + override def stop() { threadPool.shutdownNow() } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index a39418b71618f..e6ad90192eba2 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -16,7 +16,7 @@ class DummyTaskSetManager( initNumTasks: Int, clusterScheduler: ClusterScheduler, taskSet: TaskSet) - extends TaskSetManager(clusterScheduler,taskSet) { + extends ClusterTaskSetManager(clusterScheduler,taskSet) { parent = null weight = 1 From c3db3ea55467c3fb053453c8c567db357d939640 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 30 May 2013 20:49:40 +0800 Subject: [PATCH 609/696] 1. Add unit test for local scheduler 2. Move localTaskSetManager to a new file --- .../scheduler/local/LocalScheduler.scala | 241 +++--------------- .../scheduler/local/LocalTaskSetManager.scala | 173 +++++++++++++ .../spark/scheduler/LocalSchedulerSuite.scala | 171 +++++++++++++ 3 files changed, 385 insertions(+), 200 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala create mode 100644 core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 664dc9e886e2c..69dacfc2bd87c 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -15,7 +15,7 @@ import spark.scheduler.cluster._ import akka.actor._ /** - * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally + * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally * the scheduler also allows each task to fail up to maxFailures times, which is useful for * testing fault recovery. */ @@ -26,10 +26,8 @@ private[spark] case class LocalStatusUpdate(taskId: Long, state: TaskState, seri private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { def receive = { case LocalReviveOffers => - logInfo("LocalReviveOffers") launchTask(localScheduler.resourceOffer(freeCores)) case LocalStatusUpdate(taskId, state, serializeData) => - logInfo("LocalStatusUpdate") freeCores += 1 localScheduler.statusUpdate(taskId, state, serializeData) launchTask(localScheduler.resourceOffer(freeCores)) @@ -48,168 +46,6 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: I } } -private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { - var parent: Schedulable = null - var weight: Int = 1 - var minShare: Int = 0 - var runningTasks: Int = 0 - var priority: Int = taskSet.priority - var stageId: Int = taskSet.stageId - var name: String = "TaskSet_"+taskSet.stageId.toString - - - var failCount = new Array[Int](taskSet.tasks.size) - val taskInfos = new HashMap[Long, TaskInfo] - val numTasks = taskSet.tasks.size - var numFinished = 0 - val ser = SparkEnv.get.closureSerializer.newInstance() - val copiesRunning = new Array[Int](numTasks) - val finished = new Array[Boolean](numTasks) - val numFailures = new Array[Int](numTasks) - - def increaseRunningTasks(taskNum: Int): Unit = { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - def decreaseRunningTasks(taskNum: Int): Unit = { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - def addSchedulable(schedulable: Schedulable): Unit = { - //nothing - } - - def removeSchedulable(schedulable: Schedulable): Unit = { - //nothing - } - - def getSchedulableByName(name: String): Schedulable = { - return null - } - - def executorLost(executorId: String, host: String): Unit = { - //nothing - } - - def checkSpeculatableTasks(): Boolean = { - return true - } - - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - sortedTaskSetQueue += this - return sortedTaskSetQueue - } - - def hasPendingTasks(): Boolean = { - return true - } - - def findTask(): Option[Int] = { - for (i <- 0 to numTasks-1) { - if (copiesRunning(i) == 0 && !finished(i)) { - return Some(i) - } - } - return None - } - - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { - Thread.currentThread().setContextClassLoader(sched.classLoader) - SparkEnv.set(sched.env) - logInfo("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) - if (availableCpus > 0 && numFinished < numTasks) { - findTask() match { - case Some(index) => - logInfo(taskSet.tasks(index).toString) - val taskId = sched.attemptId.getAndIncrement() - val task = taskSet.tasks(index) - logInfo("taskId:%d,task:%s".format(index,task)) - val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) - taskInfos(taskId) = info - val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) - logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") - val taskName = "task %s:%d".format(taskSet.id, index) - copiesRunning(index) += 1 - increaseRunningTasks(1) - return Some(new TaskDescription(taskId, null, taskName, bytes)) - case None => {} - } - } - return None - } - - def numPendingTasksForHostPort(hostPort: String): Int = { - return 0 - } - - def numRackLocalPendingTasksForHost(hostPort :String): Int = { - return 0 - } - - def numPendingTasksForHost(hostPort: String): Int = { - return 0 - } - - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - state match { - case TaskState.FINISHED => - taskEnded(tid, state, serializedData) - case TaskState.FAILED => - taskFailed(tid, state, serializedData) - case _ => {} - } - } - - def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - val index = info.index - val task = taskSet.tasks(index) - info.markSuccessful() - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics) - numFinished += 1 - decreaseRunningTasks(1) - finished(index) = true - if (numFinished == numTasks) { - sched.taskSetFinished(this) - } - } - - def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - val index = info.index - val task = taskSet.tasks(index) - info.markFailed() - decreaseRunningTasks(1) - val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) - if (!finished(index)) { - copiesRunning(index) -= 1 - numFailures(index) += 1 - val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) - if (numFailures(index) > 4) { - val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) - //logError(errorMessage) - //sched.listener.taskEnded(task, reason, null, null, info, null) - sched.listener.taskSetFailed(taskSet, errorMessage) - sched.taskSetFinished(this) - decreaseRunningTasks(runningTasks) - } - } - } - - def error(message: String) { - } -} - private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext) extends TaskScheduler with Logging { @@ -233,7 +69,6 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val taskSetTaskIds = new HashMap[String, HashSet[Long]] var localActor: ActorRef = null - // TODO: Need to take into account stage priority in scheduling override def start() { //default scheduler is FIFO @@ -250,7 +85,6 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } schedulableBuilder.buildPools() - //val properties = new ArrayBuffer[(String, String)] localActor = env.actorSystem.actorOf( Props(new LocalActor(this, threads)), "Test") } @@ -260,51 +94,56 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } override def submitTasks(taskSet: TaskSet) { - var manager = new LocalTaskSetManager(this, taskSet) - schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) - activeTaskSets(taskSet.id) = manager - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - localActor ! LocalReviveOffers + synchronized { + var manager = new LocalTaskSetManager(this, taskSet) + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) + activeTaskSets(taskSet.id) = manager + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + localActor ! LocalReviveOffers + } } def resourceOffer(freeCores: Int): Seq[TaskDescription] = { - var freeCpuCores = freeCores - val tasks = new ArrayBuffer[TaskDescription](freeCores) - val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() - for (manager <- sortedTaskSetQueue) { - logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) - } + synchronized { + var freeCpuCores = freeCores + val tasks = new ArrayBuffer[TaskDescription](freeCores) + val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() + for (manager <- sortedTaskSetQueue) { + logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + } - var launchTask = false - for (manager <- sortedTaskSetQueue) { + var launchTask = false + for (manager <- sortedTaskSetQueue) { do { launchTask = false - logInfo("freeCores is" + freeCpuCores) manager.slaveOffer(null,null,freeCpuCores) match { case Some(task) => - tasks += task - taskIdToTaskSetId(task.taskId) = manager.taskSet.id - taskSetTaskIds(manager.taskSet.id) += task.taskId - freeCpuCores -= 1 - launchTask = true + tasks += task + taskIdToTaskSetId(task.taskId) = manager.taskSet.id + taskSetTaskIds(manager.taskSet.id) += task.taskId + freeCpuCores -= 1 + launchTask = true case None => {} - } + } } while(launchTask) + } + return tasks } - return tasks } def taskSetFinished(manager: TaskSetManager) { - activeTaskSets -= manager.taskSet.id - manager.parent.removeSchedulable(manager) - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) - taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskSetTaskIds -= manager.taskSet.id + synchronized { + activeTaskSets -= manager.taskSet.id + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds -= manager.taskSet.id + } } def runTask(taskId: Long, bytes: ByteBuffer) { logInfo("Running " + taskId) - val info = new TaskInfo(taskId, 0 , System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + val info = new TaskInfo(taskId, 0, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) // Set the Spark execution environment for the worker thread SparkEnv.set(env) val ser = SparkEnv.get.closureSerializer.newInstance() @@ -344,8 +183,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: case t: Throwable => { val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure)) - } } + } } /** @@ -376,11 +215,13 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } } - def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) - { - val taskSetId = taskIdToTaskSetId(taskId) - val taskSetManager = activeTaskSets(taskSetId) - taskSetManager.statusUpdate(taskId, state, serializedData) + def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) { + synchronized { + val taskSetId = taskIdToTaskSetId(taskId) + val taskSetManager = activeTaskSets(taskSetId) + taskSetTaskIds(taskSetId) -= taskId + taskSetManager.statusUpdate(taskId, state, serializedData) + } } override def stop() { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala new file mode 100644 index 0000000000000..f2e07d162a035 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -0,0 +1,173 @@ +package spark.scheduler.local + +import java.io.File +import java.util.concurrent.atomic.AtomicInteger +import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + +import spark._ +import spark.TaskState.TaskState +import spark.scheduler._ +import spark.scheduler.cluster._ + +private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { + var parent: Schedulable = null + var weight: Int = 1 + var minShare: Int = 0 + var runningTasks: Int = 0 + var priority: Int = taskSet.priority + var stageId: Int = taskSet.stageId + var name: String = "TaskSet_"+taskSet.stageId.toString + + + var failCount = new Array[Int](taskSet.tasks.size) + val taskInfos = new HashMap[Long, TaskInfo] + val numTasks = taskSet.tasks.size + var numFinished = 0 + val ser = SparkEnv.get.closureSerializer.newInstance() + val copiesRunning = new Array[Int](numTasks) + val finished = new Array[Boolean](numTasks) + val numFailures = new Array[Int](numTasks) + val MAX_TASK_FAILURES = sched.maxFailures + + def increaseRunningTasks(taskNum: Int): Unit = { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int): Unit = { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + def addSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def removeSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def getSchedulableByName(name: String): Schedulable = { + return null + } + + def executorLost(executorId: String, host: String): Unit = { + //nothing + } + + def checkSpeculatableTasks(): Boolean = { + return true + } + + def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue + } + + def hasPendingTasks(): Boolean = { + return true + } + + def findTask(): Option[Int] = { + for (i <- 0 to numTasks-1) { + if (copiesRunning(i) == 0 && !finished(i)) { + return Some(i) + } + } + return None + } + + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + SparkEnv.set(sched.env) + logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) + if (availableCpus > 0 && numFinished < numTasks) { + findTask() match { + case Some(index) => + logInfo(taskSet.tasks(index).toString) + val taskId = sched.attemptId.getAndIncrement() + val task = taskSet.tasks(index) + val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + taskInfos(taskId) = info + val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") + val taskName = "task %s:%d".format(taskSet.id, index) + copiesRunning(index) += 1 + increaseRunningTasks(1) + return Some(new TaskDescription(taskId, null, taskName, bytes)) + case None => {} + } + } + return None + } + + def numPendingTasksForHostPort(hostPort: String): Int = { + return 0 + } + + def numRackLocalPendingTasksForHost(hostPort :String): Int = { + return 0 + } + + def numPendingTasksForHost(hostPort: String): Int = { + return 0 + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + state match { + case TaskState.FINISHED => + taskEnded(tid, state, serializedData) + case TaskState.FAILED => + taskFailed(tid, state, serializedData) + case _ => {} + } + } + + def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markSuccessful() + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics) + numFinished += 1 + decreaseRunningTasks(1) + finished(index) = true + if (numFinished == numTasks) { + sched.taskSetFinished(this) + } + } + + def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markFailed() + decreaseRunningTasks(1) + val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + if (!finished(index)) { + copiesRunning(index) -= 1 + numFailures(index) += 1 + val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) + if (numFailures(index) > MAX_TASK_FAILURES) { + val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) + decreaseRunningTasks(runningTasks) + sched.listener.taskSetFailed(taskSet, errorMessage) + // need to delete failed Taskset from schedule queue + sched.taskSetFinished(this) + } + } + } + + def error(message: String) { + } +} diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala new file mode 100644 index 0000000000000..37d14ed113b61 --- /dev/null +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -0,0 +1,171 @@ +package spark.scheduler + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter + +import spark._ +import spark.scheduler._ +import spark.scheduler.cluster._ +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ConcurrentMap, HashMap} +import java.util.concurrent.Semaphore +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.AtomicInteger + +import java.util.Properties + +class Lock() { + var finished = false + def jobWait() = { + synchronized { + while(!finished) { + this.wait() + } + } + } + + def jobFinished() = { + synchronized { + finished = true + this.notifyAll() + } + } +} + +object TaskThreadInfo { + val threadToLock = HashMap[Int, Lock]() + val threadToRunning = HashMap[Int, Boolean]() +} + + +class LocalSchedulerSuite extends FunSuite with LocalSparkContext { + + def createThread(threadIndex: Int, poolName: String, sc: SparkContext, sem: Semaphore) { + + TaskThreadInfo.threadToRunning(threadIndex) = false + val nums = sc.parallelize(threadIndex to threadIndex, 1) + TaskThreadInfo.threadToLock(threadIndex) = new Lock() + new Thread { + if (poolName != null) { + sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) + } + override def run() { + val ans = nums.map(number => { + TaskThreadInfo.threadToRunning(number) = true + TaskThreadInfo.threadToLock(number).jobWait() + number + }).collect() + assert(ans.toList === List(threadIndex)) + sem.release() + TaskThreadInfo.threadToRunning(threadIndex) = false + } + }.start() + Thread.sleep(2000) + } + + test("Local FIFO scheduler end-to-end test") { + System.setProperty("spark.cluster.schedulingmode", "FIFO") + sc = new SparkContext("local[4]", "test") + val sem = new Semaphore(0) + + createThread(1,null,sc,sem) + createThread(2,null,sc,sem) + createThread(3,null,sc,sem) + createThread(4,null,sc,sem) + createThread(5,null,sc,sem) + createThread(6,null,sc,sem) + assert(TaskThreadInfo.threadToRunning(1) === true) + assert(TaskThreadInfo.threadToRunning(2) === true) + assert(TaskThreadInfo.threadToRunning(3) === true) + assert(TaskThreadInfo.threadToRunning(4) === true) + assert(TaskThreadInfo.threadToRunning(5) === false) + assert(TaskThreadInfo.threadToRunning(6) === false) + + TaskThreadInfo.threadToLock(1).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(1) === false) + assert(TaskThreadInfo.threadToRunning(2) === true) + assert(TaskThreadInfo.threadToRunning(3) === true) + assert(TaskThreadInfo.threadToRunning(4) === true) + assert(TaskThreadInfo.threadToRunning(5) === true) + assert(TaskThreadInfo.threadToRunning(6) === false) + + TaskThreadInfo.threadToLock(3).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(1) === false) + assert(TaskThreadInfo.threadToRunning(2) === true) + assert(TaskThreadInfo.threadToRunning(3) === false) + assert(TaskThreadInfo.threadToRunning(4) === true) + assert(TaskThreadInfo.threadToRunning(5) === true) + assert(TaskThreadInfo.threadToRunning(6) === true) + + TaskThreadInfo.threadToLock(2).jobFinished() + TaskThreadInfo.threadToLock(4).jobFinished() + TaskThreadInfo.threadToLock(5).jobFinished() + TaskThreadInfo.threadToLock(6).jobFinished() + sem.acquire(6) + } + + test("Local fair scheduler end-to-end test") { + sc = new SparkContext("local[8]", "LocalSchedulerSuite") + val sem = new Semaphore(0) + System.setProperty("spark.cluster.schedulingmode", "FAIR") + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + System.setProperty("spark.fairscheduler.allocation.file", xmlPath) + + createThread(10,"1",sc,sem) + createThread(20,"2",sc,sem) + createThread(30,"3",sc,sem) + + assert(TaskThreadInfo.threadToRunning(10) === true) + assert(TaskThreadInfo.threadToRunning(20) === true) + assert(TaskThreadInfo.threadToRunning(30) === true) + + createThread(11,"1",sc,sem) + createThread(21,"2",sc,sem) + createThread(31,"3",sc,sem) + + assert(TaskThreadInfo.threadToRunning(11) === true) + assert(TaskThreadInfo.threadToRunning(21) === true) + assert(TaskThreadInfo.threadToRunning(31) === true) + + createThread(12,"1",sc,sem) + createThread(22,"2",sc,sem) + createThread(32,"3",sc,sem) + + assert(TaskThreadInfo.threadToRunning(12) === true) + assert(TaskThreadInfo.threadToRunning(22) === true) + assert(TaskThreadInfo.threadToRunning(32) === false) + + TaskThreadInfo.threadToLock(10).jobFinished() + Thread.sleep(1000) + assert(TaskThreadInfo.threadToRunning(32) === true) + + createThread(23,"2",sc,sem) + createThread(33,"3",sc,sem) + + TaskThreadInfo.threadToLock(11).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(23) === true) + assert(TaskThreadInfo.threadToRunning(33) === false) + + TaskThreadInfo.threadToLock(12).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(33) === true) + + TaskThreadInfo.threadToLock(20).jobFinished() + TaskThreadInfo.threadToLock(21).jobFinished() + TaskThreadInfo.threadToLock(22).jobFinished() + TaskThreadInfo.threadToLock(23).jobFinished() + TaskThreadInfo.threadToLock(30).jobFinished() + TaskThreadInfo.threadToLock(31).jobFinished() + TaskThreadInfo.threadToLock(32).jobFinished() + TaskThreadInfo.threadToLock(33).jobFinished() + + sem.acquire(11) + } +} From bed1b08169df91e97cb9ebaf8e58daeb655ff55d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 May 2013 16:21:49 -0700 Subject: [PATCH 610/696] Do not create symlink for local add file. Instead, copy the file. This prevents Spark from changing the original file's permission, and also allow add file to work on non-posix operating systems. --- core/src/main/scala/spark/Utils.scala | 78 +++++++++++++-------------- 1 file changed, 39 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 84626df553a38..ec15326014e8d 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -4,20 +4,26 @@ import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} -import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} +import java.util.regex.Pattern + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ import scala.io.Source + import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder + +import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} + import spark.serializer.SerializerInstance import spark.deploy.SparkHadoopUtil -import java.util.regex.Pattern + /** * Various utility methods used by Spark. */ private object Utils extends Logging { + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -68,7 +74,6 @@ private object Utils extends Logging { return buf } - private val shutdownDeletePaths = new collection.mutable.HashSet[String]() // Register the path to be deleted via shutdown hook @@ -87,19 +92,19 @@ private object Utils extends Logging { } } - // Note: if file is child of some registered path, while not equal to it, then return true; else false - // This is to ensure that two shutdown hooks do not try to delete each others paths - resulting in IOException - // and incomplete cleanup + // Note: if file is child of some registered path, while not equal to it, then return true; + // else false. This is to ensure that two shutdown hooks do not try to delete each others + // paths - resulting in IOException and incomplete cleanup. def hasRootAsShutdownDeleteDir(file: File): Boolean = { - val absolutePath = file.getAbsolutePath() - val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find(path => ! absolutePath.equals(path) && absolutePath.startsWith(path) ).isDefined + shutdownDeletePaths.find { path => + !absolutePath.equals(path) && absolutePath.startsWith(path) + }.isDefined + } + if (retval) { + logInfo("path = " + file + ", already present as root for deletion.") } - - if (retval) logInfo("path = " + file + ", already present as root for deletion.") - retval } @@ -131,7 +136,7 @@ private object Utils extends Logging { if (! hasRootAsShutdownDeleteDir(dir)) Utils.deleteRecursively(dir) } }) - return dir + dir } /** Copy all data from an InputStream to an OutputStream */ @@ -174,35 +179,30 @@ private object Utils extends Logging { Utils.copyStream(in, out, true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { tempFile.delete() - throw new SparkException("File " + targetFile + " exists and does not match contents of" + - " " + url) + throw new SparkException( + "File " + targetFile + " exists and does not match contents of" + " " + url) } else { Files.move(tempFile, targetFile) } case "file" | null => - val sourceFile = if (uri.isAbsolute) { - new File(uri) - } else { - new File(url) - } - if (targetFile.exists && !Files.equal(sourceFile, targetFile)) { - throw new SparkException("File " + targetFile + " exists and does not match contents of" + - " " + url) - } else { - // Remove the file if it already exists - targetFile.delete() - // Symlink the file locally. - if (uri.isAbsolute) { - // url is absolute, i.e. it starts with "file:///". Extract the source - // file's absolute path from the url. - val sourceFile = new File(uri) - logInfo("Symlinking " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) - FileUtil.symLink(sourceFile.getAbsolutePath, targetFile.getAbsolutePath) + // In the case of a local file, copy the local file to the target directory. + // Note the difference between uri vs url. + val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) + if (targetFile.exists) { + // If the target file already exists, warn the user if + if (!Files.equal(sourceFile, targetFile)) { + throw new SparkException( + "File " + targetFile + " exists and does not match contents of" + " " + url) } else { - // url is not absolute, i.e. itself is the path to the source file. - logInfo("Symlinking " + url + " to " + targetFile.getAbsolutePath) - FileUtil.symLink(url, targetFile.getAbsolutePath) + // Do nothing if the file contents are the same, i.e. this file has been copied + // previously. + logInfo(sourceFile.getAbsolutePath + " has been previously copied to " + + targetFile.getAbsolutePath) } + } else { + // The file does not exist in the target directory. Copy it there. + logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) + Files.copy(sourceFile, targetFile) } case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others @@ -323,8 +323,6 @@ private object Utils extends Logging { InetAddress.getByName(address).getHostName } - - def localHostPort(): String = { val retval = System.getProperty("spark.hostPort", null) if (retval == null) { @@ -382,6 +380,7 @@ private object Utils extends Logging { // Typically, this will be of order of number of nodes in cluster // If not, we should change it to LRUCache or something. private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() + def parseHostPort(hostPort: String): (String, Int) = { { // Check cache first. @@ -390,7 +389,8 @@ private object Utils extends Logging { } val indx: Int = hostPort.lastIndexOf(':') - // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now. + // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... + // but then hadoop does not support ipv6 right now. // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 if (-1 == indx) { val retval = (hostPort, 0) From f6ad3781b1d9a044789f114d13787b9d05223da3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 May 2013 16:28:08 -0700 Subject: [PATCH 611/696] Fixed the flaky unpersist test in RDDSuite. --- core/src/test/scala/spark/RDDSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index a761dd77c5118..3f69e99780af6 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -106,9 +106,9 @@ class RDDSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() rdd.count - assert(sc.persistentRdds.isEmpty == false) + assert(sc.persistentRdds.isEmpty === false) rdd.unpersist() - assert(sc.persistentRdds.isEmpty == true) + assert(sc.persistentRdds.isEmpty === true) failAfter(Span(3000, Millis)) { try { @@ -116,12 +116,12 @@ class RDDSuite extends FunSuite with LocalSparkContext { Thread.sleep(200) } } catch { - case e: Exception => + case _ => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } } - assert(sc.getRDDStorageInfo.isEmpty == true) + assert(sc.getRDDStorageInfo.isEmpty === true) } test("caching with failures") { From 926f41cc522def181c167b71dc919a0759c5d3f6 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 30 May 2013 17:55:11 +0800 Subject: [PATCH 612/696] fix block manager UI display issue when enable spark.cleaner.ttl --- core/src/main/scala/spark/storage/StorageUtils.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 8f52168c241b3..81e607868d558 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -64,12 +64,12 @@ object StorageUtils { // Find the id of the RDD, e.g. rdd_1 => 1 val rddId = rddKey.split("_").last.toInt // Get the friendly name for the rdd, if available. - val rdd = sc.persistentRdds(rddId) - val rddName = Option(rdd.name).getOrElse(rddKey) - val rddStorageLevel = rdd.getStorageLevel - - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize) - }.toArray + sc.persistentRdds.get(rddId).map { r => + val rddName = Option(r.name).getOrElse(rddKey) + val rddStorageLevel = r.getStorageLevel + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) + } + }.flatMap(x => x).toArray scala.util.Sorting.quickSort(rddInfos) From ba5e544461e8ca9216af703033f6b0de6dbc56ec Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 31 May 2013 01:48:16 -0700 Subject: [PATCH 613/696] More block manager cleanup. Implemented a removeRdd method in BlockManager, and use that to implement RDD.unpersist. Previously, unpersist needs to send B akka messages, where B = number of blocks. Now unpersist only needs to send W akka messages, where W = the number of workers. --- core/src/main/scala/spark/RDD.scala | 21 +- .../scala/spark/storage/BlockManager.scala | 31 ++- .../spark/storage/BlockManagerMaster.scala | 49 +++-- .../storage/BlockManagerMasterActor.scala | 192 ++++++++---------- .../spark/storage/BlockManagerMessages.scala | 6 + .../storage/BlockManagerSlaveActor.scala | 8 +- .../spark/storage/BlockManagerWorker.scala | 10 +- .../spark/storage/BlockManagerSuite.scala | 36 ++-- 8 files changed, 187 insertions(+), 166 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index dde131696fdf2..e6c0438d76da6 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -1,13 +1,10 @@ package spark -import java.net.URL -import java.util.{Date, Random} -import java.util.{HashMap => JHashMap} +import java.util.Random import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.NullWritable @@ -32,7 +29,6 @@ import spark.rdd.MapPartitionsWithIndexRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD import spark.rdd.ShuffledRDD -import spark.rdd.SubtractedRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD import spark.rdd.ZippedPartitionsRDD2 @@ -141,10 +137,15 @@ abstract class RDD[T: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): RDD[T] = persist() - /** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */ - def unpersist(): RDD[T] = { + /** + * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. + * + * @param blocking Whether to block until all blocks are deleted. + * @return This RDD. + */ + def unpersist(blocking: Boolean = true): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - sc.env.blockManager.master.removeRdd(id) + sc.env.blockManager.master.removeRdd(id, blocking) sc.persistentRdds.remove(id) storageLevel = StorageLevel.NONE this @@ -269,8 +270,8 @@ abstract class RDD[T: ClassManifest]( def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { var fraction = 0.0 var total = 0 - var multiplier = 3.0 - var initialCount = count() + val multiplier = 3.0 + val initialCount = count() var maxSelected = 0 if (initialCount > Integer.MAX_VALUE - 1) { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index d35c43f194bcd..3a5d4ef448f67 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -3,8 +3,7 @@ package spark.storage import java.io.{InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet, Queue} -import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet} import akka.actor.{ActorSystem, Cancellable, Props} import akka.dispatch.{Await, Future} @@ -15,7 +14,7 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.{Logging, SizeEstimator, SparkEnv, SparkException, Utils} +import spark.{Logging, SparkEnv, SparkException, Utils} import spark.network._ import spark.serializer.Serializer import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} @@ -95,9 +94,11 @@ private[spark] class BlockManager( new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) // If we use Netty for shuffle, start a new Netty-based shuffle sender service. - private val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean - private val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt - private val nettyPort = if (useNetty) diskStore.startShuffleBlockSender(nettyPortConfig) else 0 + private val nettyPort: Int = { + val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt + if (useNetty) diskStore.startShuffleBlockSender(nettyPortConfig) else 0 + } val connectionManager = new ConnectionManager(0) implicit val futureExecContext = connectionManager.futureExecContext @@ -824,10 +825,24 @@ private[spark] class BlockManager( } } + /** + * Remove all blocks belonging to the given RDD. + * @return The number of blocks removed. + */ + def removeRdd(rddId: Int): Int = { + // TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps + // from RDD.id to blocks. + logInfo("Removing RDD " + rddId) + val rddPrefix = "rdd_" + rddId + "_" + val blocksToRemove = blockInfo.filter(_._1.startsWith(rddPrefix)).map(_._1) + blocksToRemove.foreach(blockId => removeBlock(blockId, false)) + blocksToRemove.size + } + /** * Remove a block from both memory and disk. */ - def removeBlock(blockId: String) { + def removeBlock(blockId: String, tellMaster: Boolean = true) { logInfo("Removing block " + blockId) val info = blockInfo.get(blockId).orNull if (info != null) info.synchronized { @@ -839,7 +854,7 @@ private[spark] class BlockManager( "the disk or memory store") } blockInfo.remove(blockId) - if (info.tellMaster) { + if (tellMaster && info.tellMaster) { reportBlockStatus(blockId, info) } } else { diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index ac26c16867045..7099e40618185 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -1,19 +1,11 @@ package spark.storage -import java.io._ -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random - -import akka.actor.{Actor, ActorRef, ActorSystem, Props} -import akka.dispatch.Await +import akka.actor.ActorRef +import akka.dispatch.{Await, Future} import akka.pattern.ask -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import akka.util.Duration -import spark.{Logging, SparkException, Utils} +import spark.{Logging, SparkException} private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { @@ -91,15 +83,28 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi /** * Remove all blocks belonging to the given RDD. */ - def removeRdd(rddId: Int) { - val rddBlockPrefix = "rdd_" + rddId + "_" - // Get the list of blocks in block manager, and remove ones that are part of this RDD. - // The runtime complexity is linear to the number of blocks persisted in the cluster. - // It could be expensive if the cluster is large and has a lot of blocks persisted. - getStorageStatus.flatMap(_.blocks).foreach { case(blockId, status) => - if (blockId.startsWith(rddBlockPrefix)) { - removeBlock(blockId) - } + def removeRdd(rddId: Int, blocking: Boolean) { + // The logic to remove an RDD is somewhat complicated: + // 1. Send BlockManagerMasterActor a RemoveRdd message. + // 2. Upon receiving the RemoveRdd message, BlockManagerMasterActor will forward the message + // to all workers to remove blocks belonging to the RDD, and return a Future for the results. + // 3. The Future is sent back here, and on successful completion of the Future, this function + // sends a RemoveRddMetaData message to BlockManagerMasterActor. + // 4. Upon receiving the RemoveRddMetaData message, BlockManagerMasterActor will delete the meta + // data for the given RDD. + // + // The reason we are doing it this way is to reduce the amount of messages the driver sends. + // The number of messages that need to be sent is only the number of workers the cluster has, + // rather than the number of blocks in the cluster. Note that we can further reduce the number + // of messages by tracking for a given RDD, where are its blocks. Then we can send only to the + // workers that have the given RDD. But this remains future work. + val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) + future onComplete { + case Left(throwable) => logError("Failed to remove RDD " + rddId, throwable) + case Right(numBlocks) => tell(RemoveRddMetaData(rddId, numBlocks.sum)) + } + if (blocking) { + Await.result(future, timeout) } } @@ -114,7 +119,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi } def getStorageStatus: Array[StorageStatus] = { - askDriverWithReply[ArrayBuffer[StorageStatus]](GetStorageStatus).toArray + askDriverWithReply[Array[StorageStatus]](GetStorageStatus) } /** Stop the driver actor, called only on the Spark driver node */ diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 9b64f95df8073..00aa97bf78f55 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -2,15 +2,16 @@ package spark.storage import java.util.{HashMap => JHashMap} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable import scala.collection.JavaConversions._ -import scala.util.Random import akka.actor.{Actor, ActorRef, Cancellable} -import akka.util.{Duration, Timeout} +import akka.dispatch.Future +import akka.pattern.ask +import akka.util.Duration import akka.util.duration._ -import spark.{Logging, Utils} +import spark.{Logging, Utils, SparkException} /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -21,13 +22,16 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = - new HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] + new mutable.HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] // Mapping from executor ID to block manager ID. - private val blockManagerIdByExecutor = new HashMap[String, BlockManagerId] + private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId] // Mapping from block id to the set of block managers that have the block. - private val blockLocations = new JHashMap[String, Pair[Int, HashSet[BlockManagerId]]] + private val blockLocations = new JHashMap[String, mutable.HashSet[BlockManagerId]] + + val akkaTimeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") initLogging() @@ -50,28 +54,38 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { def receive = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => register(blockManagerId, maxMemSize, slaveActor) + sender ! true case UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => + // TODO: Ideally we want to handle all the message replies in receive instead of in the + // individual private methods. updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) case GetLocations(blockId) => - getLocations(blockId) + sender ! getLocations(blockId) case GetLocationsMultipleBlockIds(blockIds) => - getLocationsMultipleBlockIds(blockIds) + sender ! getLocationsMultipleBlockIds(blockIds) case GetPeers(blockManagerId, size) => - getPeersDeterministic(blockManagerId, size) - /*getPeers(blockManagerId, size)*/ + sender ! getPeers(blockManagerId, size) case GetMemoryStatus => - getMemoryStatus + sender ! memoryStatus case GetStorageStatus => - getStorageStatus + sender ! storageStatus + + case RemoveRdd(rddId) => + sender ! removeRdd(rddId) + + case RemoveRddMetaData(rddId, numBlocks) => + removeRddMetaData(rddId, numBlocks) + sender ! true case RemoveBlock(blockId) => - removeBlock(blockId) + removeBlockFromWorkers(blockId) + sender ! true case RemoveExecutor(execId) => removeExecutor(execId) @@ -81,7 +95,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { logInfo("Stopping BlockManagerMaster") sender ! true if (timeoutCheckingTask != null) { - timeoutCheckingTask.cancel + timeoutCheckingTask.cancel() } context.stop(self) @@ -89,13 +103,34 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { expireDeadHosts() case HeartBeat(blockManagerId) => - heartBeat(blockManagerId) + sender ! heartBeat(blockManagerId) case other => - logInfo("Got unknown message: " + other) + logWarning("Got unknown message: " + other) + } + + private def removeRdd(rddId: Int): Future[Seq[Int]] = { + // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. + // The dispatcher is used as an implicit argument into the Future sequence construction. + import context.dispatcher + Future.sequence(blockManagerInfo.values.map { bm => + bm.slaveActor.ask(RemoveRdd(rddId))(akkaTimeout).mapTo[Int] + }.toSeq) + } + + private def removeRddMetaData(rddId: Int, numBlocks: Int) { + val prefix = "rdd_" + rddId + "_" + // Find all blocks for the given RDD, remove the block from both blockLocations and + // the blockManagerInfo that is tracking the blocks. + val blocks = blockLocations.keySet().filter(_.startsWith(prefix)) + blocks.foreach { blockId => + val bms: mutable.HashSet[BlockManagerId] = blockLocations.get(blockId) + bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId))) + blockLocations.remove(blockId) + } } - def removeBlockManager(blockManagerId: BlockManagerId) { + private def removeBlockManager(blockManagerId: BlockManagerId) { val info = blockManagerInfo(blockManagerId) // Remove the block manager from blockManagerIdByExecutor. @@ -106,7 +141,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { val iterator = info.blocks.keySet.iterator while (iterator.hasNext) { val blockId = iterator.next - val locations = blockLocations.get(blockId)._2 + val locations = blockLocations.get(blockId) locations -= blockManagerId if (locations.size == 0) { blockLocations.remove(locations) @@ -114,11 +149,11 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } } - def expireDeadHosts() { + private def expireDeadHosts() { logTrace("Checking for hosts with no recent heart beats in BlockManagerMaster.") val now = System.currentTimeMillis() val minSeenTime = now - slaveTimeout - val toRemove = new HashSet[BlockManagerId] + val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { if (info.lastSeenMs < minSeenTime) { logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + @@ -129,31 +164,26 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { toRemove.foreach(removeBlockManager) } - def removeExecutor(execId: String) { + private def removeExecutor(execId: String) { logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.") blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) - sender ! true } - def heartBeat(blockManagerId: BlockManagerId) { + private def heartBeat(blockManagerId: BlockManagerId): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { - if (blockManagerId.executorId == "" && !isLocal) { - sender ! true - } else { - sender ! false - } + blockManagerId.executorId == "" && !isLocal } else { blockManagerInfo(blockManagerId).updateLastSeenMs() - sender ! true + true } } // Remove a block from the slaves that have it. This can only be used to remove // blocks that the master knows about. - private def removeBlock(blockId: String) { - val block = blockLocations.get(blockId) - if (block != null) { - block._2.foreach { blockManagerId: BlockManagerId => + private def removeBlockFromWorkers(blockId: String) { + val locations = blockLocations.get(blockId) + if (locations != null) { + locations.foreach { blockManagerId: BlockManagerId => val blockManager = blockManagerInfo.get(blockManagerId) if (blockManager.isDefined) { // Remove the block from the slave's BlockManager. @@ -163,23 +193,20 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } } } - sender ! true } // Return a map from the block manager id to max memory and remaining memory. - private def getMemoryStatus() { - val res = blockManagerInfo.map { case(blockManagerId, info) => + private def memoryStatus: Map[BlockManagerId, (Long, Long)] = { + blockManagerInfo.map { case(blockManagerId, info) => (blockManagerId, (info.maxMem, info.remainingMem)) }.toMap - sender ! res } - private def getStorageStatus() { - val res = blockManagerInfo.map { case(blockManagerId, info) => + private def storageStatus: Array[StorageStatus] = { + blockManagerInfo.map { case(blockManagerId, info) => import collection.JavaConverters._ StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap) - } - sender ! res + }.toArray } private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { @@ -188,7 +215,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } else if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { case Some(manager) => - // A block manager of the same host name already exists + // A block manager of the same executor already exists. + // This should never happen. Let's just quit. logError("Got two different block manager registrations on " + id.executorId) System.exit(1) case None => @@ -197,7 +225,6 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { blockManagerInfo(id) = new BlockManagerMasterActor.BlockManagerInfo( id, System.currentTimeMillis(), maxMemSize, slaveActor) } - sender ! true } private def updateBlockInfo( @@ -226,12 +253,12 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) - var locations: HashSet[BlockManagerId] = null + var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { - locations = blockLocations.get(blockId)._2 + locations = blockLocations.get(blockId) } else { - locations = new HashSet[BlockManagerId] - blockLocations.put(blockId, (storageLevel.replication, locations)) + locations = new mutable.HashSet[BlockManagerId] + blockLocations.put(blockId, locations) } if (storageLevel.isValid) { @@ -247,70 +274,24 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { sender ! true } - private def getLocations(blockId: String) { - val startTimeMs = System.currentTimeMillis() - val tmp = " " + blockId + " " - if (blockLocations.containsKey(blockId)) { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(blockLocations.get(blockId)._2) - sender ! res.toSeq - } else { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - sender ! res - } - } - - private def getLocationsMultipleBlockIds(blockIds: Array[String]) { - def getLocations(blockId: String): Seq[BlockManagerId] = { - val tmp = blockId - if (blockLocations.containsKey(blockId)) { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(blockLocations.get(blockId)._2) - return res.toSeq - } else { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - return res.toSeq - } - } - - var res: ArrayBuffer[Seq[BlockManagerId]] = new ArrayBuffer[Seq[BlockManagerId]] - for (blockId <- blockIds) { - res.append(getLocations(blockId)) - } - sender ! res.toSeq + private def getLocations(blockId: String): Seq[BlockManagerId] = { + if (blockLocations.containsKey(blockId)) blockLocations.get(blockId).toSeq else Seq.empty } - private def getPeers(blockManagerId: BlockManagerId, size: Int) { - var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(peers) - res -= blockManagerId - val rand = new Random(System.currentTimeMillis()) - while (res.length > size) { - res.remove(rand.nextInt(res.length)) - } - sender ! res.toSeq + private def getLocationsMultipleBlockIds(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = { + blockIds.map(blockId => getLocations(blockId)) } - private def getPeersDeterministic(blockManagerId: BlockManagerId, size: Int) { - var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + private def getPeers(blockManagerId: BlockManagerId, size: Int): Seq[BlockManagerId] = { + val peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray val selfIndex = peers.indexOf(blockManagerId) if (selfIndex == -1) { - throw new Exception("Self index for " + blockManagerId + " not found") + throw new SparkException("Self index for " + blockManagerId + " not found") } // Note that this logic will select the same node multiple times if there aren't enough peers - var index = selfIndex - while (res.size < size) { - index += 1 - if (index == selfIndex) { - throw new Exception("More peer expected than available") - } - res += peers(index % peers.size) - } - sender ! res.toSeq + Array.tabulate[BlockManagerId](size) { i => peers((selfIndex + i + 1) % peers.length) }.toSeq } } @@ -384,6 +365,13 @@ object BlockManagerMasterActor { } } + def removeBlock(blockId: String) { + if (_blocks.containsKey(blockId)) { + _remainingMem += _blocks.get(blockId).memSize + _blocks.remove(blockId) + } + } + def remainingMem: Long = _remainingMem def lastSeenMs: Long = _lastSeenMs diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index cff48d9909a16..88268fd41bcfe 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -16,6 +16,12 @@ sealed trait ToBlockManagerSlave private[spark] case class RemoveBlock(blockId: String) extends ToBlockManagerSlave +// Remove all blocks belonging to a specific RDD. +private[spark] case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + +// Remove the meta data for a RDD. This is only sent to the master by the master. +private[spark] case class RemoveRddMetaData(rddId: Int, numBlocks: Int) extends ToBlockManagerMaster + ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index f570cdc52dd1b..b264d1deb59b9 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -11,6 +11,12 @@ import spark.{Logging, SparkException, Utils} */ class BlockManagerSlaveActor(blockManager: BlockManager) extends Actor { override def receive = { - case RemoveBlock(blockId) => blockManager.removeBlock(blockId) + + case RemoveBlock(blockId) => + blockManager.removeBlock(blockId) + + case RemoveRdd(rddId) => + val numBlocksRemoved = blockManager.removeRdd(rddId) + sender ! numBlocksRemoved } } diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index 15225f93a626d..3057ade23395e 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -2,13 +2,7 @@ package spark.storage import java.nio.ByteBuffer -import scala.actors._ -import scala.actors.Actor._ -import scala.actors.remote._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random - -import spark.{Logging, Utils, SparkEnv} +import spark.{Logging, Utils} import spark.network._ /** @@ -88,8 +82,6 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends private[spark] object BlockManagerWorker extends Logging { private var blockManagerWorker: BlockManagerWorker = null - private val DATA_TRANSFER_TIME_OUT_MS: Long = 500 - private val REQUEST_RETRY_INTERVAL_MS: Long = 1000 initLogging() diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index bff2475686a47..b9d5f9668e654 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -15,10 +15,10 @@ import org.scalatest.time.SpanSugar._ import spark.JavaSerializer import spark.KryoSerializer import spark.SizeEstimator -import spark.Utils import spark.util.AkkaUtils import spark.util.ByteBufferInputStream + class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { var store: BlockManager = null var store2: BlockManager = null @@ -124,7 +124,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, false) + store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) // Checking whether blocks are in memory assert(store.getSingle("a1") != None, "a1 was not in store") @@ -170,7 +170,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2-to-remove", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3-to-remove", a3, StorageLevel.MEMORY_ONLY, false) + store.putSingle("a3-to-remove", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 @@ -218,7 +218,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY) store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY) store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY) - master.removeRdd(0) + master.removeRdd(0, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { store.getSingle("rdd_0_0") should be (None) @@ -232,6 +232,14 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.getSingle("nonrddblock") should not be (None) master.getLocations("nonrddblock") should have size (1) } + + store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY) + master.removeRdd(0, blocking = true) + store.getSingle("rdd_0_0") should be (None) + master.getLocations("rdd_0_0") should have size 0 + store.getSingle("rdd_0_1") should be (None) + master.getLocations("rdd_0_1") should have size 0 } test("reregistration on heart beat") { @@ -262,7 +270,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - store.putSingle("a2", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) store.waitForAsyncReregister() assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") @@ -280,7 +288,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, true) + store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { @@ -490,9 +498,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, true) - store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, true) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list2") != None, "list2 was not in store") assert(store.get("list2").get.size == 2) assert(store.get("list3") != None, "list3 was not in store") @@ -501,7 +509,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.get("list2") != None, "list2 was not in store") assert(store.get("list2").get.size == 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, true) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list1") != None, "list1 was not in store") assert(store.get("list1").get.size == 2) assert(store.get("list2") != None, "list2 was not in store") @@ -516,9 +524,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val list3 = List(new Array[Byte](200), new Array[Byte](200)) val list4 = List(new Array[Byte](200), new Array[Byte](200)) // First store list1 and list2, both in memory, and list3, on disk only - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, true) - store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, true) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) // At this point LRU should not kick in because list3 is only on disk assert(store.get("list1") != None, "list2 was not in store") assert(store.get("list1").get.size === 2) @@ -533,7 +541,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.get("list3") != None, "list1 was not in store") assert(store.get("list3").get.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, true) + store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2") != None, "list3 was not in store") assert(store.get("list2").get.size === 2) From de1167bf2c32d52c865a4a0c7213b665ebd61f93 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 31 May 2013 15:54:57 -0700 Subject: [PATCH 614/696] Incorporated Charles' feedback to put rdd metadata removal in BlockManagerMasterActor. --- .../spark/storage/BlockManagerMaster.scala | 21 +++--------------- .../storage/BlockManagerMasterActor.scala | 22 +++++++++---------- .../spark/storage/BlockManagerMessages.scala | 3 --- 3 files changed, 13 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 7099e40618185..58888b1ebb55b 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -84,24 +84,9 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi * Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { - // The logic to remove an RDD is somewhat complicated: - // 1. Send BlockManagerMasterActor a RemoveRdd message. - // 2. Upon receiving the RemoveRdd message, BlockManagerMasterActor will forward the message - // to all workers to remove blocks belonging to the RDD, and return a Future for the results. - // 3. The Future is sent back here, and on successful completion of the Future, this function - // sends a RemoveRddMetaData message to BlockManagerMasterActor. - // 4. Upon receiving the RemoveRddMetaData message, BlockManagerMasterActor will delete the meta - // data for the given RDD. - // - // The reason we are doing it this way is to reduce the amount of messages the driver sends. - // The number of messages that need to be sent is only the number of workers the cluster has, - // rather than the number of blocks in the cluster. Note that we can further reduce the number - // of messages by tracking for a given RDD, where are its blocks. Then we can send only to the - // workers that have the given RDD. But this remains future work. val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) - future onComplete { - case Left(throwable) => logError("Failed to remove RDD " + rddId, throwable) - case Right(numBlocks) => tell(RemoveRddMetaData(rddId, numBlocks.sum)) + future onFailure { + case e: Throwable => logError("Failed to remove RDD " + rddId, e) } if (blocking) { Await.result(future, timeout) @@ -156,7 +141,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi val future = driverActor.ask(message)(timeout) val result = Await.result(future, timeout) if (result == null) { - throw new Exception("BlockManagerMaster returned null") + throw new SparkException("BlockManagerMaster returned null") } return result.asInstanceOf[T] } catch { diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 00aa97bf78f55..2d05e0ccf12fa 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -79,10 +79,6 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { case RemoveRdd(rddId) => sender ! removeRdd(rddId) - case RemoveRddMetaData(rddId, numBlocks) => - removeRddMetaData(rddId, numBlocks) - sender ! true - case RemoveBlock(blockId) => removeBlockFromWorkers(blockId) sender ! true @@ -110,15 +106,9 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } private def removeRdd(rddId: Int): Future[Seq[Int]] = { - // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. - // The dispatcher is used as an implicit argument into the Future sequence construction. - import context.dispatcher - Future.sequence(blockManagerInfo.values.map { bm => - bm.slaveActor.ask(RemoveRdd(rddId))(akkaTimeout).mapTo[Int] - }.toSeq) - } + // First remove the metadata for the given RDD, and then asynchronously remove the blocks + // from the slaves. - private def removeRddMetaData(rddId: Int, numBlocks: Int) { val prefix = "rdd_" + rddId + "_" // Find all blocks for the given RDD, remove the block from both blockLocations and // the blockManagerInfo that is tracking the blocks. @@ -128,6 +118,14 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId))) blockLocations.remove(blockId) } + + // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. + // The dispatcher is used as an implicit argument into the Future sequence construction. + import context.dispatcher + val removeMsg = RemoveRdd(rddId) + Future.sequence(blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq) } private def removeBlockManager(blockManagerId: BlockManagerId) { diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 88268fd41bcfe..0010726c8d32e 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -19,9 +19,6 @@ case class RemoveBlock(blockId: String) extends ToBlockManagerSlave // Remove all blocks belonging to a specific RDD. private[spark] case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave -// Remove the meta data for a RDD. This is only sent to the master by the master. -private[spark] case class RemoveRddMetaData(rddId: Int, numBlocks: Int) extends ToBlockManagerMaster - ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. From 9f84315c055d7a53da8787eb26b336726fc33e8a Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Sat, 1 Jun 2013 00:26:10 +0000 Subject: [PATCH 615/696] enhance pipe to support what we can do in hadoop streaming --- core/src/main/scala/spark/RDD.scala | 18 ++++++++++++++ core/src/main/scala/spark/rdd/PipedRDD.scala | 25 ++++++++++++++++++-- 2 files changed, 41 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index dde131696fdf2..5a41db23c20a4 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -348,17 +348,35 @@ abstract class RDD[T: ClassManifest]( */ def pipe(command: String): RDD[String] = new PipedRDD(this, command) + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = + new PipedRDD(this, command, transform, arguments) + /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: Seq[String]): RDD[String] = new PipedRDD(this, command) + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = + new PipedRDD(this, command, transform, arguments) + /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: Seq[String], env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: Seq[String], env: Map[String, String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = + new PipedRDD(this, command, env, transform, arguments) + /** * Return a new RDD by applying a function to each partition of this RDD. */ diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 962a1b21ad1d3..969404c95f64f 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -18,14 +18,21 @@ import spark.{RDD, SparkEnv, Partition, TaskContext} class PipedRDD[T: ClassManifest]( prev: RDD[T], command: Seq[String], - envVars: Map[String, String]) + envVars: Map[String, String], + transform: (T, String => Unit) => Any, + arguments: Seq[String] + ) extends RDD[String](prev) { + def this(prev: RDD[T], command: Seq[String], envVars : Map[String, String]) = this(prev, command, envVars, null, null) def this(prev: RDD[T], command: Seq[String]) = this(prev, command, Map()) + def this(prev: RDD[T], command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, command, Map(), transform, arguments) // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command)) + def this(prev: RDD[T], command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, PipedRDD.tokenize(command), Map(), transform, arguments) + override def getPartitions: Array[Partition] = firstParent[T].partitions @@ -52,8 +59,22 @@ class PipedRDD[T: ClassManifest]( override def run() { SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) + + // input the arguments firstly + if ( arguments != null) { + for (elem <- arguments) { + out.println(elem) + } + // ^A \n as the marker of the end of the arguments + out.println("\u0001") + } for (elem <- firstParent[T].iterator(split, context)) { - out.println(elem) + if (transform != null) { + transform(elem, out.println(_)) + } + else { + out.println(elem) + } } out.close() } From 3be7bdcefda13d67633f9b9f6d901722fd5649de Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Sat, 1 Jun 2013 19:32:17 +0530 Subject: [PATCH 616/696] Adding example to make Spark RDD from Cassandra --- .../scala/spark/examples/CassandraTest.scala | 154 ++++++++++++++++++ project/SparkBuild.scala | 4 +- 2 files changed, 156 insertions(+), 2 deletions(-) create mode 100644 examples/src/main/scala/spark/examples/CassandraTest.scala diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala new file mode 100644 index 0000000000000..790b24e6f38b1 --- /dev/null +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -0,0 +1,154 @@ +package spark.examples + +import org.apache.hadoop.mapreduce.Job +import org.apache.cassandra.hadoop.{ConfigHelper, ColumnFamilyInputFormat} +import org.apache.cassandra.thrift.{IndexExpression, SliceRange, SlicePredicate} +import spark.{RDD, SparkContext} +import SparkContext._ +import java.nio.ByteBuffer +import java.util.SortedMap +import org.apache.cassandra.db.IColumn +import org.apache.cassandra.utils.ByteBufferUtil +import scala.collection.JavaConversions._ + + +/* + * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra support for Hadoop. + * + * To run this example, run this file with the following command params - + * + * + * So if you want to run this on localhost this will be, + * local[3] localhost 9160 + * + * The example makes some assumptions: + * 1. You have already created a keyspace called casDemo and it has a column family named Words + * 2. There are column family has a column named "para" which has test content. + * + * You can create the content by running the following script at the bottom of this file with cassandra-cli. + * + */ +object CassandraTest { + def main(args: Array[String]) { + + //Get a SparkContext + val sc = new SparkContext(args(0), "casDemo") + + //Build the job configuration with ConfigHelper provided by Cassandra + val job = new Job() + job.setInputFormatClass(classOf[ColumnFamilyInputFormat]) + + ConfigHelper.setInputInitialAddress(job.getConfiguration(), args(1)) + + ConfigHelper.setInputRpcPort(job.getConfiguration(), args(2)) + + ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") + + val predicate = new SlicePredicate() + val sliceRange = new SliceRange() + sliceRange.setStart(Array.empty[Byte]) + sliceRange.setFinish(Array.empty[Byte]) + predicate.setSlice_range(sliceRange) + ConfigHelper.setInputSlicePredicate(job.getConfiguration(), predicate) + + ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + + //Make a new Hadoop RDD + val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), + classOf[ColumnFamilyInputFormat], + classOf[ByteBuffer], + classOf[SortedMap[ByteBuffer, IColumn]]) + + // Let us first get all the paragraphs from the retrieved rows + val paraRdd = casRdd flatMap { + case (key, value) => { + value.filter(v => ByteBufferUtil.string(v._1).compareTo("para") == 0).map(v => ByteBufferUtil.string(v._2.value())) + } + } + + //Lets get the word count in paras + val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) + + counts.collect() foreach { + case(word, count) => println(word + ":" + count) + } + } +} + +/* +create keyspace casDemo; +use casDemo; + +create column family Words with comparator = UTF8Type; +update column family Words with column_metadata = [{column_name: book, validation_class: UTF8Type}, {column_name: para, validation_class: UTF8Type}]; + +assume Words keys as utf8; + +set Words['3musk001']['book'] = 'The Three Musketeers'; +set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market town of + Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to + be in as perfect a state of revolution as if the Huguenots had just made + a second La Rochelle of it. Many citizens, seeing the women flying + toward the High Street, leaving their children crying at the open doors, + hastened to don the cuirass, and supporting their somewhat uncertain + courage with a musket or a partisan, directed their steps toward the + hostelry of the Jolly Miller, before which was gathered, increasing + every minute, a compact group, vociferous and full of curiosity.'; + +set Words['3musk002']['book'] = 'The Three Musketeers'; +set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without some city + or other registering in its archives an event of this kind. There were + nobles, who made war against each other; there was the king, who made + war against the cardinal; there was Spain, which made war against the + king. Then, in addition to these concealed or public, secret or open + wars, there were robbers, mendicants, Huguenots, wolves, and scoundrels, + who made war upon everybody. The citizens always took up arms readily + against thieves, wolves or scoundrels, often against nobles or + Huguenots, sometimes against the king, but never against cardinal or + Spain. It resulted, then, from this habit that on the said first Monday + of April, 1625, the citizens, on hearing the clamor, and seeing neither + the red-and-yellow standard nor the livery of the Duc de Richelieu, + rushed toward the hostel of the Jolly Miller. When arrived there, the + cause of the hubbub was apparent to all'; + +set Words['3musk003']['book'] = 'The Three Musketeers'; +set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however large + the sum may be; but you ought also to endeavor to perfect yourself in + the exercises becoming a gentleman. I will write a letter today to the + Director of the Royal Academy, and tomorrow he will admit you without + any expense to yourself. Do not refuse this little service. Our + best-born and richest gentlemen sometimes solicit it without being able + to obtain it. You will learn horsemanship, swordsmanship in all its + branches, and dancing. You will make some desirable acquaintances; and + from time to time you can call upon me, just to tell me how you are + getting on, and to say whether I can be of further service to you.'; + + +set Words['thelostworld001']['book'] = 'The Lost World'; +set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined against the + red curtain. How beautiful she was! And yet how aloof! We had been + friends, quite good friends; but never could I get beyond the same + comradeship which I might have established with one of my + fellow-reporters upon the Gazette,--perfectly frank, perfectly kindly, + and perfectly unsexual. My instincts are all against a woman being too + frank and at her ease with me. It is no compliment to a man. Where + the real sex feeling begins, timidity and distrust are its companions, + heritage from old wicked days when love and violence went often hand in + hand. The bent head, the averted eye, the faltering voice, the wincing + figure--these, and not the unshrinking gaze and frank reply, are the + true signals of passion. Even in my short life I had learned as much + as that--or had inherited it in that race memory which we call instinct.'; + +set Words['thelostworld002']['book'] = 'The Lost World'; +set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, red-headed news + editor, and I rather hoped that he liked me. Of course, Beaumont was + the real boss; but he lived in the rarefied atmosphere of some Olympian + height from which he could distinguish nothing smaller than an + international crisis or a split in the Cabinet. Sometimes we saw him + passing in lonely majesty to his inner sanctum, with his eyes staring + vaguely and his mind hovering over the Balkans or the Persian Gulf. He + was above and beyond us. But McArdle was his first lieutenant, and it + was he that we knew. The old man nodded as I entered the room, and he + pushed his spectacles far up on his bald forehead.'; + +*/ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0ea23b446fd31..5152b7b79b9d7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -201,8 +201,8 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") - ) + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11", + "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru:1.3") exclude("com.ning","compress-lzf") exclude("io.netty","netty") exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro"))) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From 81c2adc15c9e232846d4ad0adf14d007039409fa Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Sun, 2 Jun 2013 12:51:15 +0530 Subject: [PATCH 617/696] Removing infix call --- examples/src/main/scala/spark/examples/CassandraTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 790b24e6f38b1..49b940d8a730b 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -60,7 +60,7 @@ object CassandraTest { classOf[SortedMap[ByteBuffer, IColumn]]) // Let us first get all the paragraphs from the retrieved rows - val paraRdd = casRdd flatMap { + val paraRdd = casRdd.flatMap { case (key, value) => { value.filter(v => ByteBufferUtil.string(v._1).compareTo("para") == 0).map(v => ByteBufferUtil.string(v._2.value())) } @@ -69,8 +69,8 @@ object CassandraTest { //Lets get the word count in paras val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) - counts.collect() foreach { - case(word, count) => println(word + ":" + count) + counts.collect().foreach { + case (word, count) => println(word + ":" + count) } } } From 6d8423fd1b490d541f0ea379068b8954002d624f Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Sun, 2 Jun 2013 13:03:45 +0530 Subject: [PATCH 618/696] Adding deps to examples/pom.xml Fixing exclusion in examples deps in SparkBuild.scala --- examples/pom.xml | 35 +++++++++++++++++++++++++++++++++++ project/SparkBuild.scala | 2 +- 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/examples/pom.xml b/examples/pom.xml index c42d2bcdb9ed5..b4c5251d6874f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -34,6 +34,41 @@ scalacheck_${scala.version} test + + org.apache.cassandra + cassandra-all + 1.2.5 + + + com.google.guava + guava + + + com.googlecode.concurrentlinkedhashmap + concurrentlinkedhashmap-lru + + + com.ning + compress-lzf + + + io.netty + netty + + + jline + jline + + + log4j + log4j + + + org.apache.cassandra.deps + avro + + + target/scala-${scala.version}/classes diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5152b7b79b9d7..7f3e223c2e7f8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -202,7 +202,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11", - "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru:1.3") exclude("com.ning","compress-lzf") exclude("io.netty","netty") exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro"))) + "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") exclude("com.ning","compress-lzf") exclude("io.netty","netty") exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro"))) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") From 4a9913d66a61ac9ef9cab0e08f6151dc2624fd11 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Sun, 2 Jun 2013 23:21:09 +0000 Subject: [PATCH 619/696] add ut for pipe enhancement --- core/src/test/scala/spark/PipedRDDSuite.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index a6344edf8f161..ee55952a945b5 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -19,6 +19,37 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { assert(c(3) === "4") } + test("advanced pipe") { + sc = new SparkContext("local", "test") + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + + val piped = nums.pipe(Seq("cat"), (i:Int, f: String=> Unit) => f(i + "_"), Array("0")) + + val c = piped.collect() + + assert(c.size === 8) + assert(c(0) === "0") + assert(c(1) === "\u0001") + assert(c(2) === "1_") + assert(c(3) === "2_") + assert(c(4) === "0") + assert(c(5) === "\u0001") + assert(c(6) === "3_") + assert(c(7) === "4_") + + val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) + val d = nums1.groupBy(str=>str.split("\t")(0)).pipe(Seq("cat"), (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}, Array("0")).collect() + assert(d.size === 8) + assert(d(0) === "0") + assert(d(1) === "\u0001") + assert(d(2) === "b\t2_") + assert(d(3) === "b\t4_") + assert(d(4) === "0") + assert(d(5) === "\u0001") + assert(d(6) === "a\t1_") + assert(d(7) === "a\t3_") + } + test("pipe with env variable") { sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) From 606bb1b450064a2b909e4275ce45325dbbef4eca Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 31 May 2013 15:40:41 +0800 Subject: [PATCH 620/696] Fix schedulingAlgorithm bugs for unit test --- .../scheduler/cluster/SchedulingAlgorithm.scala | 17 +++++++++++++---- .../spark/scheduler/ClusterSchedulerSuite.scala | 9 ++++++--- 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index a5d6285c993c4..13120edf63dfd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -40,15 +40,24 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true + var compare:Int = 0 if (s1Needy && !s2Needy) { - res = true + return true } else if (!s1Needy && s2Needy) { - res = false + return false } else if (s1Needy && s2Needy) { - res = minShareRatio1 <= minShareRatio2 + compare = minShareRatio1.compareTo(minShareRatio2) + } else { + compare = taskToWeightRatio1.compareTo(taskToWeightRatio2) + } + + if (compare < 0) { + res = true + } else if (compare > 0) { + res = false } else { - res = taskToWeightRatio1 <= taskToWeightRatio2 + return s1.name < s2.name } return res } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index a39418b71618f..c861597c6b0ee 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -88,7 +88,7 @@ class DummyTask(stageId: Int) extends Task[Int](stageId) } } -class ClusterSchedulerSuite extends FunSuite with LocalSparkContext { +class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging { def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): DummyTaskSetManager = { new DummyTaskSetManager(priority, stage, numTasks, cs , taskSet) @@ -96,8 +96,11 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext { def resourceOffer(rootPool: Pool): Int = { val taskSetQueue = rootPool.getSortedTaskSetQueue() - for (taskSet <- taskSetQueue) - { + /* Just for Test*/ + for (manager <- taskSetQueue) { + logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) + } + for (taskSet <- taskSetQueue) { taskSet.slaveOffer("execId_1", "hostname_1", 1) match { case Some(task) => return taskSet.stageId From 56c64c403383e90a5fd33b6a1f72527377d9bee0 Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Mon, 3 Jun 2013 12:48:35 +0530 Subject: [PATCH 621/696] A better way to read column value if you are sure the column exists in every row. --- examples/src/main/scala/spark/examples/CassandraTest.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 49b940d8a730b..6b9fd502e2095 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -10,6 +10,8 @@ import java.util.SortedMap import org.apache.cassandra.db.IColumn import org.apache.cassandra.utils.ByteBufferUtil import scala.collection.JavaConversions._ +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /* @@ -60,9 +62,9 @@ object CassandraTest { classOf[SortedMap[ByteBuffer, IColumn]]) // Let us first get all the paragraphs from the retrieved rows - val paraRdd = casRdd.flatMap { + val paraRdd = casRdd.map { case (key, value) => { - value.filter(v => ByteBufferUtil.string(v._1).compareTo("para") == 0).map(v => ByteBufferUtil.string(v._2.value())) + ByteBufferUtil.string(value.get(ByteBufferUtil.bytes("para")).value()) } } From b104c7f5c7e2b173fe1b10035efbc00e43df13ec Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Mon, 3 Jun 2013 15:15:52 +0530 Subject: [PATCH 622/696] Example to write the output to cassandra --- .../scala/spark/examples/CassandraTest.scala | 48 +++++++++++++++++-- 1 file changed, 43 insertions(+), 5 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 6b9fd502e2095..2cc62b9fe92a4 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -1,17 +1,16 @@ package spark.examples import org.apache.hadoop.mapreduce.Job -import org.apache.cassandra.hadoop.{ConfigHelper, ColumnFamilyInputFormat} -import org.apache.cassandra.thrift.{IndexExpression, SliceRange, SlicePredicate} +import org.apache.cassandra.hadoop.{ColumnFamilyOutputFormat, ConfigHelper, ColumnFamilyInputFormat} +import org.apache.cassandra.thrift._ import spark.{RDD, SparkContext} -import SparkContext._ +import spark.SparkContext._ import java.nio.ByteBuffer import java.util.SortedMap import org.apache.cassandra.db.IColumn import org.apache.cassandra.utils.ByteBufferUtil import scala.collection.JavaConversions._ -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + /* @@ -44,8 +43,15 @@ object CassandraTest { ConfigHelper.setInputRpcPort(job.getConfiguration(), args(2)) + ConfigHelper.setOutputInitialAddress(job.getConfiguration(), args(1)) + + ConfigHelper.setOutputRpcPort(job.getConfiguration(), args(2)) + ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") + ConfigHelper.setOutputColumnFamily(job.getConfiguration(), "casDemo", "WordCount") + + val predicate = new SlicePredicate() val sliceRange = new SliceRange() sliceRange.setStart(Array.empty[Byte]) @@ -55,6 +61,8 @@ object CassandraTest { ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + //Make a new Hadoop RDD val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), classOf[ColumnFamilyInputFormat], @@ -74,6 +82,33 @@ object CassandraTest { counts.collect().foreach { case (word, count) => println(word + ":" + count) } + + counts.map { + case (word, count) => { + val colWord = new org.apache.cassandra.thrift.Column() + colWord.setName(ByteBufferUtil.bytes("word")) + colWord.setValue(ByteBufferUtil.bytes(word)) + colWord.setTimestamp(System.currentTimeMillis) + + val colCount = new org.apache.cassandra.thrift.Column() + colCount.setName(ByteBufferUtil.bytes("wcount")) + colCount.setValue(ByteBufferUtil.bytes(count.toLong)) + colCount.setTimestamp(System.currentTimeMillis) + + + val outputkey = ByteBufferUtil.bytes(word + "-COUNT-" + System.currentTimeMillis) + + val mutations: java.util.List[Mutation] = new Mutation() :: new Mutation() :: Nil + mutations.get(0).setColumn_or_supercolumn(new ColumnOrSuperColumn()) + mutations.get(0).column_or_supercolumn.setColumn(colWord) + + mutations.get(1).setColumn_or_supercolumn(new ColumnOrSuperColumn()) + mutations.get(1).column_or_supercolumn.setColumn(colCount) + (outputkey, mutations) + } + }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], + classOf[ColumnFamilyOutputFormat], job.getConfiguration) + } } @@ -81,6 +116,9 @@ object CassandraTest { create keyspace casDemo; use casDemo; +create column family WordCount with comparator = UTF8Type; +update column family WordCount with column_metadata = [{column_name: word, validation_class: UTF8Type}, {column_name: wcount, validation_class: LongType}]; + create column family Words with comparator = UTF8Type; update column family Words with column_metadata = [{column_name: book, validation_class: UTF8Type}, {column_name: para, validation_class: UTF8Type}]; From d1286231e0db15e480bd7d6a600b419db3391b27 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 29 May 2013 20:14:59 -0700 Subject: [PATCH 623/696] Sometime Maven build runs out of PermGen space. --- pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pom.xml b/pom.xml index 6ee64d07c26ec..ce77ba37c6598 100644 --- a/pom.xml +++ b/pom.xml @@ -59,6 +59,9 @@ 1.6.1 4.1.2 1.2.17 + + 0m + 512m @@ -392,6 +395,10 @@ -Xms64m -Xmx1024m + -XX:PermSize + ${PermGen} + -XX:MaxPermSize + ${MaxPermGen} -source From 8bd4e1210422d9985e6105fd9529e813fe45c14e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 4 Jun 2013 18:14:24 -0400 Subject: [PATCH 624/696] Bump akka and blockmanager timeouts to 60 seconds --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- core/src/main/scala/spark/util/AkkaUtils.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 09572b19db2db..150c98f57c811 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -950,7 +950,7 @@ object BlockManager extends Logging { } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerHeartBeatMs", "5000").toLong + System.getProperty("spark.storage.blockManagerHeartBeatMs", "60000").toLong def getDisableHeartBeatsForTesting: Boolean = System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 9fb7e001badcb..def993236ba9b 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -29,7 +29,7 @@ private[spark] object AkkaUtils { def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt + val akkaTimeout = System.getProperty("spark.akka.timeout", "60").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" // 10 seconds is the default akka timeout, but in a cluster, we need higher by default. From 061fd3ae369e744f076e21044de26a00982a408f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 4 Jun 2013 18:50:15 -0400 Subject: [PATCH 625/696] Fixing bug in BlockManager timeout --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- core/src/main/scala/spark/storage/BlockManagerMasterActor.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 150c98f57c811..65c789ea8fb2a 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -950,7 +950,7 @@ object BlockManager extends Logging { } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerHeartBeatMs", "60000").toLong + System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting: Boolean = System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 9b64f95df8073..0dcb9fb2ac16b 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -35,7 +35,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", - "5000").toLong + "60000").toLong var timeoutCheckingTask: Cancellable = null From 9d359043574f6801ba15ec9d016eba0f00ac2349 Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Tue, 4 Jun 2013 22:12:47 -0700 Subject: [PATCH 626/696] In the current code, when both partitions happen to have zero-length, the return mean will be NaN. Consequently, the result of mean after reducing over all partitions will also be NaN, which is not correct if there are partitions with non-zero length. This patch fixes this issue. --- .../main/scala/spark/util/StatCounter.scala | 26 ++++++++++++------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala index 5f80180339d35..2b980340b7367 100644 --- a/core/src/main/scala/spark/util/StatCounter.scala +++ b/core/src/main/scala/spark/util/StatCounter.scala @@ -37,17 +37,23 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { if (other == this) { merge(other.copy()) // Avoid overwriting fields in a weird order } else { - val delta = other.mu - mu - if (other.n * 10 < n) { - mu = mu + (delta * other.n) / (n + other.n) - } else if (n * 10 < other.n) { - mu = other.mu - (delta * n) / (n + other.n) - } else { - mu = (mu * n + other.mu * other.n) / (n + other.n) + if (n == 0) { + mu = other.mu + m2 = other.m2 + n = other.n + } else if (other.n != 0) { + val delta = other.mu - mu + if (other.n * 10 < n) { + mu = mu + (delta * other.n) / (n + other.n) + } else if (n * 10 < other.n) { + mu = other.mu - (delta * n) / (n + other.n) + } else { + mu = (mu * n + other.mu * other.n) / (n + other.n) + } + m2 += other.m2 + (delta * delta * n * other.n) / (n + other.n) + n += other.n } - m2 += other.m2 + (delta * delta * n * other.n) / (n + other.n) - n += other.n - this + this } } From e179ff8a32fc08cc308dc99bac2527d350d0d970 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 5 Jun 2013 22:41:05 +0000 Subject: [PATCH 627/696] update according to comments --- core/src/main/scala/spark/RDD.scala | 89 ++++++++++++++++--- core/src/main/scala/spark/rdd/PipedRDD.scala | 33 +++---- core/src/test/scala/spark/PipedRDDSuite.scala | 7 +- 3 files changed, 99 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 5a41db23c20a4..a1c9604324631 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -16,6 +16,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import spark.broadcast.Broadcast import spark.Partitioner._ import spark.partial.BoundedDouble import spark.partial.CountEvaluator @@ -351,31 +352,93 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = - new PipedRDD(this, command, transform, arguments) + def pipe(command: String, env: Map[String, String]): RDD[String] = + new PipedRDD(this, command, env) /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String]): RDD[String] = new PipedRDD(this, command) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: String, + env: Map[String, String], + transform: (T,String => Unit) => Any, + pipeContext: Broadcast[U], + delimiter: String): RDD[String] = + new PipedRDD(this, command, env, transform, pipeContext, delimiter) /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = - new PipedRDD(this, command, transform, arguments) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: String, + transform: (T,String => Unit) => Any, + pipeContext: Broadcast[U]): RDD[String] = + new PipedRDD(this, command, Map[String, String](), transform, pipeContext, "\u0001") /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String], env: Map[String, String]): RDD[String] = - new PipedRDD(this, command, env) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: String, + env: Map[String, String], + transform: (T,String => Unit) => Any, + pipeContext: Broadcast[U]): RDD[String] = + new PipedRDD(this, command, env, transform, pipeContext, "\u0001") /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String], env: Map[String, String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = - new PipedRDD(this, command, env, transform, arguments) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: Seq[String], + env: Map[String, String] = Map(), + transform: (T,String => Unit) => Any = null, + pipeContext: Broadcast[U] = null, + delimiter: String = "\u0001"): RDD[String] = + new PipedRDD(this, command, env, transform, pipeContext, delimiter) /** * Return a new RDD by applying a function to each partition of this RDD. diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 969404c95f64f..d58aaae709ab9 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -9,29 +9,33 @@ import scala.collection.mutable.ArrayBuffer import scala.io.Source import spark.{RDD, SparkEnv, Partition, TaskContext} +import spark.broadcast.Broadcast /** * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassManifest]( +class PipedRDD[T: ClassManifest, U <: Seq[String]]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], transform: (T, String => Unit) => Any, - arguments: Seq[String] + pipeContext: Broadcast[U], + delimiter: String ) extends RDD[String](prev) { - def this(prev: RDD[T], command: Seq[String], envVars : Map[String, String]) = this(prev, command, envVars, null, null) - def this(prev: RDD[T], command: Seq[String]) = this(prev, command, Map()) - def this(prev: RDD[T], command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, command, Map(), transform, arguments) - // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) - def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command)) - def this(prev: RDD[T], command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, PipedRDD.tokenize(command), Map(), transform, arguments) + def this( + prev: RDD[T], + command: String, + envVars: Map[String, String] = Map(), + transform: (T, String => Unit) => Any = null, + pipeContext: Broadcast[U] = null, + delimiter: String = "\u0001") = + this(prev, PipedRDD.tokenize(command), envVars, transform, pipeContext, delimiter) override def getPartitions: Array[Partition] = firstParent[T].partitions @@ -60,19 +64,18 @@ class PipedRDD[T: ClassManifest]( SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) - // input the arguments firstly - if ( arguments != null) { - for (elem <- arguments) { + // input the pipeContext firstly + if ( pipeContext != null) { + for (elem <- pipeContext.value) { out.println(elem) } - // ^A \n as the marker of the end of the arguments - out.println("\u0001") + // delimiter\n as the marker of the end of the pipeContext + out.println(delimiter) } for (elem <- firstParent[T].iterator(split, context)) { if (transform != null) { transform(elem, out.println(_)) - } - else { + } else { out.println(elem) } } diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index ee55952a945b5..d2852867de753 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -23,7 +23,8 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("cat"), (i:Int, f: String=> Unit) => f(i + "_"), Array("0")) + val piped = nums.pipe(Seq("cat"), Map[String, String](), + (i:Int, f: String=> Unit) => f(i + "_"), sc.broadcast(List("0"))) val c = piped.collect() @@ -38,7 +39,9 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { assert(c(7) === "4_") val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) - val d = nums1.groupBy(str=>str.split("\t")(0)).pipe(Seq("cat"), (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}, Array("0")).collect() + val d = nums1.groupBy(str=>str.split("\t")(0)). + pipe(Seq("cat"), Map[String, String](), (i:Tuple2[String, Seq[String]], f: String=> Unit) => + {for (e <- i._2){ f(e + "_")}}, sc.broadcast(List("0"))).collect() assert(d.size === 8) assert(d(0) === "0") assert(d(1) === "\u0001") From c9ca0a4a588b4c7dc553b155336ae5b95aa9ddd4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 7 Jun 2013 22:40:44 -0700 Subject: [PATCH 628/696] Small code style fix to SchedulingAlgorithm.scala --- .../scala/spark/scheduler/cluster/SchedulingAlgorithm.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index 13120edf63dfd..e071917c00c41 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -53,13 +53,12 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { } if (compare < 0) { - res = true + return true } else if (compare > 0) { - res = false + return false } else { return s1.name < s2.name } - return res } } From b58a29295b2e610cadf1cac44438337ce9b51537 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 7 Jun 2013 22:51:28 -0700 Subject: [PATCH 629/696] Small formatting and style fixes --- .../scheduler/cluster/SchedulingAlgorithm.scala | 8 ++++---- .../main/scala/spark/storage/StorageUtils.scala | 14 +++++++------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index e071917c00c41..f33310a34a29d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -13,11 +13,11 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { val priority1 = s1.priority val priority2 = s2.priority - var res = Math.signum(priority1 - priority2) + var res = math.signum(priority1 - priority2) if (res == 0) { val stageId1 = s1.stageId val stageId2 = s2.stageId - res = Math.signum(stageId1 - stageId2) + res = math.signum(stageId1 - stageId2) } if (res < 0) { return true @@ -35,8 +35,8 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1, 1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2, 1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0).toDouble + val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0).toDouble val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 81e607868d558..950c0cdf352f7 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -55,21 +55,21 @@ object StorageUtils { }.mapValues(_.values.toArray) // For each RDD, generate an RDDInfo object - val rddInfos = groupedRddBlocks.map { case(rddKey, rddBlocks) => - + val rddInfos = groupedRddBlocks.map { case (rddKey, rddBlocks) => // Add up memory and disk sizes val memSize = rddBlocks.map(_.memSize).reduce(_ + _) val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _) // Find the id of the RDD, e.g. rdd_1 => 1 val rddId = rddKey.split("_").last.toInt - // Get the friendly name for the rdd, if available. + + // Get the friendly name and storage level for the RDD, if available sc.persistentRdds.get(rddId).map { r => - val rddName = Option(r.name).getOrElse(rddKey) - val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) + val rddName = Option(r.name).getOrElse(rddKey) + val rddStorageLevel = r.getStorageLevel + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) } - }.flatMap(x => x).toArray + }.flatten.toArray scala.util.Sorting.quickSort(rddInfos) From 1a4d93c025e5d3679257a622f49dfaade4ac18c2 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Sat, 8 Jun 2013 14:23:39 +0800 Subject: [PATCH 630/696] modify to pass job annotation by localProperties and use daeamon thread to do joblogger's work --- .../spark/BlockStoreShuffleFetcher.scala | 1 + core/src/main/scala/spark/RDD.scala | 10 +- core/src/main/scala/spark/SparkContext.scala | 8 +- core/src/main/scala/spark/Utils.scala | 10 +- .../main/scala/spark/executor/Executor.scala | 1 + .../scala/spark/executor/TaskMetrics.scala | 12 +- .../scala/spark/scheduler/DAGScheduler.scala | 8 + .../scala/spark/scheduler/JobLogger.scala | 317 ++++++++++++++++++ .../scala/spark/scheduler/SparkListener.scala | 33 +- .../spark/scheduler/JobLoggerSuite.scala | 105 ++++++ .../spark/scheduler/SparkListenerSuite.scala | 2 +- 11 files changed, 495 insertions(+), 12 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/JobLogger.scala create mode 100644 core/src/test/scala/spark/scheduler/JobLoggerSuite.scala diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index e1fb02157aafe..3239f4c3854a9 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -58,6 +58,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin CompletionIterator[(K,V), Iterator[(K,V)]](itr, { val shuffleMetrics = new ShuffleReadMetrics + shuffleMetrics.shuffleFinishTime = System.currentTimeMillis shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6c0438d76da6..8c0b7ca417add 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -114,6 +114,14 @@ abstract class RDD[T: ClassManifest]( this } + /**User-defined generator of this RDD*/ + var generator = Utils.getCallSiteInfo._4 + + /**reset generator*/ + def setGenerator(_generator: String) = { + generator = _generator + } + /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. This can only be used to assign a new storage level if the RDD does not @@ -788,7 +796,7 @@ abstract class RDD[T: ClassManifest]( private var storageLevel: StorageLevel = StorageLevel.NONE /** Record user function generating this RDD. */ - private[spark] val origin = Utils.getSparkCallSite + private[spark] val origin = Utils.formatSparkCallSite private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index bc05d08fd6ce9..b67a2066c8dd7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -48,7 +48,7 @@ import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} - +import spark.scheduler.JobLogger /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -510,7 +510,7 @@ class SparkContext( def addSparkListener(listener: SparkListener) { dagScheduler.sparkListeners += listener } - + addSparkListener(new JobLogger) /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -630,7 +630,7 @@ class SparkContext( partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - val callSite = Utils.getSparkCallSite + val callSite = Utils.formatSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler, localProperties.value) @@ -713,7 +713,7 @@ class SparkContext( func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R] = { - val callSite = Utils.getSparkCallSite + val callSite = Utils.formatSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.value) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index ec15326014e8d..1630b2b4b01e7 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -528,7 +528,7 @@ private object Utils extends Logging { * (outside the spark package) that called into Spark, as well as which Spark method they called. * This is used, for example, to tell users where in their code each RDD got created. */ - def getSparkCallSite: String = { + def getCallSiteInfo = { val trace = Thread.currentThread.getStackTrace().filter( el => (!el.getMethodName.contains("getStackTrace"))) @@ -540,6 +540,7 @@ private object Utils extends Logging { var firstUserFile = "" var firstUserLine = 0 var finished = false + var firstUserClass = "" for (el <- trace) { if (!finished) { @@ -554,13 +555,18 @@ private object Utils extends Logging { else { firstUserLine = el.getLineNumber firstUserFile = el.getFileName + firstUserClass = el.getClassName finished = true } } } - "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine) + (lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) } + def formatSparkCallSite = { + val callSiteInfo = getCallSiteInfo + "%s at %s:%s".format(callSiteInfo._1, callSiteInfo._2, callSiteInfo._3) + } /** * Try to find a free port to bind to on the local host. This should ideally never be needed, * except that, unfortunately, some of the networking libraries we currently rely on (e.g. Spray) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 890938d48b667..8bebfafce4421 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -104,6 +104,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val value = task.run(taskId.toInt) val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => + m.hostname = Utils.localHostName m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index a7c56c237199b..26e80293657c4 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,6 +1,11 @@ package spark.executor class TaskMetrics extends Serializable { + /** + * host's name the task runs on + */ + var hostname: String = _ + /** * Time taken on the executor to deserialize this task */ @@ -33,10 +38,15 @@ object TaskMetrics { class ShuffleReadMetrics extends Serializable { + /** + * Time when shuffle finishs + */ + var shuffleFinishTime: Long = _ + /** * Total number of blocks fetched in a shuffle (remote or local) */ - var totalBlocksFetched : Int = _ + var totalBlocksFetched: Int = _ /** * Number of remote blocks fetched in a shuffle diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 7feeb9754289c..43dd7d6534cf4 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -289,6 +289,7 @@ class DAGScheduler( val finalStage = newStage(finalRDD, None, runId) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() + sparkListeners.foreach(_.onJobStart(job, properties)) logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") @@ -311,6 +312,7 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => + sparkListeners.foreach(_.onTaskEnd(completion)) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -321,6 +323,8 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) + val JobCancelEvent = new SparkListenerJobCancelled("SPARKCONTEXT_SHUTDOWN") + sparkListeners.foreach(_.onJobEnd(job, JobCancelEvent)) } return true } @@ -468,6 +472,7 @@ class DAGScheduler( } } if (tasks.size > 0) { + sparkListeners.foreach(_.onStageSubmitted(stage, "TASKS_SIZE=" + tasks.size)) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) @@ -522,6 +527,7 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) + sparkListeners.foreach(_.onJobEnd(job, SparkListenerJobSuccess)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -665,6 +671,8 @@ class DAGScheduler( job.listener.jobFailed(new SparkException("Job failed: " + reason)) activeJobs -= job resultStageToJob -= resultStage + val jobFailedEvent = new SparkListenerJobFailed(failedStage) + sparkListeners.foreach(_.onJobEnd(job, jobFailedEvent)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala new file mode 100644 index 0000000000000..f87acfd0b61c0 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -0,0 +1,317 @@ +package spark.scheduler + +import java.io.PrintWriter +import java.io.File +import java.io.FileNotFoundException +import java.text.SimpleDateFormat +import java.util.{Date, Properties} +import java.util.concurrent.LinkedBlockingQueue +import scala.collection.mutable.{Map, HashMap, ListBuffer} +import scala.io.Source +import spark._ +import spark.executor.TaskMetrics +import spark.scheduler.cluster.TaskInfo + +// used to record runtime information for each job, including RDD graph +// tasks' start/stop shuffle information and information from outside + +sealed trait JobLoggerEvent +case class JobLoggerOnJobStart(job: ActiveJob, properties: Properties) extends JobLoggerEvent +case class JobLoggerOnStageSubmitted(stage: Stage, info: String) extends JobLoggerEvent +case class JobLoggerOnStageCompleted(stageCompleted: StageCompleted) extends JobLoggerEvent +case class JobLoggerOnJobEnd(job: ActiveJob, event: SparkListenerEvents) extends JobLoggerEvent +case class JobLoggerOnTaskEnd(event: CompletionEvent) extends JobLoggerEvent + +class JobLogger(val logDirName: String) extends SparkListener with Logging { + private val logDir = + if (System.getenv("SPARK_LOG_DIR") != null) + System.getenv("SPARK_LOG_DIR") + else + "/tmp/spark" + private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] + private val stageIDToJobID = new HashMap[Int, Int] + private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]] + private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val eventQueue = new LinkedBlockingQueue[JobLoggerEvent] + + createLogDir() + def this() = this(String.valueOf(System.currentTimeMillis())) + + def getLogDir = logDir + def getJobIDtoPrintWriter = jobIDToPrintWriter + def getStageIDToJobID = stageIDToJobID + def getJobIDToStages = jobIDToStages + def getEventQueue = eventQueue + + new Thread("JobLogger") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event != null) { + logDebug("Got event of type " + event.getClass.getName) + event match { + case JobLoggerOnJobStart(job, info) => + processJobStartEvent(job, info) + case JobLoggerOnStageSubmitted(stage, info) => + processStageSubmittedEvent(stage, info) + case JobLoggerOnStageCompleted(stageCompleted) => + processStageCompletedEvent(stageCompleted) + case JobLoggerOnJobEnd(job, event) => + processJobEndEvent(job, event) + case JobLoggerOnTaskEnd(event) => + processTaskEndEvent(event) + case _ => + } + } + } + } + }.start() + + //create a folder for log files, the folder's name is the creation time of the jobLogger + protected def createLogDir() { + val dir = new File(logDir + "/" + logDirName + "/") + if (dir.exists()) { + return + } + if (dir.mkdirs() == false) { + logError("create log directory error:" + logDir + "/" + logDirName + "/") + } + } + + // create a log file for one job, the file name is the jobID + protected def createLogWriter(jobID: Int) { + try{ + val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) + jobIDToPrintWriter += (jobID -> fileWriter) + } catch { + case e: FileNotFoundException => e.printStackTrace() + } + } + + // close log file for one job, and clean the stage relationship in stageIDToJobID + protected def closeLogWriter(jobID: Int) = + jobIDToPrintWriter.get(jobID).foreach { fileWriter => + fileWriter.close() + jobIDToStages.get(jobID).foreach(_.foreach{ stage => + stageIDToJobID -= stage.id + }) + jobIDToPrintWriter -= jobID + jobIDToStages -= jobID + } + + // write log information to log file, withTime parameter controls whether to recored + // time stamp for the information + protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { + var writeInfo = info + if (withTime) { + val date = new Date(System.currentTimeMillis()) + writeInfo = DATE_FORMAT.format(date) + ": " +info + } + jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) + } + + protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) = + stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) + + protected def buildJobDep(jobID: Int, stage: Stage) { + if (stage.priority == jobID) { + jobIDToStages.get(jobID) match { + case Some(stageList) => stageList += stage + case None => val stageList = new ListBuffer[Stage] + stageList += stage + jobIDToStages += (jobID -> stageList) + } + stageIDToJobID += (stage.id -> jobID) + stage.parents.foreach(buildJobDep(jobID, _)) + } + } + + protected def recordStageDep(jobID: Int) { + def getRddsInStage(rdd: RDD[_]): ListBuffer[RDD[_]] = { + var rddList = new ListBuffer[RDD[_]] + rddList += rdd + rdd.dependencies.foreach{ dep => dep match { + case shufDep: ShuffleDependency[_,_] => + case _ => rddList ++= getRddsInStage(dep.rdd) + } + } + rddList + } + jobIDToStages.get(jobID).foreach {_.foreach { stage => + var depRddDesc: String = "" + getRddsInStage(stage.rdd).foreach { rdd => + depRddDesc += rdd.id + "," + } + var depStageDesc: String = "" + stage.parents.foreach { stage => + depStageDesc += "(" + stage.id + "," + stage.shuffleDep.get.shuffleId + ")" + } + jobLogInfo(jobID, "STAGE_ID=" + stage.id + " RDD_DEP=(" + + depRddDesc.substring(0, depRddDesc.length - 1) + ")" + + " STAGE_DEP=" + depStageDesc, false) + } + } + } + + // generate indents and convert to String + protected def indentString(indent: Int) = { + val sb = new StringBuilder() + for (i <- 1 to indent) { + sb.append(" ") + } + sb.toString() + } + + protected def getRddName(rdd: RDD[_]) = { + var rddName = rdd.getClass.getName + if (rdd.name != null) { + rddName = rdd.name + } + rddName + } + + protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) { + val rddInfo = "RDD_ID=" + rdd.id + "(" + getRddName(rdd) + "," + rdd.generator + ")" + jobLogInfo(jobID, indentString(indent) + rddInfo, false) + rdd.dependencies.foreach{ dep => dep match { + case shufDep: ShuffleDependency[_,_] => + val depInfo = "SHUFFLE_ID=" + shufDep.shuffleId + jobLogInfo(jobID, indentString(indent + 1) + depInfo, false) + case _ => recordRddInStageGraph(jobID, dep.rdd, indent + 1) + } + } + } + + protected def recordStageDepGraph(jobID: Int, stage: Stage, indent: Int = 0) { + var stageInfo: String = "" + if (stage.isShuffleMap) { + stageInfo = "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + + stage.shuffleDep.get.shuffleId + }else{ + stageInfo = "STAGE_ID=" + stage.id + " RESULT_STAGE" + } + if (stage.priority == jobID) { + jobLogInfo(jobID, indentString(indent) + stageInfo, false) + recordRddInStageGraph(jobID, stage.rdd, indent) + stage.parents.foreach(recordStageDepGraph(jobID, _, indent + 2)) + } else + jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.priority, false) + } + + // record task metrics into job log files + protected def recordTaskMetrics(stageID: Int, status: String, + taskInfo: TaskInfo, taskMetrics: TaskMetrics) { + val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + + " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname + val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime + val readMetrics = + taskMetrics.shuffleReadMetrics match { + case Some(metrics) => + " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime + + " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + + " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + + " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + + " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime + + " REMOTE_FETCH_TIME=" + metrics.remoteFetchTime + + " REMOTE_BYTES_READ=" + metrics.remoteBytesRead + case None => "" + } + val writeMetrics = + taskMetrics.shuffleWriteMetrics match { + case Some(metrics) => + " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten + case None => "" + } + stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) + } + + override def onStageSubmitted(stage: Stage, info: String = "") { + eventQueue.put(JobLoggerOnStageSubmitted(stage, info)) + } + + protected def processStageSubmittedEvent(stage: Stage, info: String) { + stageLogInfo(stage.id, "STAGE_ID=" + stage.id + " STATUS=SUBMITTED " + info) + } + + override def onStageCompleted(stageCompleted: StageCompleted) { + eventQueue.put(JobLoggerOnStageCompleted(stageCompleted)) + } + + protected def processStageCompletedEvent(stageCompleted: StageCompleted) { + stageLogInfo(stageCompleted.stageInfo.stage.id, "STAGE_ID=" + + stageCompleted.stageInfo.stage.id + " STATUS=COMPLETED") + + } + + override def onTaskEnd(event: CompletionEvent) { + eventQueue.put(JobLoggerOnTaskEnd(event)) + } + + protected def processTaskEndEvent(event: CompletionEvent) { + var taskStatus = "" + event.task match { + case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" + case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" + } + event.reason match { + case Success => taskStatus += " STATUS=SUCCESS" + recordTaskMetrics(event.task.stageId, taskStatus, event.taskInfo, event.taskMetrics) + case Resubmitted => + taskStatus += " STATUS=RESUBMITTED TID=" + event.taskInfo.taskId + + " STAGE_ID=" + event.task.stageId + stageLogInfo(event.task.stageId, taskStatus) + case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => + taskStatus += " STATUS=FETCHFAILED TID=" + event.taskInfo.taskId + " STAGE_ID=" + + event.task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + + mapId + " REDUCE_ID=" + reduceId + stageLogInfo(event.task.stageId, taskStatus) + case OtherFailure(message) => + taskStatus += " STATUS=FAILURE TID=" + event.taskInfo.taskId + + " STAGE_ID=" + event.task.stageId + " INFO=" + message + stageLogInfo(event.task.stageId, taskStatus) + case _ => + } + } + + override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { + eventQueue.put(JobLoggerOnJobEnd(job, event)) + } + + protected def processJobEndEvent(job: ActiveJob, event: SparkListenerEvents) { + var info = "JOB_ID=" + job.runId + " STATUS=" + var validEvent = true + event match { + case SparkListenerJobSuccess => info += "SUCCESS" + case SparkListenerJobFailed(failedStage) => + info += "FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" + failedStage.id + case SparkListenerJobCancelled(reason) => info += "CANCELLED REASON=" + reason + case _ => validEvent = false + } + if (validEvent) { + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) + } + } + + protected def recordJobProperties(jobID: Int, properties: Properties) { + if(properties != null) { + val annotation = properties.getProperty("spark.job.annotation", "") + jobLogInfo(jobID, annotation, false) + } + } + + override def onJobStart(job: ActiveJob, properties: Properties = null) { + eventQueue.put(JobLoggerOnJobStart(job, properties)) + } + + protected def processJobStartEvent(job: ActiveJob, properties: Properties) { + createLogWriter(job.runId) + recordJobProperties(job.runId, properties) + buildJobDep(job.runId, job.finalStage) + recordStageDep(job.runId) + recordStageDepGraph(job.runId, job.finalStage) + jobLogInfo(job.runId, "JOB_ID=" + job.runId + " STATUS=STARTED") + } +} diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index a65140b145833..9cf7f3ffc0ea0 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,27 +1,54 @@ package spark.scheduler +import java.util.Properties import spark.scheduler.cluster.TaskInfo import spark.util.Distribution -import spark.{Utils, Logging} +import spark.{Utils, Logging, SparkContext, TaskEndReason} import spark.executor.TaskMetrics trait SparkListener { /** * called when a stage is completed, with information on the completed stage */ - def onStageCompleted(stageCompleted: StageCompleted) + def onStageCompleted(stageCompleted: StageCompleted) { } + + /** + * called when a stage is submitted + */ + def onStageSubmitted(stage: Stage, info: String = "") { } + + /** + * called when a task ends + */ + def onTaskEnd(event: CompletionEvent) { } + + /** + * called when a job starts + */ + def onJobStart(job: ActiveJob, properties: Properties = null) { } + + /** + * called when a job ends + */ + def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { } + } sealed trait SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents +case object SparkListenerJobSuccess extends SparkListenerEvents + +case class SparkListenerJobFailed(failedStage: Stage) extends SparkListenerEvents + +case class SparkListenerJobCancelled(reason: String) extends SparkListenerEvents /** * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { - def onStageCompleted(stageCompleted: StageCompleted) { + override def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stageInfo) diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala new file mode 100644 index 0000000000000..34fd8b995e141 --- /dev/null +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -0,0 +1,105 @@ +package spark.scheduler + +import java.util.Properties +import java.util.concurrent.LinkedBlockingQueue +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers +import scala.collection.mutable +import spark._ +import spark.SparkContext._ + + +class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { + + test("inner method") { + sc = new SparkContext("local", "joblogger") + val joblogger = new JobLogger { + def createLogWriterTest(jobID: Int) = createLogWriter(jobID) + def closeLogWriterTest(jobID: Int) = closeLogWriter(jobID) + def getRddNameTest(rdd: RDD[_]) = getRddName(rdd) + def buildJobDepTest(jobID: Int, stage: Stage) = buildJobDep(jobID, stage) + } + type MyRDD = RDD[(Int, Int)] + def makeRdd( + numPartitions: Int, + dependencies: List[Dependency[_]] + ): MyRDD = { + val maxPartition = numPartitions - 1 + return new MyRDD(sc, dependencies) { + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getPartitions = (0 to maxPartition).map(i => new Partition { + override def index = i + }).toArray + } + } + val jobID = 5 + val parentRdd = makeRdd(4, Nil) + val shuffleDep = new ShuffleDependency(parentRdd, null) + val rootRdd = makeRdd(4, List(shuffleDep)) + val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) + val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) + + joblogger.onStageSubmitted(rootStage) + joblogger.getEventQueue.size should be (1) + joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) + parentRdd.setName("MyRDD") + joblogger.getRddNameTest(parentRdd) should be ("MyRDD") + joblogger.createLogWriterTest(jobID) + joblogger.getJobIDtoPrintWriter.size should be (1) + joblogger.buildJobDepTest(jobID, rootStage) + joblogger.getJobIDToStages.get(jobID).get.size should be (2) + joblogger.getStageIDToJobID.get(0) should be (Some(jobID)) + joblogger.getStageIDToJobID.get(1) should be (Some(jobID)) + joblogger.closeLogWriterTest(jobID) + joblogger.getStageIDToJobID.size should be (0) + joblogger.getJobIDToStages.size should be (0) + joblogger.getJobIDtoPrintWriter.size should be (0) + } + + test("inner variables") { + sc = new SparkContext("local[4]", "joblogger") + val joblogger = new JobLogger { + override protected def closeLogWriter(jobID: Int) = + getJobIDtoPrintWriter.get(jobID).foreach { fileWriter => + fileWriter.close() + } + } + sc.addSparkListener(joblogger) + val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } + rdd.reduceByKey(_+_).collect() + + joblogger.getLogDir should be ("/tmp/spark") + joblogger.getJobIDtoPrintWriter.size should be (1) + joblogger.getStageIDToJobID.size should be (2) + joblogger.getStageIDToJobID.get(0) should be (Some(0)) + joblogger.getStageIDToJobID.get(1) should be (Some(0)) + joblogger.getJobIDToStages.size should be (1) + } + + + test("interface functions") { + sc = new SparkContext("local[4]", "joblogger") + val joblogger = new JobLogger { + var onTaskEndCount = 0 + var onJobEndCount = 0 + var onJobStartCount = 0 + var onStageCompletedCount = 0 + var onStageSubmittedCount = 0 + override def onTaskEnd(event: CompletionEvent) = onTaskEndCount += 1 + override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) = onJobEndCount += 1 + override def onJobStart(job: ActiveJob, properties: Properties) = onJobStartCount += 1 + override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 + override def onStageSubmitted(stage: Stage, info: String = "") = onStageSubmittedCount += 1 + } + sc.addSparkListener(joblogger) + val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } + rdd.reduceByKey(_+_).collect() + + joblogger.onJobStartCount should be (1) + joblogger.onJobEndCount should be (1) + joblogger.onTaskEndCount should be (8) + joblogger.onStageSubmittedCount should be (2) + joblogger.onStageCompletedCount should be (2) + } +} diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index 42a87d8b90fe5..48aa67c543cf3 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -77,7 +77,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc class SaveStageInfo extends SparkListener { val stageInfos = mutable.Buffer[StageInfo]() - def onStageCompleted(stage: StageCompleted) { + override def onStageCompleted(stage: StageCompleted) { stageInfos += stage.stageInfo } } From 4fd86e0e10149ad1803831a308a056c7105cbe67 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Sat, 8 Jun 2013 15:45:47 +0800 Subject: [PATCH 631/696] delete test code for joblogger in SparkContext --- core/src/main/scala/spark/SparkContext.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b67a2066c8dd7..70a9d7698c490 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -48,7 +48,6 @@ import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.scheduler.JobLogger /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -510,7 +509,7 @@ class SparkContext( def addSparkListener(listener: SparkListener) { dagScheduler.sparkListeners += listener } - addSparkListener(new JobLogger) + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. From ade822011d44bd43e9ac78c1d29ec924a1f6e8e7 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Sat, 8 Jun 2013 16:26:45 +0800 Subject: [PATCH 632/696] not check return value of eventQueue.take --- .../scala/spark/scheduler/JobLogger.scala | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index f87acfd0b61c0..46b9fa974b10b 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -48,21 +48,19 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { override def run() { while (true) { val event = eventQueue.take - if (event != null) { - logDebug("Got event of type " + event.getClass.getName) - event match { - case JobLoggerOnJobStart(job, info) => - processJobStartEvent(job, info) - case JobLoggerOnStageSubmitted(stage, info) => - processStageSubmittedEvent(stage, info) - case JobLoggerOnStageCompleted(stageCompleted) => - processStageCompletedEvent(stageCompleted) - case JobLoggerOnJobEnd(job, event) => - processJobEndEvent(job, event) - case JobLoggerOnTaskEnd(event) => - processTaskEndEvent(event) - case _ => - } + logDebug("Got event of type " + event.getClass.getName) + event match { + case JobLoggerOnJobStart(job, info) => + processJobStartEvent(job, info) + case JobLoggerOnStageSubmitted(stage, info) => + processStageSubmittedEvent(stage, info) + case JobLoggerOnStageCompleted(stageCompleted) => + processStageCompletedEvent(stageCompleted) + case JobLoggerOnJobEnd(job, event) => + processJobEndEvent(job, event) + case JobLoggerOnTaskEnd(event) => + processTaskEndEvent(event) + case _ => } } } From d1bbcebae580220076ceaa65f84dcf984ab51a16 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 8 Jun 2013 16:58:42 -0700 Subject: [PATCH 633/696] Adding compression to Hadoop save functions --- .../main/scala/spark/PairRDDFunctions.scala | 39 ++++++++++++++- core/src/main/scala/spark/RDD.scala | 9 ++++ .../spark/SequenceFileRDDFunctions.scala | 15 ++++-- core/src/test/scala/spark/FileSuite.scala | 48 +++++++++++++++++++ 4 files changed, 105 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 2b0e697337610..9bf1227d6519e 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -10,6 +10,8 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.mapred.FileOutputCommitter import org.apache.hadoop.mapred.FileOutputFormat import org.apache.hadoop.mapred.HadoopWriter @@ -515,6 +517,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. Compress the result with the + * supplied codec. + */ + def saveAsHadoopFile[F <: OutputFormat[K, V]]( + path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) { + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) + } + /** * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. @@ -574,6 +586,20 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( jobCommitter.cleanupJob(jobTaskContext) } + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. Compress with the supplied codec. + */ + def saveAsHadoopFile( + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[_ <: OutputFormat[_, _]], + codec: Class[_ <: CompressionCodec]) { + saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, + new JobConf(self.context.hadoopConfiguration), Some(codec)) + } + /** * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class * supporting the key and value types K and V in this RDD. @@ -583,11 +609,22 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], - conf: JobConf = new JobConf(self.context.hadoopConfiguration)) { + conf: JobConf = new JobConf(self.context.hadoopConfiguration), + codec: Option[Class[_ <: CompressionCodec]] = None) { conf.setOutputKeyClass(keyClass) conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug conf.set("mapred.output.format.class", outputFormatClass.getName) + codec match { + case Some(c) => { + conf.setCompressMapOutput(true) + conf.set("mapred.output.compress", "true") + conf.setMapOutputCompressorClass(c) + conf.set("mapred.output.compression.codec", c.getCanonicalName) + conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) + } + case _ => + } conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf)) saveAsHadoopDataset(conf) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6c0438d76da6..e5995bea22aad 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -7,6 +7,7 @@ import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.io.BytesWritable +import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.TextOutputFormat @@ -730,6 +731,14 @@ abstract class RDD[T: ClassManifest]( .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) } + /** + * Save this RDD as a compressed text file, using string representations of elements. + */ + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { + this.map(x => (NullWritable.get(), new Text(x.toString))) + .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) + } + /** * Save this RDD as a SequenceFile of serialized objects. */ diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 518034e07bf17..2911f9036eb4a 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -18,6 +18,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.hadoop.mapred.SequenceFileOutputFormat import org.apache.hadoop.mapred.OutputCommitter import org.apache.hadoop.mapred.FileOutputCommitter +import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.Writable import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.BytesWritable @@ -62,7 +63,7 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla * byte arrays to BytesWritable, and Strings to Text. The `path` can be on any Hadoop-supported * file system. */ - def saveAsSequenceFile(path: String) { + def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) { def anyToWritable[U <% Writable](u: U): Writable = u val keyClass = getWritableClass[K] @@ -72,14 +73,18 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] + val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { - self.saveAsHadoopFile(path, keyClass, valueClass, format) + self.saveAsHadoopFile(path, keyClass, valueClass, format, jobConf, codec) } else if (!convertKey && convertValue) { - self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) } else if (convertKey && !convertValue) { - self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) } else if (convertKey && convertValue) { - self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) } } } diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index 91b48c745659b..a5d20285916ca 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -7,6 +7,8 @@ import scala.io.Source import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.hadoop.io._ +import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodec, GzipCodec} + import SparkContext._ @@ -26,6 +28,29 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(sc.textFile(outputDir).collect().toList === List("1", "2", "3", "4")) } + test("text files (compressed)") { + sc = new SparkContext("local", "test") + val tempDir = Files.createTempDir() + val normalDir = new File(tempDir, "output_normal").getAbsolutePath + val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath + val codec = new DefaultCodec() + + val data = sc.parallelize("a" * 10000, 1) + data.saveAsTextFile(normalDir) + data.saveAsTextFile(compressedOutputDir, classOf[DefaultCodec]) + + val normalFile = new File(normalDir, "part-00000") + val normalContent = sc.textFile(normalDir).collect + assert(normalContent === Array.fill(10000)("a")) + + val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension) + val compressedContent = sc.textFile(compressedOutputDir).collect + assert(compressedContent === Array.fill(10000)("a")) + + assert(compressedFile.length < normalFile.length) + } + + test("SequenceFiles") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() @@ -37,6 +62,29 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } + test("SequenceFile (compressed)") { + sc = new SparkContext("local", "test") + val tempDir = Files.createTempDir() + val normalDir = new File(tempDir, "output_normal").getAbsolutePath + val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath + val codec = new DefaultCodec() + + val data = sc.parallelize(Seq.fill(100)("abc"), 1).map(x => (x, x)) + data.saveAsSequenceFile(normalDir) + data.saveAsSequenceFile(compressedOutputDir, Some(classOf[DefaultCodec])) + + val normalFile = new File(normalDir, "part-00000") + val normalContent = sc.sequenceFile[String, String](normalDir).collect + assert(normalContent === Array.fill(100)("abc", "abc")) + + val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension) + val compressedContent = sc.sequenceFile[String, String](compressedOutputDir).collect + assert(compressedContent === Array.fill(100)("abc", "abc")) + + assert(compressedFile.length < normalFile.length) + } + + test("SequenceFile with writable key") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() From 083a3485abdcda5913c2186c4a7930ac07b061c4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Jun 2013 11:49:33 -0700 Subject: [PATCH 634/696] Clean extra whitespace --- core/src/test/scala/spark/FileSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index a5d20285916ca..e61ff7793d20c 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -50,7 +50,6 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(compressedFile.length < normalFile.length) } - test("SequenceFiles") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() @@ -84,7 +83,6 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(compressedFile.length < normalFile.length) } - test("SequenceFile with writable key") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() From df592192e736edca9e382a7f92e15bead390ef65 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Jun 2013 18:09:24 -0700 Subject: [PATCH 635/696] Monads FTW --- core/src/main/scala/spark/PairRDDFunctions.scala | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 9bf1227d6519e..15593db0d9b40 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -615,15 +615,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug conf.set("mapred.output.format.class", outputFormatClass.getName) - codec match { - case Some(c) => { - conf.setCompressMapOutput(true) - conf.set("mapred.output.compress", "true") - conf.setMapOutputCompressorClass(c) - conf.set("mapred.output.compression.codec", c.getCanonicalName) - conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) - } - case _ => + for (c <- codec) { + conf.setCompressMapOutput(true) + conf.set("mapred.output.compress", "true") + conf.setMapOutputCompressorClass(c) + conf.set("mapred.output.compression.codec", c.getCanonicalName) + conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) } conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf)) From ef14dc2e7736732932d4edceb3be8d81ba9f8bc7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Jun 2013 18:09:46 -0700 Subject: [PATCH 636/696] Adding Java-API version of compression codec --- .../scala/spark/api/java/JavaPairRDD.scala | 11 +++++ .../scala/spark/api/java/JavaRDDLike.scala | 8 ++++ core/src/test/scala/spark/JavaAPISuite.java | 46 +++++++++++++++++++ 3 files changed, 65 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 30084df4e20fc..76051597b65f0 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -6,6 +6,7 @@ import java.util.Comparator import scala.Tuple2 import scala.collection.JavaConversions._ +import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} @@ -459,6 +460,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass) } + /** Output the RDD to any Hadoop-supported file system, compressing with the supplied codec. */ + def saveAsHadoopFile[F <: OutputFormat[_, _]]( + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[F], + codec: Class[_ <: CompressionCodec]) { + rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, codec) + } + /** Output the RDD to any Hadoop-supported file system. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]]( path: String, diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 9b74d1226f1d9..76b14e2e04d0f 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -4,6 +4,7 @@ import java.util.{List => JList} import scala.Tuple2 import scala.collection.JavaConversions._ +import org.apache.hadoop.io.compress.CompressionCodec import spark.{SparkContext, Partition, RDD, TaskContext} import spark.api.java.JavaPairRDD._ import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} @@ -310,6 +311,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def saveAsTextFile(path: String) = rdd.saveAsTextFile(path) + + /** + * Save this RDD as a compressed text file, using string representations of elements. + */ + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) = + rdd.saveAsTextFile(path, codec) + /** * Save this RDD as a SequenceFile of serialized objects. */ diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 93bb69b41c790..6caa85119a6f9 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -8,6 +8,7 @@ import scala.Tuple2; import com.google.common.base.Charsets; +import org.apache.hadoop.io.compress.DefaultCodec; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -473,6 +474,19 @@ public void textFiles() throws IOException { Assert.assertEquals(expected, readRDD.collect()); } + @Test + public void textFilesCompressed() throws IOException { + File tempDir = Files.createTempDir(); + String outputDir = new File(tempDir, "output").getAbsolutePath(); + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + rdd.saveAsTextFile(outputDir, DefaultCodec.class); + + // Try reading it in as a text file RDD + List expected = Arrays.asList("1", "2", "3", "4"); + JavaRDD readRDD = sc.textFile(outputDir); + Assert.assertEquals(expected, readRDD.collect()); + } + @Test public void sequenceFile() { File tempDir = Files.createTempDir(); @@ -619,6 +633,38 @@ public String call(Tuple2 x) { }).collect().toString()); } + @Test + public void hadoopFileCompressed() { + File tempDir = Files.createTempDir(); + String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2(1, "a"), + new Tuple2(2, "aa"), + new Tuple2(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.map(new PairFunction, IntWritable, Text>() { + @Override + public Tuple2 call(Tuple2 pair) { + return new Tuple2(new IntWritable(pair._1()), new Text(pair._2())); + } + }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, + DefaultCodec.class); + + System.out.println(outputDir); + JavaPairRDD output = sc.hadoopFile(outputDir, + SequenceFileInputFormat.class, IntWritable.class, Text.class); + + Assert.assertEquals(pairs.toString(), output.map(new Function, + String>() { + @Override + public String call(Tuple2 x) { + return x.toString(); + } + }).collect().toString()); + } + @Test public void zip() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); From 190ec617997d621c11ed1aab662a6e3a06815d2f Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 10 Jun 2013 15:27:02 +0800 Subject: [PATCH 637/696] change code style and debug info --- .../main/scala/spark/scheduler/local/LocalScheduler.scala | 8 +++----- .../scala/spark/scheduler/local/LocalTaskSetManager.scala | 1 - 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 69dacfc2bd87c..93d4318b2990b 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -34,8 +34,7 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: I } def launchTask(tasks : Seq[TaskDescription]) { - for (task <- tasks) - { + for (task <- tasks) { freeCores -= 1 localScheduler.threadPool.submit(new Runnable { def run() { @@ -85,8 +84,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } schedulableBuilder.buildPools() - localActor = env.actorSystem.actorOf( - Props(new LocalActor(this, threads)), "Test") + localActor = env.actorSystem.actorOf(Props(new LocalActor(this, threads)), "Test") } override def setListener(listener: TaskSchedulerListener) { @@ -109,7 +107,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val tasks = new ArrayBuffer[TaskDescription](freeCores) val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() for (manager <- sortedTaskSetQueue) { - logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + logDebug("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } var launchTask = false diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f2e07d162a035..70b69bb26feab 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -91,7 +91,6 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas if (availableCpus > 0 && numFinished < numTasks) { findTask() match { case Some(index) => - logInfo(taskSet.tasks(index).toString) val taskId = sched.attemptId.getAndIncrement() val task = taskSet.tasks(index) val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) From fd6148c8b20bc051786ff574d3b8f3b5e79b391a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 10 Jun 2013 10:27:25 -0700 Subject: [PATCH 638/696] Removing print statement --- core/src/test/scala/spark/JavaAPISuite.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 6caa85119a6f9..d306124fca47d 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -652,7 +652,6 @@ public Tuple2 call(Tuple2 pair) { }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, DefaultCodec.class); - System.out.println(outputDir); JavaPairRDD output = sc.hadoopFile(outputDir, SequenceFileInputFormat.class, IntWritable.class, Text.class); From dc4073654b1707f115de30088938f6e53efda0ba Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 11 Jun 2013 00:08:02 -0400 Subject: [PATCH 639/696] Revert "Fix start-slave not passing instance number to spark-daemon." This reverts commit a674d67c0aebb940e3b816e2307206115baec175. --- bin/start-slave.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index dfcbc6981bae1..26b5b9d462037 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -12,4 +12,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.worker.Worker 1 "$@" +"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@" From db5bca08ff00565732946a9c0a0244a9f7021d82 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 12 Jun 2013 10:54:16 -0700 Subject: [PATCH 640/696] add a new top K method to RDD using a bounded priority queue --- core/src/main/scala/spark/RDD.scala | 24 ++++++++++ .../spark/util/BoundedPriorityQueue.scala | 48 +++++++++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 19 ++++++++ 3 files changed, 91 insertions(+) create mode 100644 core/src/main/scala/spark/util/BoundedPriorityQueue.scala diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6c0438d76da6..ec5e5e2433cd7 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -35,6 +35,7 @@ import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 import spark.storage.StorageLevel +import spark.util.BoundedPriorityQueue import SparkContext._ @@ -722,6 +723,29 @@ abstract class RDD[T: ClassManifest]( case _ => throw new UnsupportedOperationException("empty collection") } + /** + * Returns the top K elements from this RDD as defined by + * the specified implicit Ordering[T]. + * @param num the number of top elements to return + * @param ord the implicit ordering for T + * @return an array of top elements + */ + def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { + val topK = mapPartitions { items => + val queue = new BoundedPriorityQueue[T](num) + queue ++= items + Iterator(queue) + }.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + } + + val builder = Array.newBuilder[T] + builder.sizeHint(topK.size) + builder ++= topK + builder.result() + } + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala new file mode 100644 index 0000000000000..53ee95a02e766 --- /dev/null +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -0,0 +1,48 @@ +package spark.util + +import java.util.{PriorityQueue => JPriorityQueue} +import scala.collection.generic.Growable + +/** + * Bounded priority queue. This class modifies the original PriorityQueue's + * add/offer methods such that only the top K elements are retained. The top + * K elements are defined by an implicit Ordering[A]. + */ +class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A], mf: ClassManifest[A]) + extends JPriorityQueue[A](maxSize, ord) with Growable[A] { + + override def offer(a: A): Boolean = { + if (size < maxSize) super.offer(a) + else maybeReplaceLowest(a) + } + + override def add(a: A): Boolean = offer(a) + + override def ++=(xs: TraversableOnce[A]): this.type = { + xs.foreach(add) + this + } + + override def +=(elem: A): this.type = { + add(elem) + this + } + + override def +=(elem1: A, elem2: A, elems: A*): this.type = { + this += elem1 += elem2 ++= elems + } + + private def maybeReplaceLowest(a: A): Boolean = { + val head = peek() + if (head != null && ord.gt(a, head)) { + poll() + super.offer(a) + } else false + } +} + +object BoundedPriorityQueue { + import scala.collection.JavaConverters._ + implicit def asIterable[A](queue: BoundedPriorityQueue[A]): Iterable[A] = queue.asScala +} + diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 3f69e99780af6..67f3332d444d8 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -317,4 +317,23 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(sample.size === checkSample.size) for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) } + + test("top with predefined ordering") { + sc = new SparkContext("local", "test") + val nums = Array.range(1, 100000) + val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) + val topK = ints.top(5) + assert(topK.size === 5) + assert(topK.sorted === nums.sorted.takeRight(5)) + } + + test("top with custom ordering") { + sc = new SparkContext("local", "test") + val words = Vector("a", "b", "c", "d") + implicit val ord = implicitly[Ordering[String]].reverse + val rdd = sc.makeRDD(words, 2) + val topK = rdd.top(2) + assert(topK.size === 2) + assert(topK.sorted === Array("b", "a")) + } } From 3f96c6f27b08039fb7b8d295f5de2083544e979f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 12 Jun 2013 17:20:05 -0700 Subject: [PATCH 641/696] Fixed jvmArgs in maven build. --- pom.xml | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index ce77ba37c6598..c893ec755eb4c 100644 --- a/pom.xml +++ b/pom.xml @@ -60,7 +60,7 @@ 4.1.2 1.2.17 - 0m + 64m 512m @@ -395,10 +395,8 @@ -Xms64m -Xmx1024m - -XX:PermSize - ${PermGen} - -XX:MaxPermSize - ${MaxPermGen} + -XX:PermSize=${PermGen} + -XX:MaxPermSize=${MaxPermGen} -source From 967a6a699da7da007f51e59d085a357da5ec14da Mon Sep 17 00:00:00 2001 From: Mingfei Date: Thu, 13 Jun 2013 14:36:07 +0800 Subject: [PATCH 642/696] modify sparklister function interface according to comments --- .../scala/spark/scheduler/DAGScheduler.scala | 15 ++-- .../scala/spark/scheduler/JobLogger.scala | 89 +++++++++---------- .../scala/spark/scheduler/SparkListener.scala | 38 ++++---- .../spark/scheduler/JobLoggerSuite.scala | 10 +-- 4 files changed, 79 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 43dd7d6534cf4..e281e5a8dbeba 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -289,7 +289,7 @@ class DAGScheduler( val finalStage = newStage(finalRDD, None, runId) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() - sparkListeners.foreach(_.onJobStart(job, properties)) + sparkListeners.foreach(_.onJobStart(SparkListenerJobStart(job, properties))) logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") @@ -312,7 +312,7 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => - sparkListeners.foreach(_.onTaskEnd(completion)) + sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion))) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -323,8 +323,8 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - val JobCancelEvent = new SparkListenerJobCancelled("SPARKCONTEXT_SHUTDOWN") - sparkListeners.foreach(_.onJobEnd(job, JobCancelEvent)) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobCancelled(job, + "SPARKCONTEXT_SHUTDOWN"))) } return true } @@ -472,7 +472,7 @@ class DAGScheduler( } } if (tasks.size > 0) { - sparkListeners.foreach(_.onStageSubmitted(stage, "TASKS_SIZE=" + tasks.size)) + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) @@ -527,7 +527,7 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) - sparkListeners.foreach(_.onJobEnd(job, SparkListenerJobSuccess)) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobSuccess(job))) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -671,8 +671,7 @@ class DAGScheduler( job.listener.jobFailed(new SparkException("Job failed: " + reason)) activeJobs -= job resultStageToJob -= resultStage - val jobFailedEvent = new SparkListenerJobFailed(failedStage) - sparkListeners.foreach(_.onJobEnd(job, jobFailedEvent)) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobFailed(job, failedStage))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 46b9fa974b10b..002c5826cbbac 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -15,13 +15,6 @@ import spark.scheduler.cluster.TaskInfo // used to record runtime information for each job, including RDD graph // tasks' start/stop shuffle information and information from outside -sealed trait JobLoggerEvent -case class JobLoggerOnJobStart(job: ActiveJob, properties: Properties) extends JobLoggerEvent -case class JobLoggerOnStageSubmitted(stage: Stage, info: String) extends JobLoggerEvent -case class JobLoggerOnStageCompleted(stageCompleted: StageCompleted) extends JobLoggerEvent -case class JobLoggerOnJobEnd(job: ActiveJob, event: SparkListenerEvents) extends JobLoggerEvent -case class JobLoggerOnTaskEnd(event: CompletionEvent) extends JobLoggerEvent - class JobLogger(val logDirName: String) extends SparkListener with Logging { private val logDir = if (System.getenv("SPARK_LOG_DIR") != null) @@ -32,7 +25,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { private val stageIDToJobID = new HashMap[Int, Int] private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]] private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val eventQueue = new LinkedBlockingQueue[JobLoggerEvent] + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents] createLogDir() def this() = this(String.valueOf(System.currentTimeMillis())) @@ -50,15 +43,19 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { val event = eventQueue.take logDebug("Got event of type " + event.getClass.getName) event match { - case JobLoggerOnJobStart(job, info) => - processJobStartEvent(job, info) - case JobLoggerOnStageSubmitted(stage, info) => - processStageSubmittedEvent(stage, info) - case JobLoggerOnStageCompleted(stageCompleted) => - processStageCompletedEvent(stageCompleted) - case JobLoggerOnJobEnd(job, event) => - processJobEndEvent(job, event) - case JobLoggerOnTaskEnd(event) => + case SparkListenerJobStart(job, properties) => + processJobStartEvent(job, properties) + case SparkListenerStageSubmitted(stage, taskSize) => + processStageSubmittedEvent(stage, taskSize) + case StageCompleted(stageInfo) => + processStageCompletedEvent(stageInfo) + case SparkListenerJobSuccess(job) => + processJobEndEvent(job) + case SparkListenerJobFailed(job, failedStage) => + processJobEndEvent(job, failedStage) + case SparkListenerJobCancelled(job, reason) => + processJobEndEvent(job, reason) + case SparkListenerTaskEnd(event) => processTaskEndEvent(event) case _ => } @@ -225,26 +222,26 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) } - override def onStageSubmitted(stage: Stage, info: String = "") { - eventQueue.put(JobLoggerOnStageSubmitted(stage, info)) + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { + eventQueue.put(stageSubmitted) } - protected def processStageSubmittedEvent(stage: Stage, info: String) { - stageLogInfo(stage.id, "STAGE_ID=" + stage.id + " STATUS=SUBMITTED " + info) + protected def processStageSubmittedEvent(stage: Stage, taskSize: Int) { + stageLogInfo(stage.id, "STAGE_ID=" + stage.id + " STATUS=SUBMITTED" + " TASK_SIZE=" + taskSize) } override def onStageCompleted(stageCompleted: StageCompleted) { - eventQueue.put(JobLoggerOnStageCompleted(stageCompleted)) + eventQueue.put(stageCompleted) } - protected def processStageCompletedEvent(stageCompleted: StageCompleted) { - stageLogInfo(stageCompleted.stageInfo.stage.id, "STAGE_ID=" + - stageCompleted.stageInfo.stage.id + " STATUS=COMPLETED") + protected def processStageCompletedEvent(stageInfo: StageInfo) { + stageLogInfo(stageInfo.stage.id, "STAGE_ID=" + + stageInfo.stage.id + " STATUS=COMPLETED") } - override def onTaskEnd(event: CompletionEvent) { - eventQueue.put(JobLoggerOnTaskEnd(event)) + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + eventQueue.put(taskEnd) } protected def processTaskEndEvent(event: CompletionEvent) { @@ -273,24 +270,26 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { - eventQueue.put(JobLoggerOnJobEnd(job, event)) + override def onJobEnd(jobEnd: SparkListenerEvents) { + eventQueue.put(jobEnd) } - protected def processJobEndEvent(job: ActiveJob, event: SparkListenerEvents) { - var info = "JOB_ID=" + job.runId + " STATUS=" - var validEvent = true - event match { - case SparkListenerJobSuccess => info += "SUCCESS" - case SparkListenerJobFailed(failedStage) => - info += "FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" + failedStage.id - case SparkListenerJobCancelled(reason) => info += "CANCELLED REASON=" + reason - case _ => validEvent = false - } - if (validEvent) { - jobLogInfo(job.runId, info) - closeLogWriter(job.runId) - } + protected def processJobEndEvent(job: ActiveJob) { + val info = "JOB_ID=" + job.runId + " STATUS=SUCCESS" + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) + } + + protected def processJobEndEvent(job: ActiveJob, failedStage: Stage) { + val info = "JOB_ID=" + job.runId + " STATUS=FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" + + failedStage.id + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) + } + protected def processJobEndEvent(job: ActiveJob, reason: String) { + var info = "JOB_ID=" + job.runId + " STATUS=CANCELLED REASON=" + reason + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) } protected def recordJobProperties(jobID: Int, properties: Properties) { @@ -300,8 +299,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - override def onJobStart(job: ActiveJob, properties: Properties = null) { - eventQueue.put(JobLoggerOnJobStart(job, properties)) + override def onJobStart(jobStart: SparkListenerJobStart) { + eventQueue.put(jobStart) } protected def processJobStartEvent(job: ActiveJob, properties: Properties) { diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 9cf7f3ffc0ea0..9265261dc1173 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -6,6 +6,24 @@ import spark.util.Distribution import spark.{Utils, Logging, SparkContext, TaskEndReason} import spark.executor.TaskMetrics + +sealed trait SparkListenerEvents + +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends SparkListenerEvents + +case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents + +case class SparkListenerTaskEnd(event: CompletionEvent) extends SparkListenerEvents + +case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) + extends SparkListenerEvents + +case class SparkListenerJobSuccess(job: ActiveJob) extends SparkListenerEvents + +case class SparkListenerJobFailed(job: ActiveJob, failedStage: Stage) extends SparkListenerEvents + +case class SparkListenerJobCancelled(job: ActiveJob, reason: String) extends SparkListenerEvents + trait SparkListener { /** * called when a stage is completed, with information on the completed stage @@ -15,35 +33,25 @@ trait SparkListener { /** * called when a stage is submitted */ - def onStageSubmitted(stage: Stage, info: String = "") { } - + def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } + /** * called when a task ends */ - def onTaskEnd(event: CompletionEvent) { } + def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } /** * called when a job starts */ - def onJobStart(job: ActiveJob, properties: Properties = null) { } + def onJobStart(jobStart: SparkListenerJobStart) { } /** * called when a job ends */ - def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { } + def onJobEnd(jobEnd: SparkListenerEvents) { } } -sealed trait SparkListenerEvents - -case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents - -case object SparkListenerJobSuccess extends SparkListenerEvents - -case class SparkListenerJobFailed(failedStage: Stage) extends SparkListenerEvents - -case class SparkListenerJobCancelled(reason: String) extends SparkListenerEvents - /** * Simple SparkListener that logs a few summary statistics when each stage completes */ diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 34fd8b995e141..a654bf3ffdc87 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -40,7 +40,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) - joblogger.onStageSubmitted(rootStage) + joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) joblogger.getEventQueue.size should be (1) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") @@ -86,11 +86,11 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers var onJobStartCount = 0 var onStageCompletedCount = 0 var onStageSubmittedCount = 0 - override def onTaskEnd(event: CompletionEvent) = onTaskEndCount += 1 - override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) = onJobEndCount += 1 - override def onJobStart(job: ActiveJob, properties: Properties) = onJobStartCount += 1 + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 + override def onJobEnd(jobEnd: SparkListenerEvents) = onJobEndCount += 1 + override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 - override def onStageSubmitted(stage: Stage, info: String = "") = onStageSubmittedCount += 1 + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 } sc.addSparkListener(joblogger) val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } From b5b12823faf62766d880e497c90b44b21f5a433a Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Thu, 13 Jun 2013 14:05:46 +0530 Subject: [PATCH 643/696] Fixing the style as per feedback --- .../scala/spark/examples/CassandraTest.scala | 72 ++++++++++--------- 1 file changed, 37 insertions(+), 35 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 2cc62b9fe92a4..0fe1833e83f05 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -1,9 +1,11 @@ package spark.examples import org.apache.hadoop.mapreduce.Job -import org.apache.cassandra.hadoop.{ColumnFamilyOutputFormat, ConfigHelper, ColumnFamilyInputFormat} +import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat +import org.apache.cassandra.hadoop.ConfigHelper +import org.apache.cassandra.hadoop.ColumnFamilyInputFormat import org.apache.cassandra.thrift._ -import spark.{RDD, SparkContext} +import spark.SparkContext import spark.SparkContext._ import java.nio.ByteBuffer import java.util.SortedMap @@ -12,9 +14,9 @@ import org.apache.cassandra.utils.ByteBufferUtil import scala.collection.JavaConversions._ - /* - * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra support for Hadoop. + * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra + * support for Hadoop. * * To run this example, run this file with the following command params - * @@ -26,32 +28,31 @@ import scala.collection.JavaConversions._ * 1. You have already created a keyspace called casDemo and it has a column family named Words * 2. There are column family has a column named "para" which has test content. * - * You can create the content by running the following script at the bottom of this file with cassandra-cli. + * You can create the content by running the following script at the bottom of this file with + * cassandra-cli. * */ object CassandraTest { + def main(args: Array[String]) { - //Get a SparkContext + // Get a SparkContext val sc = new SparkContext(args(0), "casDemo") - //Build the job configuration with ConfigHelper provided by Cassandra + // Build the job configuration with ConfigHelper provided by Cassandra val job = new Job() job.setInputFormatClass(classOf[ColumnFamilyInputFormat]) - ConfigHelper.setInputInitialAddress(job.getConfiguration(), args(1)) - - ConfigHelper.setInputRpcPort(job.getConfiguration(), args(2)) - - ConfigHelper.setOutputInitialAddress(job.getConfiguration(), args(1)) - - ConfigHelper.setOutputRpcPort(job.getConfiguration(), args(2)) + val host: String = args(1) + val port: String = args(2) + ConfigHelper.setInputInitialAddress(job.getConfiguration(), host) + ConfigHelper.setInputRpcPort(job.getConfiguration(), port) + ConfigHelper.setOutputInitialAddress(job.getConfiguration(), host) + ConfigHelper.setOutputRpcPort(job.getConfiguration(), port) ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") - ConfigHelper.setOutputColumnFamily(job.getConfiguration(), "casDemo", "WordCount") - val predicate = new SlicePredicate() val sliceRange = new SliceRange() sliceRange.setStart(Array.empty[Byte]) @@ -60,11 +61,11 @@ object CassandraTest { ConfigHelper.setInputSlicePredicate(job.getConfiguration(), predicate) ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - //Make a new Hadoop RDD - val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), + // Make a new Hadoop RDD + val casRdd = sc.newAPIHadoopRDD( + job.getConfiguration(), classOf[ColumnFamilyInputFormat], classOf[ByteBuffer], classOf[SortedMap[ByteBuffer, IColumn]]) @@ -76,7 +77,7 @@ object CassandraTest { } } - //Lets get the word count in paras + // Lets get the word count in paras val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) counts.collect().foreach { @@ -95,20 +96,17 @@ object CassandraTest { colCount.setValue(ByteBufferUtil.bytes(count.toLong)) colCount.setTimestamp(System.currentTimeMillis) - val outputkey = ByteBufferUtil.bytes(word + "-COUNT-" + System.currentTimeMillis) val mutations: java.util.List[Mutation] = new Mutation() :: new Mutation() :: Nil mutations.get(0).setColumn_or_supercolumn(new ColumnOrSuperColumn()) mutations.get(0).column_or_supercolumn.setColumn(colWord) - mutations.get(1).setColumn_or_supercolumn(new ColumnOrSuperColumn()) mutations.get(1).column_or_supercolumn.setColumn(colCount) (outputkey, mutations) } }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], classOf[ColumnFamilyOutputFormat], job.getConfiguration) - } } @@ -117,16 +115,20 @@ create keyspace casDemo; use casDemo; create column family WordCount with comparator = UTF8Type; -update column family WordCount with column_metadata = [{column_name: word, validation_class: UTF8Type}, {column_name: wcount, validation_class: LongType}]; +update column family WordCount with column_metadata = + [{column_name: word, validation_class: UTF8Type}, + {column_name: wcount, validation_class: LongType}]; create column family Words with comparator = UTF8Type; -update column family Words with column_metadata = [{column_name: book, validation_class: UTF8Type}, {column_name: para, validation_class: UTF8Type}]; +update column family Words with column_metadata = + [{column_name: book, validation_class: UTF8Type}, + {column_name: para, validation_class: UTF8Type}]; assume Words keys as utf8; set Words['3musk001']['book'] = 'The Three Musketeers'; -set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market town of - Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to +set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market + town of Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to be in as perfect a state of revolution as if the Huguenots had just made a second La Rochelle of it. Many citizens, seeing the women flying toward the High Street, leaving their children crying at the open doors, @@ -136,8 +138,8 @@ set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625 every minute, a compact group, vociferous and full of curiosity.'; set Words['3musk002']['book'] = 'The Three Musketeers'; -set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without some city - or other registering in its archives an event of this kind. There were +set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without + some city or other registering in its archives an event of this kind. There were nobles, who made war against each other; there was the king, who made war against the cardinal; there was Spain, which made war against the king. Then, in addition to these concealed or public, secret or open @@ -152,8 +154,8 @@ set Words['3musk002']['para'] = 'In those times panics were common, and few days cause of the hubbub was apparent to all'; set Words['3musk003']['book'] = 'The Three Musketeers'; -set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however large - the sum may be; but you ought also to endeavor to perfect yourself in +set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however + large the sum may be; but you ought also to endeavor to perfect yourself in the exercises becoming a gentleman. I will write a letter today to the Director of the Royal Academy, and tomorrow he will admit you without any expense to yourself. Do not refuse this little service. Our @@ -165,8 +167,8 @@ set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means yo set Words['thelostworld001']['book'] = 'The Lost World'; -set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined against the - red curtain. How beautiful she was! And yet how aloof! We had been +set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined + against the red curtain. How beautiful she was! And yet how aloof! We had been friends, quite good friends; but never could I get beyond the same comradeship which I might have established with one of my fellow-reporters upon the Gazette,--perfectly frank, perfectly kindly, @@ -180,8 +182,8 @@ set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profil as that--or had inherited it in that race memory which we call instinct.'; set Words['thelostworld002']['book'] = 'The Lost World'; -set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, red-headed news - editor, and I rather hoped that he liked me. Of course, Beaumont was +set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, + red-headed news editor, and I rather hoped that he liked me. Of course, Beaumont was the real boss; but he lived in the rarefied atmosphere of some Olympian height from which he could distinguish nothing smaller than an international crisis or a split in the Cabinet. Sometimes we saw him From 44b8dbaedeb88f12ea911968c524883805f7ad95 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Thu, 13 Jun 2013 16:23:15 -0700 Subject: [PATCH 644/696] use Iterator.single(elem) instead of Iterator(elem) for improved performance based on scaladocs --- core/src/main/scala/spark/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index ec5e5e2433cd7..bc9c17d5075e5 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -734,7 +734,7 @@ abstract class RDD[T: ClassManifest]( val topK = mapPartitions { items => val queue = new BoundedPriorityQueue[T](num) queue ++= items - Iterator(queue) + Iterator.single(queue) }.reduce { (queue1, queue2) => queue1 ++= queue2 queue1 From 93b3f5e535c509a017a433b72249fc49c79d4a0f Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Thu, 13 Jun 2013 16:26:35 -0700 Subject: [PATCH 645/696] drop unneeded ClassManifest implicit --- core/src/main/scala/spark/util/BoundedPriorityQueue.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala index 53ee95a02e766..ef01beaea5dfa 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -8,7 +8,7 @@ import scala.collection.generic.Growable * add/offer methods such that only the top K elements are retained. The top * K elements are defined by an implicit Ordering[A]. */ -class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A], mf: ClassManifest[A]) +class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends JPriorityQueue[A](maxSize, ord) with Growable[A] { override def offer(a: A): Boolean = { From 6738178d0daf1bbe7441db7c0c773a29bb2ec388 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 13 Jun 2013 23:59:42 -0700 Subject: [PATCH 646/696] SPARK-772: groupByKey should disable map side combine. --- core/src/main/scala/spark/PairRDDFunctions.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 15593db0d9b40..fa4bbfc76f306 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -19,7 +19,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil, TaskAttemptID, TaskAttemptContext} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil} import spark.partial.BoundedDouble import spark.partial.PartialResult @@ -187,11 +187,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + // groupByKey shouldn't use map side combine because map side combine does not + // reduce the amount of data shuffled and requires all map side data be inserted + // into a hash table, leading to more objects in the old gen. def createCombiner(v: V) = ArrayBuffer(v) def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - def mergeCombiners(b1: ArrayBuffer[V], b2: ArrayBuffer[V]) = b1 ++= b2 val bufs = combineByKey[ArrayBuffer[V]]( - createCombiner _, mergeValue _, mergeCombiners _, partitioner) + createCombiner _, mergeValue _, null, partitioner, mapSideCombine=false) bufs.asInstanceOf[RDD[(K, Seq[V])]] } From 2cc188fd546fa061812f9fd4f72cf936bd01a0e6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 14 Jun 2013 00:10:54 -0700 Subject: [PATCH 647/696] SPARK-774: cogroup should also disable map side combine by default --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 7599ba1a0224b..8966f9f86e3f8 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -6,7 +6,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext} +import spark.{Aggregator, Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -49,12 +49,16 @@ private[spark] class CoGroupAggregator * * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output. - * @param mapSideCombine flag indicating whether to merge values before shuffle step. + * @param mapSideCombine flag indicating whether to merge values before shuffle step. If the flag + * is on, Spark does an extra pass over the data on the map side to merge + * all values belonging to the same key together. This can reduce the amount + * of data shuffled if and only if the number of distinct keys is very small, + * and the ratio of key size to value size is also very small. */ class CoGroupedRDD[K]( @transient var rdds: Seq[RDD[(K, _)]], part: Partitioner, - val mapSideCombine: Boolean = true, + val mapSideCombine: Boolean = false, val serializerClass: String = null) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { From 53add598f2fe09759a0df1e08f87f70503f808c5 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 15 Jun 2013 01:34:17 +0800 Subject: [PATCH 648/696] Update LocalSchedulerSuite to avoid using sleep for task launch --- .../spark/scheduler/LocalSchedulerSuite.scala | 83 +++++++++++++------ 1 file changed, 59 insertions(+), 24 deletions(-) diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 37d14ed113b61..8bd813fd149c5 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -9,9 +9,7 @@ import spark.scheduler.cluster._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.{ConcurrentMap, HashMap} import java.util.concurrent.Semaphore -import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.atomic.AtomicInteger - +import java.util.concurrent.CountDownLatch import java.util.Properties class Lock() { @@ -35,9 +33,19 @@ class Lock() { object TaskThreadInfo { val threadToLock = HashMap[Int, Lock]() val threadToRunning = HashMap[Int, Boolean]() + val threadToStarted = HashMap[Int, CountDownLatch]() } - +/* + * 1. each thread contains one job. + * 2. each job contains one stage. + * 3. each stage only contains one task. + * 4. each task(launched) must be lanched orderly(using threadToStarted) to make sure + * it will get cpu core resource, and will wait to finished after user manually + * release "Lock" and then cluster will contain another free cpu cores. + * 5. each task(pending) must use "sleep" to make sure it has been added to taskSetManager queue, + * thus it will be scheduled later when cluster has free cpu cores. + */ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { def createThread(threadIndex: Int, poolName: String, sc: SparkContext, sem: Semaphore) { @@ -45,22 +53,23 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { TaskThreadInfo.threadToRunning(threadIndex) = false val nums = sc.parallelize(threadIndex to threadIndex, 1) TaskThreadInfo.threadToLock(threadIndex) = new Lock() + TaskThreadInfo.threadToStarted(threadIndex) = new CountDownLatch(1) new Thread { - if (poolName != null) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) - } - override def run() { - val ans = nums.map(number => { - TaskThreadInfo.threadToRunning(number) = true - TaskThreadInfo.threadToLock(number).jobWait() - number - }).collect() - assert(ans.toList === List(threadIndex)) - sem.release() - TaskThreadInfo.threadToRunning(threadIndex) = false - } + if (poolName != null) { + sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) + } + override def run() { + val ans = nums.map(number => { + TaskThreadInfo.threadToRunning(number) = true + TaskThreadInfo.threadToStarted(number).countDown() + TaskThreadInfo.threadToLock(number).jobWait() + TaskThreadInfo.threadToRunning(number) = false + number + }).collect() + assert(ans.toList === List(threadIndex)) + sem.release() + } }.start() - Thread.sleep(2000) } test("Local FIFO scheduler end-to-end test") { @@ -69,11 +78,24 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { val sem = new Semaphore(0) createThread(1,null,sc,sem) + TaskThreadInfo.threadToStarted(1).await() createThread(2,null,sc,sem) + TaskThreadInfo.threadToStarted(2).await() createThread(3,null,sc,sem) + TaskThreadInfo.threadToStarted(3).await() createThread(4,null,sc,sem) + TaskThreadInfo.threadToStarted(4).await() + // thread 5 and 6 (stage pending)must meet following two points + // 1. stages (taskSetManager) of jobs in thread 5 and 6 should be add to taskSetManager + // queue before executing TaskThreadInfo.threadToLock(1).jobFinished() + // 2. priority of stage in thread 5 should be prior to priority of stage in thread 6 + // So I just use "sleep" 1s here for each thread. + // TODO: any better solution? createThread(5,null,sc,sem) + Thread.sleep(1000) createThread(6,null,sc,sem) + Thread.sleep(1000) + assert(TaskThreadInfo.threadToRunning(1) === true) assert(TaskThreadInfo.threadToRunning(2) === true) assert(TaskThreadInfo.threadToRunning(3) === true) @@ -82,8 +104,8 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { assert(TaskThreadInfo.threadToRunning(6) === false) TaskThreadInfo.threadToLock(1).jobFinished() - Thread.sleep(1000) - + TaskThreadInfo.threadToStarted(5).await() + assert(TaskThreadInfo.threadToRunning(1) === false) assert(TaskThreadInfo.threadToRunning(2) === true) assert(TaskThreadInfo.threadToRunning(3) === true) @@ -92,7 +114,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { assert(TaskThreadInfo.threadToRunning(6) === false) TaskThreadInfo.threadToLock(3).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(6).await() assert(TaskThreadInfo.threadToRunning(1) === false) assert(TaskThreadInfo.threadToRunning(2) === true) @@ -116,23 +138,31 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.fairscheduler.allocation.file", xmlPath) createThread(10,"1",sc,sem) + TaskThreadInfo.threadToStarted(10).await() createThread(20,"2",sc,sem) + TaskThreadInfo.threadToStarted(20).await() createThread(30,"3",sc,sem) + TaskThreadInfo.threadToStarted(30).await() assert(TaskThreadInfo.threadToRunning(10) === true) assert(TaskThreadInfo.threadToRunning(20) === true) assert(TaskThreadInfo.threadToRunning(30) === true) createThread(11,"1",sc,sem) + TaskThreadInfo.threadToStarted(11).await() createThread(21,"2",sc,sem) + TaskThreadInfo.threadToStarted(21).await() createThread(31,"3",sc,sem) + TaskThreadInfo.threadToStarted(31).await() assert(TaskThreadInfo.threadToRunning(11) === true) assert(TaskThreadInfo.threadToRunning(21) === true) assert(TaskThreadInfo.threadToRunning(31) === true) createThread(12,"1",sc,sem) + TaskThreadInfo.threadToStarted(12).await() createThread(22,"2",sc,sem) + TaskThreadInfo.threadToStarted(22).await() createThread(32,"3",sc,sem) assert(TaskThreadInfo.threadToRunning(12) === true) @@ -140,20 +170,25 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { assert(TaskThreadInfo.threadToRunning(32) === false) TaskThreadInfo.threadToLock(10).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(32).await() + assert(TaskThreadInfo.threadToRunning(32) === true) + //1. Similar with above scenario, sleep 1s for stage of 23 and 33 to be added to taskSetManager + // queue so that cluster will assign free cpu core to stage 23 after stage 11 finished. + //2. priority of 23 and 33 will be meaningless as using fair scheduler here. createThread(23,"2",sc,sem) createThread(33,"3",sc,sem) + Thread.sleep(1000) TaskThreadInfo.threadToLock(11).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(23).await() assert(TaskThreadInfo.threadToRunning(23) === true) assert(TaskThreadInfo.threadToRunning(33) === false) TaskThreadInfo.threadToLock(12).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(33).await() assert(TaskThreadInfo.threadToRunning(33) === true) From e8801d44900153eae6412963d2f3e2f19bfdc4e9 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Fri, 14 Jun 2013 23:39:05 -0700 Subject: [PATCH 649/696] use delegation for BoundedPriorityQueue, add Java API --- core/src/main/scala/spark/RDD.scala | 9 ++--- .../main/scala/spark/api/java/JavaRDD.scala | 1 - .../scala/spark/api/java/JavaRDDLike.scala | 27 +++++++++++++- .../spark/util/BoundedPriorityQueue.scala | 35 +++++++++---------- 4 files changed, 44 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index bc9c17d5075e5..4a4616c843d45 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -731,19 +731,14 @@ abstract class RDD[T: ClassManifest]( * @return an array of top elements */ def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { - val topK = mapPartitions { items => + mapPartitions { items => val queue = new BoundedPriorityQueue[T](num) queue ++= items Iterator.single(queue) }.reduce { (queue1, queue2) => queue1 ++= queue2 queue1 - } - - val builder = Array.newBuilder[T] - builder.sizeHint(topK.size) - builder ++= topK - builder.result() + }.toArray } /** diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index eb81ed64cd62c..626b49945429c 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -86,7 +86,6 @@ JavaRDDLike[T, JavaRDD[T]] { */ def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) - } object JavaRDD { diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 9b74d1226f1d9..3e9c779d7b840 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -1,6 +1,6 @@ package spark.api.java -import java.util.{List => JList} +import java.util.{List => JList, Comparator} import scala.Tuple2 import scala.collection.JavaConversions._ @@ -351,4 +351,29 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def toDebugString(): String = { rdd.toDebugString } + + /** + * Returns the top K elements from this RDD as defined by + * the specified Comparator[T]. + * @param num the number of top elements to return + * @param comp the comparator that defines the order + * @return an array of top elements + */ + def top(num: Int, comp: Comparator[T]): JList[T] = { + import scala.collection.JavaConversions._ + val topElems = rdd.top(num)(Ordering.comparatorToOrdering(comp)) + val arr: java.util.Collection[T] = topElems.toSeq + new java.util.ArrayList(arr) + } + + /** + * Returns the top K elements from this RDD using the + * natural ordering for T. + * @param num the number of top elements to return + * @return an array of top elements + */ + def top(num: Int): JList[T] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] + top(num, comp) + } } diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala index ef01beaea5dfa..4bc5db8bb7960 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -1,30 +1,30 @@ package spark.util +import java.io.Serializable import java.util.{PriorityQueue => JPriorityQueue} import scala.collection.generic.Growable +import scala.collection.JavaConverters._ /** - * Bounded priority queue. This class modifies the original PriorityQueue's - * add/offer methods such that only the top K elements are retained. The top - * K elements are defined by an implicit Ordering[A]. + * Bounded priority queue. This class wraps the original PriorityQueue + * class and modifies it such that only the top K elements are retained. + * The top K elements are defined by an implicit Ordering[A]. */ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) - extends JPriorityQueue[A](maxSize, ord) with Growable[A] { + extends Iterable[A] with Growable[A] with Serializable { - override def offer(a: A): Boolean = { - if (size < maxSize) super.offer(a) - else maybeReplaceLowest(a) - } + private val underlying = new JPriorityQueue[A](maxSize, ord) - override def add(a: A): Boolean = offer(a) + override def iterator: Iterator[A] = underlying.iterator.asScala override def ++=(xs: TraversableOnce[A]): this.type = { - xs.foreach(add) + xs.foreach { this += _ } this } override def +=(elem: A): this.type = { - add(elem) + if (size < maxSize) underlying.offer(elem) + else maybeReplaceLowest(elem) this } @@ -32,17 +32,14 @@ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) this += elem1 += elem2 ++= elems } + override def clear() { underlying.clear() } + private def maybeReplaceLowest(a: A): Boolean = { - val head = peek() + val head = underlying.peek() if (head != null && ord.gt(a, head)) { - poll() - super.offer(a) + underlying.poll() + underlying.offer(a) } else false } } -object BoundedPriorityQueue { - import scala.collection.JavaConverters._ - implicit def asIterable[A](queue: BoundedPriorityQueue[A]): Iterable[A] = queue.asScala -} - From 479442a9b913b08a64da4bd5848111d950105336 Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Sat, 15 Jun 2013 17:35:55 -0700 Subject: [PATCH 650/696] Add zeroLengthPartitions() test to make sure, e.g., StatCounter.scala can handle empty partitions without incorrectly returning NaN --- core/src/test/scala/spark/JavaAPISuite.java | 22 +++++++++++++++++++++ project/plugins.sbt | 2 ++ 2 files changed, 24 insertions(+) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 93bb69b41c790..3190a43e73486 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -314,6 +314,28 @@ public Boolean call(Double x) { List take = rdd.take(5); } + @Test + public void zeroLengthPartitions() { + // Create RDD with some consecutive empty partitions (including the "first" one) + JavaDoubleRDD rdd = sc + .parallelizeDoubles(Arrays.asList(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) + .filter(new Function() { + @Override + public Boolean call(Double x) { + return x > 0.0; + } + }); + + // Run the partitions, including the consecutive empty ones, through StatCounter + StatCounter stats = rdd.stats(); + Assert.assertEquals(6.0, stats.sum(), 0.01); + Assert.assertEquals(6.0/2, rdd.mean(), 0.01); + Assert.assertEquals(1.0, rdd.variance(), 0.01); + Assert.assertEquals(1.0, rdd.stdev(), 0.01); + + // Add other tests here for classes that should be able to handle empty partitions correctly + } + @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); diff --git a/project/plugins.sbt b/project/plugins.sbt index d4f2442872363..25b812a28da35 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -16,3 +16,5 @@ addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") + +libraryDependencies += "com.novocode" % "junit-interface" % "0.10-M4" % "test" From 5c886194e458c64fcf24066af351bde47dd8bf12 Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Sun, 16 Jun 2013 01:23:48 -0700 Subject: [PATCH 651/696] Move zero-length partition testing from JavaAPISuite.java to PartitioningSuite.scala --- core/src/test/scala/spark/JavaAPISuite.java | 22 ------------------- .../test/scala/spark/PartitioningSuite.scala | 21 ++++++++++++++++-- 2 files changed, 19 insertions(+), 24 deletions(-) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 3190a43e73486..93bb69b41c790 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -314,28 +314,6 @@ public Boolean call(Double x) { List take = rdd.take(5); } - @Test - public void zeroLengthPartitions() { - // Create RDD with some consecutive empty partitions (including the "first" one) - JavaDoubleRDD rdd = sc - .parallelizeDoubles(Arrays.asList(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) - .filter(new Function() { - @Override - public Boolean call(Double x) { - return x > 0.0; - } - }); - - // Run the partitions, including the consecutive empty ones, through StatCounter - StatCounter stats = rdd.stats(); - Assert.assertEquals(6.0, stats.sum(), 0.01); - Assert.assertEquals(6.0/2, rdd.mean(), 0.01); - Assert.assertEquals(1.0, rdd.variance(), 0.01); - Assert.assertEquals(1.0, rdd.stdev(), 0.01); - - // Add other tests here for classes that should be able to handle empty partitions correctly - } - @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index 60db759c25f3b..e5745c81b34b1 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -1,10 +1,10 @@ package spark import org.scalatest.FunSuite - import scala.collection.mutable.ArrayBuffer - import SparkContext._ +import spark.util.StatCounter +import scala.math._ class PartitioningSuite extends FunSuite with LocalSparkContext { @@ -120,4 +120,21 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array")) } + + test("Zero-length partitions should be correctly handled") { + // Create RDD with some consecutive empty partitions (including the "first" one) + sc = new SparkContext("local", "test") + val rdd: RDD[Double] = sc + .parallelize(Array(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) + .filter(_ >= 0.0) + + // Run the partitions, including the consecutive empty ones, through StatCounter + val stats: StatCounter = rdd.stats(); + assert(abs(6.0 - stats.sum) < 0.01); + assert(abs(6.0/2 - rdd.mean) < 0.01); + assert(abs(1.0 - rdd.variance) < 0.01); + assert(abs(1.0 - rdd.stdev) < 0.01); + + // Add other tests here for classes that should be able to handle empty partitions correctly + } } From f91195cc150a3ead122046d14bd35b4fcf28c9cb Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Sun, 16 Jun 2013 01:29:53 -0700 Subject: [PATCH 652/696] Import just scala.math.abs rather than scala.math._ --- core/src/test/scala/spark/PartitioningSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index e5745c81b34b1..16f93e71a3721 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -4,7 +4,7 @@ import org.scalatest.FunSuite import scala.collection.mutable.ArrayBuffer import SparkContext._ import spark.util.StatCounter -import scala.math._ +import scala.math.abs class PartitioningSuite extends FunSuite with LocalSparkContext { From fb6d733fa88aa124deecf155af40cc095ecca5b3 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Sun, 16 Jun 2013 22:32:55 +0000 Subject: [PATCH 653/696] update according to comments --- core/src/main/scala/spark/RDD.scala | 71 ++----------------- core/src/main/scala/spark/rdd/PipedRDD.scala | 29 +++----- core/src/test/scala/spark/PipedRDDSuite.scala | 13 ++-- 3 files changed, 24 insertions(+), 89 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index a1c9604324631..152f7be9bb868 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -355,68 +355,6 @@ abstract class RDD[T: ClassManifest]( def pipe(command: String, env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) - /** - * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. - */ - def pipe[U<: Seq[String]]( - command: String, - env: Map[String, String], - transform: (T,String => Unit) => Any, - pipeContext: Broadcast[U], - delimiter: String): RDD[String] = - new PipedRDD(this, command, env, transform, pipeContext, delimiter) - - /** - * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. - */ - def pipe[U<: Seq[String]]( - command: String, - transform: (T,String => Unit) => Any, - pipeContext: Broadcast[U]): RDD[String] = - new PipedRDD(this, command, Map[String, String](), transform, pipeContext, "\u0001") - - /** - * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. - */ - def pipe[U<: Seq[String]]( - command: String, - env: Map[String, String], - transform: (T,String => Unit) => Any, - pipeContext: Broadcast[U]): RDD[String] = - new PipedRDD(this, command, env, transform, pipeContext, "\u0001") /** * Return an RDD created by piping elements to a forked external process. @@ -432,13 +370,12 @@ abstract class RDD[T: ClassManifest]( * external process with "^A" as the delimiter in the end of context data. Delimiter can also * be customized by the last parameter delimiter. */ - def pipe[U<: Seq[String]]( + def pipe( command: Seq[String], env: Map[String, String] = Map(), - transform: (T,String => Unit) => Any = null, - pipeContext: Broadcast[U] = null, - delimiter: String = "\u0001"): RDD[String] = - new PipedRDD(this, command, env, transform, pipeContext, delimiter) + printPipeContext: (String => Unit) => Unit = null, + printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = + new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, printRDDElement) /** * Return a new RDD by applying a function to each partition of this RDD. diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index d58aaae709ab9..b2c07891ab415 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -16,14 +16,12 @@ import spark.broadcast.Broadcast * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassManifest, U <: Seq[String]]( +class PipedRDD[T: ClassManifest]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], - transform: (T, String => Unit) => Any, - pipeContext: Broadcast[U], - delimiter: String - ) + printPipeContext: (String => Unit) => Unit, + printRDDElement: (T, String => Unit) => Unit) extends RDD[String](prev) { // Similar to Runtime.exec(), if we are given a single string, split it into words @@ -32,10 +30,9 @@ class PipedRDD[T: ClassManifest, U <: Seq[String]]( prev: RDD[T], command: String, envVars: Map[String, String] = Map(), - transform: (T, String => Unit) => Any = null, - pipeContext: Broadcast[U] = null, - delimiter: String = "\u0001") = - this(prev, PipedRDD.tokenize(command), envVars, transform, pipeContext, delimiter) + printPipeContext: (String => Unit) => Unit = null, + printRDDElement: (T, String => Unit) => Unit = null) = + this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement) override def getPartitions: Array[Partition] = firstParent[T].partitions @@ -64,17 +61,13 @@ class PipedRDD[T: ClassManifest, U <: Seq[String]]( SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) - // input the pipeContext firstly - if ( pipeContext != null) { - for (elem <- pipeContext.value) { - out.println(elem) - } - // delimiter\n as the marker of the end of the pipeContext - out.println(delimiter) + // input the pipe context firstly + if ( printPipeContext != null) { + printPipeContext(out.println(_)) } for (elem <- firstParent[T].iterator(split, context)) { - if (transform != null) { - transform(elem, out.println(_)) + if (printRDDElement != null) { + printRDDElement(elem, out.println(_)) } else { out.println(elem) } diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index d2852867de753..ed075f93ec550 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -22,9 +22,12 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { test("advanced pipe") { sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val bl = sc.broadcast(List("0")) - val piped = nums.pipe(Seq("cat"), Map[String, String](), - (i:Int, f: String=> Unit) => f(i + "_"), sc.broadcast(List("0"))) + val piped = nums.pipe(Seq("cat"), + Map[String, String](), + (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, + (i:Int, f: String=> Unit) => f(i + "_")) val c = piped.collect() @@ -40,8 +43,10 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) val d = nums1.groupBy(str=>str.split("\t")(0)). - pipe(Seq("cat"), Map[String, String](), (i:Tuple2[String, Seq[String]], f: String=> Unit) => - {for (e <- i._2){ f(e + "_")}}, sc.broadcast(List("0"))).collect() + pipe(Seq("cat"), + Map[String, String](), + (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, + (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}).collect() assert(d.size === 8) assert(d(0) === "0") assert(d(1) === "\u0001") From 4508089fc342802a2f37fea6893cd47abd81fdd7 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Mon, 17 Jun 2013 05:23:46 +0000 Subject: [PATCH 654/696] refine comments and add sc.clean --- core/src/main/scala/spark/RDD.scala | 30 +++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 05ff399a7b2db..223dcdc19d877 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -361,24 +361,30 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. + * The print behavior can be customized by providing two functions. + * + * @param command command to run in forked process. + * @param env environment variables to set. + * @param printPipeContext Before piping elements, this function is called as an oppotunity + * to pipe context data. Print line function (like out.println) will be + * passed as printPipeContext's parameter. + * @param printPipeContext Use this function to customize how to pipe elements. This function + * will be called with each RDD element as the 1st parameter, and the + * print line function (like out.println()) as the 2nd parameter. + * An example of pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the elements: + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * for (e <- record._2){f(e)} + * @return the result RDD */ def pipe( command: Seq[String], env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = - new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, printRDDElement) + new PipedRDD(this, command, env, + if (printPipeContext ne null) sc.clean(printPipeContext) else null, + if (printRDDElement ne null) sc.clean(printRDDElement) else null) /** * Return a new RDD by applying a function to each partition of this RDD. From 1450296797e53f1a01166c885050091df9c96e2e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 17 Jun 2013 16:58:23 -0400 Subject: [PATCH 655/696] SPARK-781: Log the temp directory path when Spark says "Failed to create temp directory". --- core/src/main/scala/spark/Utils.scala | 4 +-- .../main/scala/spark/storage/DiskStore.scala | 34 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index ec15326014e8d..fd7b8cc8d5340 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -116,8 +116,8 @@ private object Utils extends Logging { while (dir == null) { attempts += 1 if (attempts > maxAttempts) { - throw new IOException("Failed to create a temp directory after " + maxAttempts + - " attempts!") + throw new IOException("Failed to create a temp directory under (" + root + ") after " + + maxAttempts + " attempts!") } try { dir = new File(root, "spark-" + UUID.randomUUID.toString) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c7281200e7e00..9914beec99596 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -82,15 +82,15 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def size(): Long = lastValidPosition } - val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt - var shuffleSender : ShuffleSender = null + private var shuffleSender : ShuffleSender = null // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid // having really large inodes at the top level. - val localDirs = createLocalDirs() - val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) + private val localDirs: Array[File] = createLocalDirs() + private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) addShutdownHook() @@ -99,7 +99,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) new DiskBlockObjectWriter(blockId, serializer, bufferSize) } - override def getSize(blockId: String): Long = { getFile(blockId).length() } @@ -232,8 +231,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private def createLocalDirs(): Array[File] = { logDebug("Creating local directories at root dirs '" + rootDirs + "'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") - rootDirs.split(",").map(rootDir => { - var foundLocalDir: Boolean = false + rootDirs.split(",").map { rootDir => + var foundLocalDir = false var localDir: File = null var localDirId: String = null var tries = 0 @@ -248,7 +247,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } } catch { case e: Exception => - logWarning("Attempt " + tries + " to create local dir failed", e) + logWarning("Attempt " + tries + " to create local dir " + localDir + " failed", e) } } if (!foundLocalDir) { @@ -258,7 +257,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } logInfo("Created local directory at " + localDir) localDir - }) + } } private def addShutdownHook() { @@ -266,15 +265,16 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - try { - localDirs.foreach { localDir => + localDirs.foreach { localDir => + try { if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } catch { + case t: Throwable => + logError("Exception while deleting local spark dir: " + localDir, t) } - if (shuffleSender != null) { - shuffleSender.stop - } - } catch { - case t: Throwable => logError("Exception while deleting local spark dirs", t) + } + if (shuffleSender != null) { + shuffleSender.stop } } }) From be3c406edf06d5ab9da98097c28ce3eebc958b8e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 17 Jun 2013 17:07:51 -0400 Subject: [PATCH 656/696] Fixed the typo pointed out by Matei. --- core/src/main/scala/spark/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index fd7b8cc8d5340..645c18541e205 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -116,7 +116,7 @@ private object Utils extends Logging { while (dir == null) { attempts += 1 if (attempts > maxAttempts) { - throw new IOException("Failed to create a temp directory under (" + root + ") after " + + throw new IOException("Failed to create a temp directory (under " + root + ") after " + maxAttempts + " attempts!") } try { From 2ab311f4cee3f918dc28daaebd287b11c9f63429 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 18 Jun 2013 00:40:25 +0200 Subject: [PATCH 657/696] Removed second version of junit test plugin from plugins.sbt --- project/plugins.sbt | 2 -- 1 file changed, 2 deletions(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 25b812a28da35..d4f2442872363 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -16,5 +16,3 @@ addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") - -libraryDependencies += "com.novocode" % "junit-interface" % "0.10-M4" % "test" From 1e9269c3eeeaa3a481b95521c703032ed84abd68 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 13 Jun 2013 10:46:22 +0800 Subject: [PATCH 658/696] reduce ZippedPartitionsRDD's getPreferredLocations complexity --- .../main/scala/spark/rdd/ZippedPartitionsRDD.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index dd9f3c2680974..b234428ab2b83 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -53,14 +53,10 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( val exactMatchLocations = exactMatchPreferredLocations.reduce((x, y) => x.intersect(y)) // Remove exact match and then do host local match. - val otherNodePreferredLocations = rddSplitZip.map(x => { - x._1.preferredLocations(x._2).map(hostPort => { - val host = Utils.parseHostPort(hostPort)._1 - - if (exactMatchLocations.contains(host)) null else host - }).filter(_ != null) - }) - val otherNodeLocalLocations = otherNodePreferredLocations.reduce((x, y) => x.intersect(y)) + val exactMatchHosts = exactMatchLocations.map(Utils.parseHostPort(_)._1) + val matchPreferredHosts = exactMatchPreferredLocations.map(locs => locs.map(Utils.parseHostPort(_)._1)) + .reduce((x, y) => x.intersect(y)) + val otherNodeLocalLocations = matchPreferredHosts.filter { s => !exactMatchHosts.contains(s) } otherNodeLocalLocations ++ exactMatchLocations } From 0a2a9bce1e83e891334985c29176c6426b8b1751 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Tue, 18 Jun 2013 21:30:13 +0000 Subject: [PATCH 659/696] fix typo and coding style --- core/src/main/scala/spark/RDD.scala | 14 +++++++------- core/src/main/scala/spark/rdd/PipedRDD.scala | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 223dcdc19d877..709271d4ebbdf 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -368,13 +368,13 @@ abstract class RDD[T: ClassManifest]( * @param printPipeContext Before piping elements, this function is called as an oppotunity * to pipe context data. Print line function (like out.println) will be * passed as printPipeContext's parameter. - * @param printPipeContext Use this function to customize how to pipe elements. This function - * will be called with each RDD element as the 1st parameter, and the - * print line function (like out.println()) as the 2nd parameter. - * An example of pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the elements: - * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = - * for (e <- record._2){f(e)} + * @param printRDDElement Use this function to customize how to pipe elements. This function + * will be called with each RDD element as the 1st parameter, and the + * print line function (like out.println()) as the 2nd parameter. + * An example of pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the elements: + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * for (e <- record._2){f(e)} * @return the result RDD */ def pipe( diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index b2c07891ab415..c0baf43d43eef 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -62,7 +62,7 @@ class PipedRDD[T: ClassManifest]( val out = new PrintWriter(proc.getOutputStream) // input the pipe context firstly - if ( printPipeContext != null) { + if (printPipeContext != null) { printPipeContext(out.println(_)) } for (elem <- firstParent[T].iterator(split, context)) { From 7902baddc797f86f5bdbcc966f5cd60545638bf7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 19 Jun 2013 13:34:30 +0200 Subject: [PATCH 660/696] Update ASM to version 4.0 --- core/pom.xml | 4 ++-- core/src/main/scala/spark/ClosureCleaner.scala | 11 +++++------ pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- .../main/scala/spark/repl/ExecutorClassLoader.scala | 3 +-- 5 files changed, 12 insertions(+), 14 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index d8687bf9911b3..88f0ed70f3c03 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -32,8 +32,8 @@ compress-lzf - asm - asm-all + org.ow2.asm + asm com.google.protobuf diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala index 50d6a1c5c9fb9..d5e7132ff96e0 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/spark/ClosureCleaner.scala @@ -5,8 +5,7 @@ import java.lang.reflect.Field import scala.collection.mutable.Map import scala.collection.mutable.Set -import org.objectweb.asm.{ClassReader, MethodVisitor, Type} -import org.objectweb.asm.commons.EmptyVisitor +import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import org.objectweb.asm.Opcodes._ import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream} @@ -162,10 +161,10 @@ private[spark] object ClosureCleaner extends Logging { } } -private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor { +private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { - return new EmptyVisitor { + return new MethodVisitor(ASM4) { override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) { if (op == GETFIELD) { for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { @@ -188,7 +187,7 @@ private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) exten } } -private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor { +private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { var myName: String = null override def visit(version: Int, access: Int, name: String, sig: String, @@ -198,7 +197,7 @@ private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisi override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { - return new EmptyVisitor { + return new MethodVisitor(ASM4) { override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { val argTypes = Type.getArgumentTypes(desc) diff --git a/pom.xml b/pom.xml index c893ec755eb4c..3bcb2a3f345dc 100644 --- a/pom.xml +++ b/pom.xml @@ -190,9 +190,9 @@ 0.8.4 - asm - asm-all - 3.3.1 + org.ow2.asm + asm + 4.0 com.google.protobuf diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 824af821f9586..b1f3f9a2eaf9d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -148,7 +148,7 @@ object SparkBuild extends Build { "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "commons-daemon" % "commons-daemon" % "1.0.10", "com.ning" % "compress-lzf" % "0.8.4", - "asm" % "asm-all" % "3.3.1", + "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", "com.typesafe.akka" % "akka-actor" % "2.0.3" excludeAll(excludeNetty), diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala index 13d81ec1cf096..0e9aa863b55a6 100644 --- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala @@ -8,7 +8,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.objectweb.asm._ -import org.objectweb.asm.commons.EmptyVisitor import org.objectweb.asm.Opcodes._ @@ -83,7 +82,7 @@ extends ClassLoader(parent) { } class ConstructorCleaner(className: String, cv: ClassVisitor) -extends ClassAdapter(cv) { +extends ClassVisitor(ASM4, cv) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { val mv = cv.visitMethod(access, name, desc, sig, exceptions) From ae7a5da6b31f5bf64f713b3d9bff6e441d8615b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 20 Jun 2013 18:44:46 +0200 Subject: [PATCH 661/696] Fix some dependency issues in SBT build (same will be needed for Maven): - Exclude a version of ASM 3.x that comes from HBase - Don't use a special ASF repo for HBase - Update SLF4J version - Add sbt-dependency-graph plugin so we can easily find dependency trees --- project/SparkBuild.scala | 10 +++++----- project/plugins.sbt | 2 ++ 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b1f3f9a2eaf9d..24c8b734d0c5f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -125,12 +125,13 @@ object SparkBuild extends Build { publishMavenStyle in MavenCompile := true, publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn - ) + ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings - val slf4jVersion = "1.6.1" + val slf4jVersion = "1.7.2" val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeAsm = ExclusionRule(organization = "asm") def coreSettings = sharedSettings ++ Seq( name := "spark-core", @@ -201,11 +202,10 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), libraryDependencies ++= Seq( "com.twitter" % "algebird-core_2.9.2" % "0.1.11", - "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty), + "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty, excludeAsm), "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") @@ -224,7 +224,7 @@ object SparkBuild extends Build { name := "spark-streaming", libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), - "com.github.sgroschupf" % "zkclient" % "0.1", + "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-zeromq" % "2.0.3" excludeAll(excludeNetty) ) diff --git a/project/plugins.sbt b/project/plugins.sbt index d4f2442872363..f806e6648134e 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -16,3 +16,5 @@ addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") + +addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.3") From 52407951541399e60a5292394b3a443a5e7ff281 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 17:38:23 +0800 Subject: [PATCH 662/696] edit according to comments --- core/src/main/scala/spark/RDD.scala | 6 +- core/src/main/scala/spark/Utils.scala | 10 +-- .../scala/spark/executor/TaskMetrics.scala | 2 +- .../scala/spark/scheduler/DAGScheduler.scala | 14 ++-- .../scala/spark/scheduler/JobLogger.scala | 72 +++++++++---------- .../scala/spark/scheduler/SparkListener.scala | 25 +++---- .../spark/scheduler/JobLoggerSuite.scala | 2 +- 7 files changed, 62 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8c0b7ca417add..b17398953b053 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -114,10 +114,10 @@ abstract class RDD[T: ClassManifest]( this } - /**User-defined generator of this RDD*/ - var generator = Utils.getCallSiteInfo._4 + /** User-defined generator of this RDD*/ + var generator = Utils.getCallSiteInfo.firstUserClass - /**reset generator*/ + /** Reset generator*/ def setGenerator(_generator: String) = { generator = _generator } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1630b2b4b01e7..1cfaee79b19eb 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,13 +522,14 @@ private object Utils extends Logging { execute(command, new File(".")) } - + class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, + val firstUserLine: Int, val firstUserClass: String) /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. * This is used, for example, to tell users where in their code each RDD got created. */ - def getCallSiteInfo = { + def getCallSiteInfo: CallSiteInfo = { val trace = Thread.currentThread.getStackTrace().filter( el => (!el.getMethodName.contains("getStackTrace"))) @@ -560,12 +561,13 @@ private object Utils extends Logging { } } } - (lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) + new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) } def formatSparkCallSite = { val callSiteInfo = getCallSiteInfo - "%s at %s:%s".format(callSiteInfo._1, callSiteInfo._2, callSiteInfo._3) + "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, + callSiteInfo.firstUserLine) } /** * Try to find a free port to bind to on the local host. This should ideally never be needed, diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 26e80293657c4..1dc13754f90e4 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -2,7 +2,7 @@ package spark.executor class TaskMetrics extends Serializable { /** - * host's name the task runs on + * Host's name the task runs on */ var hostname: String = _ diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index e281e5a8dbeba..4336f2f36d2e7 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -312,7 +312,8 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => - sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion))) + sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, + completion.reason, completion.taskInfo, completion.taskMetrics))) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -323,8 +324,8 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobCancelled(job, - "SPARKCONTEXT_SHUTDOWN"))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, + JobFailed(error)))) } return true } @@ -527,7 +528,7 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobSuccess(job))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobSucceeded))) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -668,10 +669,11 @@ class DAGScheduler( val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - job.listener.jobFailed(new SparkException("Job failed: " + reason)) + val error = new SparkException("Job failed: " + reason) + job.listener.jobFailed(error) activeJobs -= job resultStageToJob -= resultStage - sparkListeners.foreach(_.onJobEnd(SparkListenerJobFailed(job, failedStage))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 002c5826cbbac..760a0252b7091 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -12,7 +12,7 @@ import spark._ import spark.executor.TaskMetrics import spark.scheduler.cluster.TaskInfo -// used to record runtime information for each job, including RDD graph +// Used to record runtime information for each job, including RDD graph // tasks' start/stop shuffle information and information from outside class JobLogger(val logDirName: String) extends SparkListener with Logging { @@ -49,21 +49,17 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { processStageSubmittedEvent(stage, taskSize) case StageCompleted(stageInfo) => processStageCompletedEvent(stageInfo) - case SparkListenerJobSuccess(job) => - processJobEndEvent(job) - case SparkListenerJobFailed(job, failedStage) => - processJobEndEvent(job, failedStage) - case SparkListenerJobCancelled(job, reason) => - processJobEndEvent(job, reason) - case SparkListenerTaskEnd(event) => - processTaskEndEvent(event) + case SparkListenerJobEnd(job, result) => + processJobEndEvent(job, result) + case SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics) => + processTaskEndEvent(task, reason, taskInfo, taskMetrics) case _ => } } } }.start() - //create a folder for log files, the folder's name is the creation time of the jobLogger + // Create a folder for log files, the folder's name is the creation time of the jobLogger protected def createLogDir() { val dir = new File(logDir + "/" + logDirName + "/") if (dir.exists()) { @@ -244,54 +240,50 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { eventQueue.put(taskEnd) } - protected def processTaskEndEvent(event: CompletionEvent) { + protected def processTaskEndEvent(task: Task[_], reason: TaskEndReason, + taskInfo: TaskInfo, taskMetrics: TaskMetrics) { var taskStatus = "" - event.task match { + task match { case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" } - event.reason match { + reason match { case Success => taskStatus += " STATUS=SUCCESS" - recordTaskMetrics(event.task.stageId, taskStatus, event.taskInfo, event.taskMetrics) + recordTaskMetrics(task.stageId, taskStatus, taskInfo, taskMetrics) case Resubmitted => - taskStatus += " STATUS=RESUBMITTED TID=" + event.taskInfo.taskId + - " STAGE_ID=" + event.task.stageId - stageLogInfo(event.task.stageId, taskStatus) + taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + + " STAGE_ID=" + task.stageId + stageLogInfo(task.stageId, taskStatus) case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => - taskStatus += " STATUS=FETCHFAILED TID=" + event.taskInfo.taskId + " STAGE_ID=" + - event.task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + + taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + + task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId - stageLogInfo(event.task.stageId, taskStatus) + stageLogInfo(task.stageId, taskStatus) case OtherFailure(message) => - taskStatus += " STATUS=FAILURE TID=" + event.taskInfo.taskId + - " STAGE_ID=" + event.task.stageId + " INFO=" + message - stageLogInfo(event.task.stageId, taskStatus) + taskStatus += " STATUS=FAILURE TID=" + taskInfo.taskId + + " STAGE_ID=" + task.stageId + " INFO=" + message + stageLogInfo(task.stageId, taskStatus) case _ => } } - override def onJobEnd(jobEnd: SparkListenerEvents) { + override def onJobEnd(jobEnd: SparkListenerJobEnd) { eventQueue.put(jobEnd) } - protected def processJobEndEvent(job: ActiveJob) { - val info = "JOB_ID=" + job.runId + " STATUS=SUCCESS" - jobLogInfo(job.runId, info) - closeLogWriter(job.runId) - } - - protected def processJobEndEvent(job: ActiveJob, failedStage: Stage) { - val info = "JOB_ID=" + job.runId + " STATUS=FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" - + failedStage.id - jobLogInfo(job.runId, info) - closeLogWriter(job.runId) - } - protected def processJobEndEvent(job: ActiveJob, reason: String) { - var info = "JOB_ID=" + job.runId + " STATUS=CANCELLED REASON=" + reason - jobLogInfo(job.runId, info) + protected def processJobEndEvent(job: ActiveJob, reason: JobResult) { + var info = "JOB_ID=" + job.runId + reason match { + case JobSucceeded => info += " STATUS=SUCCESS" + case JobFailed(exception) => + info += " STATUS=FAILED REASON=" + exception.getMessage.split("\\s+").foreach(info += _ + "_") + case _ => + } + jobLogInfo(job.runId, info.substring(0, info.length - 1).toUpperCase) closeLogWriter(job.runId) } - + protected def recordJobProperties(jobID: Int, properties: Properties) { if(properties != null) { val annotation = properties.getProperty("spark.job.annotation", "") diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 9265261dc1173..bac984b5c942c 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -3,52 +3,49 @@ package spark.scheduler import java.util.Properties import spark.scheduler.cluster.TaskInfo import spark.util.Distribution -import spark.{Utils, Logging, SparkContext, TaskEndReason} +import spark.{Logging, SparkContext, TaskEndReason, Utils} import spark.executor.TaskMetrics - sealed trait SparkListenerEvents case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents -case class SparkListenerTaskEnd(event: CompletionEvent) extends SparkListenerEvents +case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, + taskMetrics: TaskMetrics) extends SparkListenerEvents case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) extends SparkListenerEvents - -case class SparkListenerJobSuccess(job: ActiveJob) extends SparkListenerEvents - -case class SparkListenerJobFailed(job: ActiveJob, failedStage: Stage) extends SparkListenerEvents -case class SparkListenerJobCancelled(job: ActiveJob, reason: String) extends SparkListenerEvents +case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) + extends SparkListenerEvents trait SparkListener { /** - * called when a stage is completed, with information on the completed stage + * Called when a stage is completed, with information on the completed stage */ def onStageCompleted(stageCompleted: StageCompleted) { } /** - * called when a stage is submitted + * Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } /** - * called when a task ends + * Called when a task ends */ def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } /** - * called when a job starts + * Called when a job starts */ def onJobStart(jobStart: SparkListenerJobStart) { } /** - * called when a job ends + * Called when a job ends */ - def onJobEnd(jobEnd: SparkListenerEvents) { } + def onJobEnd(jobEnd: SparkListenerJobEnd) { } } diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index a654bf3ffdc87..4000c4d5209b6 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -87,7 +87,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers var onStageCompletedCount = 0 var onStageSubmittedCount = 0 override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 - override def onJobEnd(jobEnd: SparkListenerEvents) = onJobEndCount += 1 + override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1 override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 From aa7aa587beff22e2db50d2afadd95097856a299a Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 17:48:41 +0800 Subject: [PATCH 663/696] some format modification --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 5 ++--- core/src/main/scala/spark/scheduler/JobLogger.scala | 10 +++++----- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 4336f2f36d2e7..e412baa803c59 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -324,8 +324,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, - JobFailed(error)))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) } return true } @@ -671,9 +670,9 @@ class DAGScheduler( val job = resultStageToJob(resultStage) val error = new SparkException("Job failed: " + reason) job.listener.jobFailed(error) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) activeJobs -= job resultStageToJob -= resultStage - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 760a0252b7091..178bfaba3d923 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -70,7 +70,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - // create a log file for one job, the file name is the jobID + // Create a log file for one job, the file name is the jobID protected def createLogWriter(jobID: Int) { try{ val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) @@ -80,7 +80,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - // close log file for one job, and clean the stage relationship in stageIDToJobID + // Close log file, and clean the stage relationship in stageIDToJobID protected def closeLogWriter(jobID: Int) = jobIDToPrintWriter.get(jobID).foreach { fileWriter => fileWriter.close() @@ -91,7 +91,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { jobIDToStages -= jobID } - // write log information to log file, withTime parameter controls whether to recored + // Write log information to log file, withTime parameter controls whether to recored // time stamp for the information protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { var writeInfo = info @@ -145,7 +145,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - // generate indents and convert to String + // Generate indents and convert to String protected def indentString(indent: Int) = { val sb = new StringBuilder() for (i <- 1 to indent) { @@ -190,7 +190,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.priority, false) } - // record task metrics into job log files + // Record task metrics into job log files protected def recordTaskMetrics(stageID: Int, status: String, taskInfo: TaskInfo, taskMetrics: TaskMetrics) { val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + From 4b9862ac9cf2d00c5245e9a8b0fcb05b82030c98 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 17:55:32 +0800 Subject: [PATCH 664/696] small format modification --- core/src/main/scala/spark/Utils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1cfaee79b19eb..96d86647f82dc 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,8 +522,8 @@ private object Utils extends Logging { execute(command, new File(".")) } - class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, - val firstUserLine: Int, val firstUserClass: String) + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, + val firstUserLine: Int, val firstUserClass: String) /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. From 2fc794a6c7f1b86e5c0103a9c82af2be7fafb347 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 18:21:35 +0800 Subject: [PATCH 665/696] small modify in DAGScheduler --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index e412baa803c59..f7d60be5dbfc2 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -289,7 +289,6 @@ class DAGScheduler( val finalStage = newStage(finalRDD, None, runId) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() - sparkListeners.foreach(_.onJobStart(SparkListenerJobStart(job, properties))) logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") @@ -299,6 +298,7 @@ class DAGScheduler( // Compute very short actions like first() or take() with no parent stages locally. runLocally(job) } else { + sparkListeners.foreach(_.onJobStart(SparkListenerJobStart(job, properties))) idToActiveJob(runId) = job activeJobs += job resultStageToJob(finalStage) = job From 40afe0d2a5562738ef2ff37ed1d448ae2d0cc927 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sun, 10 Mar 2013 13:54:46 -0700 Subject: [PATCH 666/696] Add Python timing instrumentation --- .../main/scala/spark/api/python/PythonRDD.scala | 12 ++++++++++++ python/pyspark/serializers.py | 4 ++++ python/pyspark/worker.py | 16 +++++++++++++++- 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 807119ca8c08a..e9978d713fe4a 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -47,6 +47,7 @@ private[spark] class PythonRDD[T: ClassManifest]( currentEnvVars.put(variable, value) } + val startTime = System.currentTimeMillis val proc = pb.start() val env = SparkEnv.get @@ -108,6 +109,17 @@ private[spark] class PythonRDD[T: ClassManifest]( val obj = new Array[Byte](length) stream.readFully(obj) obj + case -3 => + // Timing data from child + val bootTime = stream.readLong() + val initTime = stream.readLong() + val finishTime = stream.readLong() + val boot = bootTime - startTime + val init = initTime - bootTime + val finish = finishTime - initTime + val total = finishTime - startTime + logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) + read case -2 => // Signals that an exception has been thrown in python val exLength = stream.readInt() diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 115cf28cc2ba4..5a95144983aec 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -46,6 +46,10 @@ def read_long(stream): return struct.unpack("!q", length)[0] +def write_long(value, stream): + stream.write(struct.pack("!q", value)) + + def read_int(stream): length = stream.read(4) if length == "": diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 812e7a9da5a03..4c33ae49dcc95 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,6 +1,8 @@ """ Worker that receives input from Piped RDD. """ +import time +preboot_time = time.time() import os import sys import traceback @@ -12,7 +14,7 @@ from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, read_with_length, write_int, \ - read_long, read_int, dump_pickle, load_pickle, read_from_pickle_file + read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file # Redirect stdout to stderr so that users must return values from functions. @@ -24,7 +26,16 @@ def load_obj(): return load_pickle(standard_b64decode(sys.stdin.readline().strip())) +def report_times(preboot, boot, init, finish): + write_int(-3, old_stdout) + write_long(1000 * preboot, old_stdout) + write_long(1000 * boot, old_stdout) + write_long(1000 * init, old_stdout) + write_long(1000 * finish, old_stdout) + + def main(): + boot_time = time.time() split_index = read_int(sys.stdin) spark_files_dir = load_pickle(read_with_length(sys.stdin)) SparkFiles._root_directory = spark_files_dir @@ -41,6 +52,7 @@ def main(): dumps = lambda x: x else: dumps = dump_pickle + init_time = time.time() iterator = read_from_pickle_file(sys.stdin) try: for obj in func(split_index, iterator): @@ -49,6 +61,8 @@ def main(): write_int(-2, old_stdout) write_with_length(traceback.format_exc(), old_stdout) sys.exit(-1) + finish_time = time.time() + report_times(preboot_time, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output write_int(-1, old_stdout) for aid, accum in _accumulatorRegistry.items(): From c79a6078c34c207ad9f9910252f5849424828bf1 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 6 May 2013 16:34:30 -0700 Subject: [PATCH 667/696] Prefork Python worker processes --- core/src/main/scala/spark/SparkEnv.scala | 11 ++ .../scala/spark/api/python/PythonRDD.scala | 66 ++++------- .../scala/spark/api/python/PythonWorker.scala | 89 ++++++++++++++ python/pyspark/daemon.py | 109 ++++++++++++++++++ python/pyspark/worker.py | 61 +++++----- 5 files changed, 263 insertions(+), 73 deletions(-) create mode 100644 core/src/main/scala/spark/api/python/PythonWorker.scala create mode 100644 python/pyspark/daemon.py diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index be1a04d619c1b..5691e24c320f6 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,5 +1,8 @@ package spark +import collection.mutable +import serializer.Serializer + import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider @@ -9,6 +12,7 @@ import spark.storage.BlockManagerMaster import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils +import spark.api.python.PythonWorker /** @@ -37,6 +41,8 @@ class SparkEnv ( // If executorId is NOT found, return defaultHostPort var executorIdToHostPort: Option[(String, String) => String]) { + private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorker]() + def stop() { httpFileServer.stop() mapOutputTracker.stop() @@ -50,6 +56,11 @@ class SparkEnv ( actorSystem.awaitTermination() } + def getPythonWorker(pythonExec: String, envVars: Map[String, String]): PythonWorker = { + synchronized { + pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorker(pythonExec, envVars)) + } + } def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = { val env = SparkEnv.get diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index e9978d713fe4a..e5acc54c016aa 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -2,10 +2,9 @@ package spark.api.python import java.io._ import java.net._ -import java.util.{List => JList, ArrayList => JArrayList, Collections} +import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ -import scala.io.Source import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import spark.broadcast.Broadcast @@ -16,7 +15,7 @@ import spark.rdd.PipedRDD private[spark] class PythonRDD[T: ClassManifest]( parent: RDD[T], command: Seq[String], - envVars: java.util.Map[String, String], + envVars: JMap[String, String], preservePartitoning: Boolean, pythonExec: String, broadcastVars: JList[Broadcast[Array[Byte]]], @@ -25,7 +24,7 @@ private[spark] class PythonRDD[T: ClassManifest]( // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) - def this(parent: RDD[T], command: String, envVars: java.util.Map[String, String], + def this(parent: RDD[T], command: String, envVars: JMap[String, String], preservePartitoning: Boolean, pythonExec: String, broadcastVars: JList[Broadcast[Array[Byte]]], accumulator: Accumulator[JList[Array[Byte]]]) = @@ -36,36 +35,18 @@ private[spark] class PythonRDD[T: ClassManifest]( override val partitioner = if (preservePartitoning) parent.partitioner else None - override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { - val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME") - - val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/python/pyspark/worker.py")) - // Add the environmental variables to the process. - val currentEnvVars = pb.environment() - - for ((variable, value) <- envVars) { - currentEnvVars.put(variable, value) - } + override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis - val proc = pb.start() + val worker = SparkEnv.get.getPythonWorker(pythonExec, envVars.toMap).create val env = SparkEnv.get - // Start a thread to print the process's stderr to ours - new Thread("stderr reader for " + pythonExec) { - override def run() { - for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { - System.err.println(line) - } - } - }.start() - // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val out = new PrintWriter(proc.getOutputStream) - val dOut = new DataOutputStream(proc.getOutputStream) + val out = new PrintWriter(worker.getOutputStream) + val dOut = new DataOutputStream(worker.getOutputStream) // Partition index dOut.writeInt(split.index) // sparkFilesDir @@ -89,16 +70,21 @@ private[spark] class PythonRDD[T: ClassManifest]( } dOut.flush() out.flush() - proc.getOutputStream.close() + worker.shutdownOutput() } }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(proc.getInputStream) + val stream = new DataInputStream(worker.getInputStream) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj - _nextObj = read() + if (hasNext) { + // FIXME: can deadlock if worker is waiting for us to + // respond to current message (currently irrelevant because + // output is shutdown before we read any input) + _nextObj = read() + } obj } @@ -110,7 +96,7 @@ private[spark] class PythonRDD[T: ClassManifest]( stream.readFully(obj) obj case -3 => - // Timing data from child + // Timing data from worker val bootTime = stream.readLong() val initTime = stream.readLong() val finishTime = stream.readLong() @@ -127,23 +113,21 @@ private[spark] class PythonRDD[T: ClassManifest]( stream.readFully(obj) throw new PythonException(new String(obj)) case -1 => - // We've finished the data section of the output, but we can still read some - // accumulator updates; let's do that, breaking when we get EOFException - while (true) { - val len2 = stream.readInt() + // We've finished the data section of the output, but we can still + // read some accumulator updates; let's do that, breaking when we + // get a negative length record. + var len2 = stream.readInt + while (len2 >= 0) { val update = new Array[Byte](len2) stream.readFully(update) accumulator += Collections.singletonList(update) + len2 = stream.readInt } new Array[Byte](0) } } catch { case eof: EOFException => { - val exitStatus = proc.waitFor() - if (exitStatus != 0) { - throw new Exception("Subprocess exited with status " + exitStatus) - } - new Array[Byte](0) + throw new SparkException("Python worker exited unexpectedly (crashed)", eof) } case e => throw e } @@ -171,7 +155,7 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends override def compute(split: Partition, context: TaskContext) = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (a, b) - case x => throw new Exception("PairwiseRDD: unexpected value: " + x) + case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) } val asJavaPairRDD : JavaPairRDD[Array[Byte], Array[Byte]] = JavaPairRDD.fromRDD(this) } @@ -227,7 +211,7 @@ private[spark] object PythonRDD { dOut.write(s) dOut.writeByte(Pickle.STOP) } else { - throw new Exception("Unexpected RDD type") + throw new SparkException("Unexpected RDD type") } } diff --git a/core/src/main/scala/spark/api/python/PythonWorker.scala b/core/src/main/scala/spark/api/python/PythonWorker.scala new file mode 100644 index 0000000000000..8ee3c6884f98f --- /dev/null +++ b/core/src/main/scala/spark/api/python/PythonWorker.scala @@ -0,0 +1,89 @@ +package spark.api.python + +import java.io.DataInputStream +import java.net.{Socket, SocketException, InetAddress} + +import scala.collection.JavaConversions._ + +import spark._ + +private[spark] class PythonWorker(pythonExec: String, envVars: Map[String, String]) + extends Logging { + var daemon: Process = null + val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) + var daemonPort: Int = 0 + + def create(): Socket = { + synchronized { + // Start the daemon if it hasn't been started + startDaemon + + // Attempt to connect, restart and retry once if it fails + try { + new Socket(daemonHost, daemonPort) + } catch { + case exc: SocketException => { + logWarning("Python daemon unexpectedly quit, attempting to restart") + stopDaemon + startDaemon + new Socket(daemonHost, daemonPort) + } + case e => throw e + } + } + } + + private def startDaemon() { + synchronized { + // Is it already running? + if (daemon != null) { + return + } + + try { + // Create and start the daemon + val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") + val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) + val workerEnv = pb.environment() + workerEnv.putAll(envVars) + daemon = pb.start() + daemonPort = new DataInputStream(daemon.getInputStream).readInt + + // Redirect the stderr to ours + new Thread("stderr reader for " + pythonExec) { + override def run() { + // FIXME HACK: We copy the stream on the level of bytes to + // attempt to dodge encoding problems. + val in = daemon.getErrorStream + var buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + }.start() + } catch { + case e => { + stopDaemon + throw e + } + } + + // Important: don't close daemon's stdin (daemon.getOutputStream) so it can correctly + // detect our disappearance. + } + } + + private def stopDaemon() { + synchronized { + // Request shutdown of existing daemon by sending SIGTERM + if (daemon != null) { + daemon.destroy + } + + daemon = null + daemonPort = 0 + } + } +} diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py new file mode 100644 index 0000000000000..642f30b2b9644 --- /dev/null +++ b/python/pyspark/daemon.py @@ -0,0 +1,109 @@ +import os +import sys +import multiprocessing +from errno import EINTR, ECHILD +from socket import socket, AF_INET, SOCK_STREAM, SOMAXCONN +from signal import signal, SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN +from pyspark.worker import main as worker_main +from pyspark.serializers import write_int + +try: + POOLSIZE = multiprocessing.cpu_count() +except NotImplementedError: + POOLSIZE = 4 + +should_exit = False + + +def worker(listen_sock): + # Redirect stdout to stderr + os.dup2(2, 1) + + # Manager sends SIGHUP to request termination of workers in the pool + def handle_sighup(signum, frame): + global should_exit + should_exit = True + signal(SIGHUP, handle_sighup) + + while not should_exit: + # Wait until a client arrives or we have to exit + sock = None + while not should_exit and sock is None: + try: + sock, addr = listen_sock.accept() + except EnvironmentError as err: + if err.errno != EINTR: + raise + + if sock is not None: + # Fork a child to handle the client + if os.fork() == 0: + # Leave the worker pool + signal(SIGHUP, SIG_DFL) + listen_sock.close() + # Handle the client then exit + sockfile = sock.makefile() + worker_main(sockfile, sockfile) + sockfile.close() + sock.close() + os._exit(0) + else: + sock.close() + + assert should_exit + os._exit(0) + + +def manager(): + # Create a new process group to corral our children + os.setpgid(0, 0) + + # Create a listening socket on the AF_INET loopback interface + listen_sock = socket(AF_INET, SOCK_STREAM) + listen_sock.bind(('127.0.0.1', 0)) + listen_sock.listen(max(1024, 2 * POOLSIZE, SOMAXCONN)) + listen_host, listen_port = listen_sock.getsockname() + write_int(listen_port, sys.stdout) + + # Launch initial worker pool + for idx in range(POOLSIZE): + if os.fork() == 0: + worker(listen_sock) + raise RuntimeError("worker() unexpectedly returned") + listen_sock.close() + + def shutdown(): + global should_exit + os.kill(0, SIGHUP) + should_exit = True + + # Gracefully exit on SIGTERM, don't die on SIGHUP + signal(SIGTERM, lambda signum, frame: shutdown()) + signal(SIGHUP, SIG_IGN) + + # Cleanup zombie children + def handle_sigchld(signum, frame): + try: + pid, status = os.waitpid(0, os.WNOHANG) + if (pid, status) != (0, 0) and not should_exit: + raise RuntimeError("pool member crashed: %s, %s" % (pid, status)) + except EnvironmentError as err: + if err.errno not in (ECHILD, EINTR): + raise + signal(SIGCHLD, handle_sigchld) + + # Initialization complete + sys.stdout.close() + while not should_exit: + try: + # Spark tells us to exit by closing stdin + if sys.stdin.read() == '': + shutdown() + except EnvironmentError as err: + if err.errno != EINTR: + shutdown() + raise + + +if __name__ == '__main__': + manager() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4c33ae49dcc95..94d612ea6ed92 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,10 +1,9 @@ """ Worker that receives input from Piped RDD. """ -import time -preboot_time = time.time() import os import sys +import time import traceback from base64 import standard_b64decode # CloudPickler needs to be imported so that depicklers are registered using the @@ -17,57 +16,55 @@ read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file -# Redirect stdout to stderr so that users must return values from functions. -old_stdout = os.fdopen(os.dup(1), 'w') -os.dup2(2, 1) - - -def load_obj(): - return load_pickle(standard_b64decode(sys.stdin.readline().strip())) +def load_obj(infile): + return load_pickle(standard_b64decode(infile.readline().strip())) -def report_times(preboot, boot, init, finish): - write_int(-3, old_stdout) - write_long(1000 * preboot, old_stdout) - write_long(1000 * boot, old_stdout) - write_long(1000 * init, old_stdout) - write_long(1000 * finish, old_stdout) +def report_times(outfile, boot, init, finish): + write_int(-3, outfile) + write_long(1000 * boot, outfile) + write_long(1000 * init, outfile) + write_long(1000 * finish, outfile) -def main(): +def main(infile, outfile): boot_time = time.time() - split_index = read_int(sys.stdin) - spark_files_dir = load_pickle(read_with_length(sys.stdin)) + split_index = read_int(infile) + spark_files_dir = load_pickle(read_with_length(infile)) SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True sys.path.append(spark_files_dir) - num_broadcast_variables = read_int(sys.stdin) + num_broadcast_variables = read_int(infile) for _ in range(num_broadcast_variables): - bid = read_long(sys.stdin) - value = read_with_length(sys.stdin) + bid = read_long(infile) + value = read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, load_pickle(value)) - func = load_obj() - bypassSerializer = load_obj() + func = load_obj(infile) + bypassSerializer = load_obj(infile) if bypassSerializer: dumps = lambda x: x else: dumps = dump_pickle init_time = time.time() - iterator = read_from_pickle_file(sys.stdin) + iterator = read_from_pickle_file(infile) try: for obj in func(split_index, iterator): - write_with_length(dumps(obj), old_stdout) + write_with_length(dumps(obj), outfile) except Exception as e: - write_int(-2, old_stdout) - write_with_length(traceback.format_exc(), old_stdout) - sys.exit(-1) + write_int(-2, outfile) + write_with_length(traceback.format_exc(), outfile) + raise finish_time = time.time() - report_times(preboot_time, boot_time, init_time, finish_time) + report_times(outfile, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output - write_int(-1, old_stdout) + write_int(-1, outfile) for aid, accum in _accumulatorRegistry.items(): - write_with_length(dump_pickle((aid, accum._value)), old_stdout) + write_with_length(dump_pickle((aid, accum._value)), outfile) + write_int(-1, outfile) if __name__ == '__main__': - main() + # Redirect stdout to stderr so that users must return values from functions. + old_stdout = os.fdopen(os.dup(1), 'w') + os.dup2(2, 1) + main(sys.stdin, old_stdout) From 62c4781400dd908c2fccdcebf0dc816ff0cb8ed4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 10 May 2013 15:48:48 -0700 Subject: [PATCH 668/696] Add tests and fixes for Python daemon shutdown --- core/src/main/scala/spark/SparkEnv.scala | 1 + .../scala/spark/api/python/PythonWorker.scala | 4 ++ python/pyspark/daemon.py | 46 ++++++++++--------- python/pyspark/tests.py | 43 +++++++++++++++++ python/pyspark/worker.py | 2 + 5 files changed, 74 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 5691e24c320f6..5b55d4521230b 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -44,6 +44,7 @@ class SparkEnv ( private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorker]() def stop() { + pythonWorkers.foreach { case(key, worker) => worker.stop() } httpFileServer.stop() mapOutputTracker.stop() shuffleFetcher.stop() diff --git a/core/src/main/scala/spark/api/python/PythonWorker.scala b/core/src/main/scala/spark/api/python/PythonWorker.scala index 8ee3c6884f98f..74c8c6d37a811 100644 --- a/core/src/main/scala/spark/api/python/PythonWorker.scala +++ b/core/src/main/scala/spark/api/python/PythonWorker.scala @@ -33,6 +33,10 @@ private[spark] class PythonWorker(pythonExec: String, envVars: Map[String, Strin } } + def stop() { + stopDaemon + } + private def startDaemon() { synchronized { // Is it already running? diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 642f30b2b9644..ab9c19df578f8 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -12,7 +12,7 @@ except NotImplementedError: POOLSIZE = 4 -should_exit = False +should_exit = multiprocessing.Event() def worker(listen_sock): @@ -21,14 +21,13 @@ def worker(listen_sock): # Manager sends SIGHUP to request termination of workers in the pool def handle_sighup(signum, frame): - global should_exit - should_exit = True + assert should_exit.is_set() signal(SIGHUP, handle_sighup) - while not should_exit: + while not should_exit.is_set(): # Wait until a client arrives or we have to exit sock = None - while not should_exit and sock is None: + while not should_exit.is_set() and sock is None: try: sock, addr = listen_sock.accept() except EnvironmentError as err: @@ -36,8 +35,8 @@ def handle_sighup(signum, frame): raise if sock is not None: - # Fork a child to handle the client - if os.fork() == 0: + # Fork to handle the client + if os.fork() != 0: # Leave the worker pool signal(SIGHUP, SIG_DFL) listen_sock.close() @@ -50,7 +49,7 @@ def handle_sighup(signum, frame): else: sock.close() - assert should_exit + assert should_exit.is_set() os._exit(0) @@ -73,9 +72,7 @@ def manager(): listen_sock.close() def shutdown(): - global should_exit - os.kill(0, SIGHUP) - should_exit = True + should_exit.set() # Gracefully exit on SIGTERM, don't die on SIGHUP signal(SIGTERM, lambda signum, frame: shutdown()) @@ -85,8 +82,8 @@ def shutdown(): def handle_sigchld(signum, frame): try: pid, status = os.waitpid(0, os.WNOHANG) - if (pid, status) != (0, 0) and not should_exit: - raise RuntimeError("pool member crashed: %s, %s" % (pid, status)) + if status != 0 and not should_exit.is_set(): + raise RuntimeError("worker crashed: %s, %s" % (pid, status)) except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): raise @@ -94,15 +91,20 @@ def handle_sigchld(signum, frame): # Initialization complete sys.stdout.close() - while not should_exit: - try: - # Spark tells us to exit by closing stdin - if sys.stdin.read() == '': - shutdown() - except EnvironmentError as err: - if err.errno != EINTR: - shutdown() - raise + try: + while not should_exit.is_set(): + try: + # Spark tells us to exit by closing stdin + if os.read(0, 512) == '': + shutdown() + except EnvironmentError as err: + if err.errno != EINTR: + shutdown() + raise + finally: + should_exit.set() + # Send SIGHUP to notify workers of shutdown + os.kill(0, SIGHUP) if __name__ == '__main__': diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 6a1962d26795d..1e34d473650bf 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -12,6 +12,7 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.java_gateway import SPARK_HOME +from pyspark.serializers import read_int class PySparkTestCase(unittest.TestCase): @@ -117,5 +118,47 @@ def func(x): self.sc.parallelize([1]).foreach(func) +class TestDaemon(unittest.TestCase): + def connect(self, port): + from socket import socket, AF_INET, SOCK_STREAM + sock = socket(AF_INET, SOCK_STREAM) + sock.connect(('127.0.0.1', port)) + # send a split index of -1 to shutdown the worker + sock.send("\xFF\xFF\xFF\xFF") + sock.close() + return True + + def do_termination_test(self, terminator): + from subprocess import Popen, PIPE + from errno import ECONNREFUSED + + # start daemon + daemon_path = os.path.join(os.path.dirname(__file__), "daemon.py") + daemon = Popen([sys.executable, daemon_path], stdin=PIPE, stdout=PIPE) + + # read the port number + port = read_int(daemon.stdout) + + # daemon should accept connections + self.assertTrue(self.connect(port)) + + # request shutdown + terminator(daemon) + time.sleep(1) + + # daemon should no longer accept connections + with self.assertRaises(EnvironmentError) as trap: + self.connect(port) + self.assertEqual(trap.exception.errno, ECONNREFUSED) + + def test_termination_stdin(self): + """Ensure that daemon and workers terminate when stdin is closed.""" + self.do_termination_test(lambda daemon: daemon.stdin.close()) + + def test_termination_sigterm(self): + """Ensure that daemon and workers terminate on SIGTERM.""" + from signal import SIGTERM + self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 94d612ea6ed92..f76ee3c236db5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,6 +30,8 @@ def report_times(outfile, boot, init, finish): def main(infile, outfile): boot_time = time.time() split_index = read_int(infile) + if split_index == -1: # for unit tests + return spark_files_dir = load_pickle(read_with_length(infile)) SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True From edb18ca928c988a713b9228bb74af1737f2b614b Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 13 May 2013 08:53:47 -0700 Subject: [PATCH 669/696] Rename PythonWorker to PythonWorkerFactory --- core/src/main/scala/spark/SparkEnv.scala | 8 +++---- .../scala/spark/api/python/PythonRDD.scala | 2 +- ...Worker.scala => PythonWorkerFactory.scala} | 22 ++++++++++--------- 3 files changed, 17 insertions(+), 15 deletions(-) rename core/src/main/scala/spark/api/python/{PythonWorker.scala => PythonWorkerFactory.scala} (76%) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 5b55d4521230b..0a23c4565892e 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -12,7 +12,7 @@ import spark.storage.BlockManagerMaster import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils -import spark.api.python.PythonWorker +import spark.api.python.PythonWorkerFactory /** @@ -41,7 +41,7 @@ class SparkEnv ( // If executorId is NOT found, return defaultHostPort var executorIdToHostPort: Option[(String, String) => String]) { - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorker]() + private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() def stop() { pythonWorkers.foreach { case(key, worker) => worker.stop() } @@ -57,9 +57,9 @@ class SparkEnv ( actorSystem.awaitTermination() } - def getPythonWorker(pythonExec: String, envVars: Map[String, String]): PythonWorker = { + def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { - pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorker(pythonExec, envVars)) + pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index e5acc54c016aa..3c48071b3f340 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -38,8 +38,8 @@ private[spark] class PythonRDD[T: ClassManifest]( override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis - val worker = SparkEnv.get.getPythonWorker(pythonExec, envVars.toMap).create val env = SparkEnv.get + val worker = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + pythonExec) { diff --git a/core/src/main/scala/spark/api/python/PythonWorker.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala similarity index 76% rename from core/src/main/scala/spark/api/python/PythonWorker.scala rename to core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 74c8c6d37a811..ebbd226b3eab5 100644 --- a/core/src/main/scala/spark/api/python/PythonWorker.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -1,13 +1,13 @@ package spark.api.python -import java.io.DataInputStream +import java.io.{DataInputStream, IOException} import java.net.{Socket, SocketException, InetAddress} import scala.collection.JavaConversions._ import spark._ -private[spark] class PythonWorker(pythonExec: String, envVars: Map[String, String]) +private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) extends Logging { var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) @@ -56,14 +56,16 @@ private[spark] class PythonWorker(pythonExec: String, envVars: Map[String, Strin // Redirect the stderr to ours new Thread("stderr reader for " + pythonExec) { override def run() { - // FIXME HACK: We copy the stream on the level of bytes to - // attempt to dodge encoding problems. - val in = daemon.getErrorStream - var buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME HACK: We copy the stream on the level of bytes to + // attempt to dodge encoding problems. + val in = daemon.getErrorStream + var buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } } } }.start() From 7c5ff733ee1d3729b4b26f7c5542ca00c4d64139 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 23 May 2013 11:50:24 -0700 Subject: [PATCH 670/696] PySpark daemon: fix deadlock, improve error handling --- python/pyspark/daemon.py | 67 ++++++++++++++++++++++++++++++---------- 1 file changed, 50 insertions(+), 17 deletions(-) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index ab9c19df578f8..2b5e9b35811c8 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,6 +1,7 @@ import os import sys import multiprocessing +from ctypes import c_bool from errno import EINTR, ECHILD from socket import socket, AF_INET, SOCK_STREAM, SOMAXCONN from signal import signal, SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN @@ -12,7 +13,12 @@ except NotImplementedError: POOLSIZE = 4 -should_exit = multiprocessing.Event() +exit_flag = multiprocessing.Value(c_bool, False) + + +def should_exit(): + global exit_flag + return exit_flag.value def worker(listen_sock): @@ -20,14 +26,29 @@ def worker(listen_sock): os.dup2(2, 1) # Manager sends SIGHUP to request termination of workers in the pool - def handle_sighup(signum, frame): - assert should_exit.is_set() + def handle_sighup(*args): + assert should_exit() signal(SIGHUP, handle_sighup) - while not should_exit.is_set(): + # Cleanup zombie children + def handle_sigchld(*args): + pid = status = None + try: + while (pid, status) != (0, 0): + pid, status = os.waitpid(0, os.WNOHANG) + except EnvironmentError as err: + if err.errno == EINTR: + # retry + handle_sigchld() + elif err.errno != ECHILD: + raise + signal(SIGCHLD, handle_sigchld) + + # Handle clients + while not should_exit(): # Wait until a client arrives or we have to exit sock = None - while not should_exit.is_set() and sock is None: + while not should_exit() and sock is None: try: sock, addr = listen_sock.accept() except EnvironmentError as err: @@ -35,8 +56,10 @@ def handle_sighup(signum, frame): raise if sock is not None: - # Fork to handle the client - if os.fork() != 0: + # Fork a child to handle the client. + # The client is handled in the child so that the manager + # never receives SIGCHLD unless a worker crashes. + if os.fork() == 0: # Leave the worker pool signal(SIGHUP, SIG_DFL) listen_sock.close() @@ -49,8 +72,18 @@ def handle_sighup(signum, frame): else: sock.close() - assert should_exit.is_set() - os._exit(0) + +def launch_worker(listen_sock): + if os.fork() == 0: + try: + worker(listen_sock) + except Exception as err: + import traceback + traceback.print_exc() + os._exit(1) + else: + assert should_exit() + os._exit(0) def manager(): @@ -66,23 +99,22 @@ def manager(): # Launch initial worker pool for idx in range(POOLSIZE): - if os.fork() == 0: - worker(listen_sock) - raise RuntimeError("worker() unexpectedly returned") + launch_worker(listen_sock) listen_sock.close() def shutdown(): - should_exit.set() + global exit_flag + exit_flag.value = True # Gracefully exit on SIGTERM, don't die on SIGHUP signal(SIGTERM, lambda signum, frame: shutdown()) signal(SIGHUP, SIG_IGN) # Cleanup zombie children - def handle_sigchld(signum, frame): + def handle_sigchld(*args): try: pid, status = os.waitpid(0, os.WNOHANG) - if status != 0 and not should_exit.is_set(): + if status != 0 and not should_exit(): raise RuntimeError("worker crashed: %s, %s" % (pid, status)) except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): @@ -92,7 +124,7 @@ def handle_sigchld(signum, frame): # Initialization complete sys.stdout.close() try: - while not should_exit.is_set(): + while not should_exit(): try: # Spark tells us to exit by closing stdin if os.read(0, 512) == '': @@ -102,7 +134,8 @@ def handle_sigchld(signum, frame): shutdown() raise finally: - should_exit.set() + signal(SIGTERM, SIG_DFL) + exit_flag.value = True # Send SIGHUP to notify workers of shutdown os.kill(0, SIGHUP) From 1ba3c173034c37ef99fc312c84943d2ab8885670 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 20 Jun 2013 12:49:10 -0400 Subject: [PATCH 671/696] use parens when calling method with side-effects --- core/src/main/scala/spark/SparkEnv.scala | 2 +- .../main/scala/spark/api/python/PythonRDD.scala | 4 ++-- .../spark/api/python/PythonWorkerFactory.scala | 14 +++++++------- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 0a23c4565892e..7ccde2e8182f3 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -59,7 +59,7 @@ class SparkEnv ( def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { - pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create + pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create() } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 3c48071b3f340..63140cf37f5bb 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -116,12 +116,12 @@ private[spark] class PythonRDD[T: ClassManifest]( // We've finished the data section of the output, but we can still // read some accumulator updates; let's do that, breaking when we // get a negative length record. - var len2 = stream.readInt + var len2 = stream.readInt() while (len2 >= 0) { val update = new Array[Byte](len2) stream.readFully(update) accumulator += Collections.singletonList(update) - len2 = stream.readInt + len2 = stream.readInt() } new Array[Byte](0) } diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index ebbd226b3eab5..8844411d738af 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -16,7 +16,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String def create(): Socket = { synchronized { // Start the daemon if it hasn't been started - startDaemon + startDaemon() // Attempt to connect, restart and retry once if it fails try { @@ -24,8 +24,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } catch { case exc: SocketException => { logWarning("Python daemon unexpectedly quit, attempting to restart") - stopDaemon - startDaemon + stopDaemon() + startDaemon() new Socket(daemonHost, daemonPort) } case e => throw e @@ -34,7 +34,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } def stop() { - stopDaemon + stopDaemon() } private def startDaemon() { @@ -51,7 +51,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val workerEnv = pb.environment() workerEnv.putAll(envVars) daemon = pb.start() - daemonPort = new DataInputStream(daemon.getInputStream).readInt + daemonPort = new DataInputStream(daemon.getInputStream).readInt() // Redirect the stderr to ours new Thread("stderr reader for " + pythonExec) { @@ -71,7 +71,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String }.start() } catch { case e => { - stopDaemon + stopDaemon() throw e } } @@ -85,7 +85,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String synchronized { // Request shutdown of existing daemon by sending SIGTERM if (daemon != null) { - daemon.destroy + daemon.destroy() } daemon = null From c75bed0eebb1f937db02eb98deecd380724f747d Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 21 Jun 2013 12:13:48 -0400 Subject: [PATCH 672/696] Fix reporting of PySpark exceptions --- python/pyspark/daemon.py | 22 ++++++++++++++++++---- python/pyspark/worker.py | 2 +- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 2b5e9b35811c8..78a2da1e18ea4 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -21,6 +21,15 @@ def should_exit(): return exit_flag.value +def compute_real_exit_code(exit_code): + # SystemExit's code can be integer or string, but os._exit only accepts integers + import numbers + if isinstance(exit_code, numbers.Integral): + return exit_code + else: + return 1 + + def worker(listen_sock): # Redirect stdout to stderr os.dup2(2, 1) @@ -65,10 +74,15 @@ def handle_sigchld(*args): listen_sock.close() # Handle the client then exit sockfile = sock.makefile() - worker_main(sockfile, sockfile) - sockfile.close() - sock.close() - os._exit(0) + exit_code = 0 + try: + worker_main(sockfile, sockfile) + except SystemExit as exc: + exit_code = exc.code + finally: + sockfile.close() + sock.close() + os._exit(compute_real_exit_code(exit_code)) else: sock.close() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index f76ee3c236db5..379bbfd4c2cfe 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -55,7 +55,7 @@ def main(infile, outfile): except Exception as e: write_int(-2, outfile) write_with_length(traceback.format_exc(), outfile) - raise + sys.exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output From b350f34703d4f29bbd0e603df852f7aae230b2a2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 07:48:20 -0700 Subject: [PATCH 673/696] Increase memory for tests to prevent a crash on JDK 7 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 24c8b734d0c5f..faf6e2ae8e8b0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -54,7 +54,7 @@ object SparkBuild extends Build { // Fork new JVMs for tests and set Java options for those fork := true, - javaOptions += "-Xmx2g", + javaOptions += "-Xmx2500m", // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), From d92d3f7938dec954ea31de232f50cafd4b644065 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 10:24:19 -0700 Subject: [PATCH 674/696] Fix resolution of example code with Maven builds --- run | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/run b/run index c0065c53f1709..e656e38ccf275 100755 --- a/run +++ b/run @@ -132,10 +132,14 @@ if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" fi CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" +# Add the shaded JAR for Maven builds if [ -e $REPL_BIN_DIR/target ]; then for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH="$CLASSPATH:$jar" done + # The shaded JAR doesn't contain examples, so include those separately + EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do @@ -148,9 +152,9 @@ if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; # Use the JAR from the SBT build export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then +if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` fi # Add hadoop conf dir - else FileSystem.*, etc fail ! From dff19884490c1500bffcd921088456e022e7ef43 Mon Sep 17 00:00:00 2001 From: Jon Hartlaub Date: Wed, 23 Jan 2013 10:39:52 -0800 Subject: [PATCH 675/696] Fixed memory leak in DAGScheduler- needs more tests under error conditions. --- .../scala/spark/scheduler/DAGScheduler.scala | 26 ++++- .../test/scala/spark/DAGSchedulerSuite.scala | 94 +++++++++++++++++++ 2 files changed, 115 insertions(+), 5 deletions(-) create mode 100644 core/src/test/scala/spark/DAGSchedulerSuite.scala diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f7d60be5dbfc2..0bf586bba519e 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -74,6 +74,8 @@ class DAGScheduler( val nextRunId = new AtomicInteger(0) + val runIdToStageIds = new HashMap[Int, HashSet[Int]] + val nextStageId = new AtomicInteger(0) val idToStage = new TimeStampedHashMap[Int, Stage] @@ -161,6 +163,8 @@ class DAGScheduler( val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) idToStage(id) = stage stageToInfos(stage) = StageInfo(stage) + val stageIdSet = runIdToStageIds.getOrElseUpdate(priority, new HashSet) + stageIdSet += id stage } @@ -378,11 +382,7 @@ class DAGScheduler( logDebug("Got event of type " + event.getClass.getName) } - if (event != null) { - if (processEvent(event)) { - return - } - } + if (event != null && processEvent(event)) return val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability // Periodically resubmit failed stages if some map output fetches have failed and we have @@ -528,6 +528,7 @@ class DAGScheduler( resultStageToJob -= stage markStageAsFinished(stage) sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobSucceeded))) + removeStages(job) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -673,6 +674,8 @@ class DAGScheduler( sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) activeJobs -= job resultStageToJob -= resultStage + removeStages(job) + job.listener.jobFailed(new SparkException("Job failed: " + reason)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -749,6 +752,19 @@ class DAGScheduler( logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) } + def removeStages(job: ActiveJob) = { + runIdToStageIds(job.runId).foreach(stageId => { + idToStage.get(stageId).map( stage => { + pendingTasks -= stage + waiting -= stage + running -= stage + failed -= stage + }) + idToStage -= stageId + }) + runIdToStageIds -= job.runId + } + def stop() { eventQueue.put(StopDAGScheduler) metadataCleaner.cancel() diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala new file mode 100644 index 0000000000000..99fd83e347f87 --- /dev/null +++ b/core/src/test/scala/spark/DAGSchedulerSuite.scala @@ -0,0 +1,94 @@ +/* +* Copyright (c) Clear Story Data, Inc. All Rights Reserved. +* +* Please see the COPYRIGHT file in the root of this repository for more +* details. +*/ +package spark + +import org.scalatest.FunSuite +import scheduler.{DAGScheduler, TaskSchedulerListener, TaskSet, TaskScheduler} +import collection.mutable + +class TaskSchedulerMock(f: (Int) => TaskEndReason ) extends TaskScheduler { + // Listener object to pass upcalls into + var listener: TaskSchedulerListener = null + var taskCount = 0 + + override def start(): Unit = {} + + // Disconnect from the cluster. + override def stop(): Unit = {} + + // Submit a sequence of tasks to run. + override def submitTasks(taskSet: TaskSet): Unit = { + taskSet.tasks.foreach( task => { + val m = new mutable.HashMap[Long, Any]() + m.put(task.stageId, 1) + taskCount += 1 + listener.taskEnded(task, f(taskCount), 1, m) + }) + } + + // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called. + override def setListener(listener: TaskSchedulerListener) { + this.listener = listener + } + + // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. + override def defaultParallelism(): Int = { + 2 + } +} + +class DAGSchedulerSuite extends FunSuite { + def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { + assert(dagScheduler.pendingTasks.isEmpty) + assert(dagScheduler.activeJobs.isEmpty) + assert(dagScheduler.failed.isEmpty) + assert(dagScheduler.runIdToStageIds.isEmpty) + assert(dagScheduler.idToStage.isEmpty) + assert(dagScheduler.resultStageToJob.isEmpty) + assert(dagScheduler.running.isEmpty) + assert(dagScheduler.shuffleToMapStage.isEmpty) + assert(dagScheduler.waiting.isEmpty) + } + + test("oneGoodJob") { + val sc = new SparkContext("local", "test") + val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + try { + val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) + val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + val callSite = Utils.getSparkCallSite + + val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) + assertDagSchedulerEmpty(dagScheduler) + } finally { + dagScheduler.stop() + sc.stop() + // pause to let dagScheduler stop (separate thread) + Thread.sleep(10) + } + } + + test("manyGoodJobs") { + val sc = new SparkContext("local", "test") + val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + try { + val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) + val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + val callSite = Utils.getSparkCallSite + + 1.to(100).foreach( v => { + val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) + }) + assertDagSchedulerEmpty(dagScheduler) + } finally { + dagScheduler.stop() + sc.stop() + // pause to let dagScheduler stop (separate thread) + Thread.sleep(10) + } + } +} From 524198ad2cf2e9f3c3e2977309e1e9b32b767b81 Mon Sep 17 00:00:00 2001 From: Jon Hartlaub Date: Thu, 24 Jan 2013 15:23:29 -0800 Subject: [PATCH 676/696] Removed copyright --- core/src/test/scala/spark/DAGSchedulerSuite.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala index 99fd83e347f87..2a3b30ae429b3 100644 --- a/core/src/test/scala/spark/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/DAGSchedulerSuite.scala @@ -1,9 +1,3 @@ -/* -* Copyright (c) Clear Story Data, Inc. All Rights Reserved. -* -* Please see the COPYRIGHT file in the root of this repository for more -* details. -*/ package spark import org.scalatest.FunSuite From 1d6f4fbc5acc0e0c6bf228c26282671d28e5e558 Mon Sep 17 00:00:00 2001 From: Thomas Dudziak Date: Thu, 3 Jan 2013 16:09:10 -0800 Subject: [PATCH 677/696] CSD version numbering and distribution management --- bagel/pom.xml | 2 +- core/pom.xml | 10 +++++----- examples/pom.xml | 2 +- pom.xml | 17 ++++++++++++++--- repl-bin/pom.xml | 4 ++-- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 25 insertions(+), 14 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index b83a0ef6c0f37..154d4a69343c0 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 88f0ed70f3c03..d23ac6fca6799 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml @@ -143,14 +143,14 @@ true - - + + - - + + diff --git a/examples/pom.xml b/examples/pom.xml index 3e5271ec2f963..965086fc8eff2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 3bcb2a3f345dc..50a83cdc41e01 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ @@ -15,8 +15,8 @@ - scm:git:git@github.com:mesos/spark.git - scm:git:git@github.com:mesos/spark.git + scm:git:git@github.com:clearstorydata/spark.git + scm:git:git@github.com:clearstorydata/spark.git @@ -157,6 +157,17 @@ + + + ${distRepo.snapshots.id} + ${distRepo.snapshots.url} + + + ${distRepo.releases.id} + ${distRepo.releases.url} + + + diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 7a7280313edb7..a40b602506563 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml @@ -52,7 +52,7 @@ - + reference.conf diff --git a/repl/pom.xml b/repl/pom.xml index 92a2020b48d0b..733fa5298c6bf 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 4dc9a19d51fc1..43d3573fa0252 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml From a9b4f4ddf01223df2da7cc24228d2e05f04b1683 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 15:37:13 -0700 Subject: [PATCH 678/696] Bump some dependencies, pin others, for Scala 2.9.2 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 50a83cdc41e01..3ade2004c0f72 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ org.eclipse.jetty jetty-server - 7.5.3.v20111011 + 7.6.8.v20121106 com.google.guava @@ -320,7 +320,7 @@ com.novocode junit-interface - 0.8 + 0.9 test From 1d8188ba319fb90adf3bb8585f983b6166f90d45 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 15:38:35 -0700 Subject: [PATCH 679/696] DAGSchedulerSuite now part of core/src/test/scala/spark/scheduler --- .../test/scala/spark/DAGSchedulerSuite.scala | 88 ------------------- 1 file changed, 88 deletions(-) delete mode 100644 core/src/test/scala/spark/DAGSchedulerSuite.scala diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala deleted file mode 100644 index 2a3b30ae429b3..0000000000000 --- a/core/src/test/scala/spark/DAGSchedulerSuite.scala +++ /dev/null @@ -1,88 +0,0 @@ -package spark - -import org.scalatest.FunSuite -import scheduler.{DAGScheduler, TaskSchedulerListener, TaskSet, TaskScheduler} -import collection.mutable - -class TaskSchedulerMock(f: (Int) => TaskEndReason ) extends TaskScheduler { - // Listener object to pass upcalls into - var listener: TaskSchedulerListener = null - var taskCount = 0 - - override def start(): Unit = {} - - // Disconnect from the cluster. - override def stop(): Unit = {} - - // Submit a sequence of tasks to run. - override def submitTasks(taskSet: TaskSet): Unit = { - taskSet.tasks.foreach( task => { - val m = new mutable.HashMap[Long, Any]() - m.put(task.stageId, 1) - taskCount += 1 - listener.taskEnded(task, f(taskCount), 1, m) - }) - } - - // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called. - override def setListener(listener: TaskSchedulerListener) { - this.listener = listener - } - - // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. - override def defaultParallelism(): Int = { - 2 - } -} - -class DAGSchedulerSuite extends FunSuite { - def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { - assert(dagScheduler.pendingTasks.isEmpty) - assert(dagScheduler.activeJobs.isEmpty) - assert(dagScheduler.failed.isEmpty) - assert(dagScheduler.runIdToStageIds.isEmpty) - assert(dagScheduler.idToStage.isEmpty) - assert(dagScheduler.resultStageToJob.isEmpty) - assert(dagScheduler.running.isEmpty) - assert(dagScheduler.shuffleToMapStage.isEmpty) - assert(dagScheduler.waiting.isEmpty) - } - - test("oneGoodJob") { - val sc = new SparkContext("local", "test") - val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) - try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) - val func = (tc: TaskContext, iter: Iterator[Int]) => 1 - val callSite = Utils.getSparkCallSite - - val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) - assertDagSchedulerEmpty(dagScheduler) - } finally { - dagScheduler.stop() - sc.stop() - // pause to let dagScheduler stop (separate thread) - Thread.sleep(10) - } - } - - test("manyGoodJobs") { - val sc = new SparkContext("local", "test") - val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) - try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) - val func = (tc: TaskContext, iter: Iterator[Int]) => 1 - val callSite = Utils.getSparkCallSite - - 1.to(100).foreach( v => { - val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) - }) - assertDagSchedulerEmpty(dagScheduler) - } finally { - dagScheduler.stop() - sc.stop() - // pause to let dagScheduler stop (separate thread) - Thread.sleep(10) - } - } -} From f8ca23b9d0ecd5889f23db1ba8dc7305bb40b2e4 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 16:07:47 -0700 Subject: [PATCH 680/696] wip: Bringing Jon's DAGScheduler tests into the new DAGSchedulerSuite --- .../spark/scheduler/DAGSchedulerSuite.scala | 81 ++++++++++++++++++- 1 file changed, 80 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef9505c7..e475976f9f342 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -14,12 +14,13 @@ import spark.{Dependency, ShuffleDependency, OneToOneDependency} import spark.FetchFailedException import spark.MapOutputTracker import spark.RDD +import spark.rdd.ParallelCollectionRDD import spark.SparkContext import spark.SparkException import spark.Partition import spark.TaskContext import spark.TaskEndReason - +import spark.Utils import spark.{FetchFailed, Success} /** @@ -385,6 +386,42 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(results === Map(0 -> 42)) } + // test("oneGoodJob") { + // val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + // try { + // val rdd = new ParallelCollectionRDD(sc, 1.to(100).toSeq, 5, Map.empty) + // val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + // val callSite = Utils.getSparkCallSite + + // dagScheduler.runJob(rdd, func, 0 until rdd.partitions.size, callSite, false, {(_: Int, _: Int) => {}}) + // assertDagSchedulerEmpty(dagScheduler) + // } finally { + // dagScheduler.stop() + // sc.stop() + // // pause to let dagScheduler stop (separate thread) + // Thread.sleep(10) + // } + // } + + // test("manyGoodJobs") { + // val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + // try { + // val rdd = new ParallelCollectionRDD(sc, 1.to(100).toSeq, 5, Map.empty) + // val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + // val callSite = Utils.getSparkCallSite + + // 1.to(100).foreach( v => { + // dagScheduler.runJob(rdd, func, 0 until rdd.partitions.size, callSite, false, {(_: Int, _: Int) => {}}) + // }) + // assertDagSchedulerEmpty(dagScheduler) + // } finally { + // dagScheduler.stop() + // sc.stop() + // // pause to let dagScheduler stop (separate thread) + // Thread.sleep(10) + // } + // } + /** Assert that the supplied TaskSet has exactly the given preferredLocations. Note, converts taskSet's locations to host only. */ private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { assert(locations.size === taskSet.tasks.size) @@ -400,4 +437,46 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345, 0) + private def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { + assert(dagScheduler.pendingTasks.isEmpty) + assert(dagScheduler.activeJobs.isEmpty) + assert(dagScheduler.failed.isEmpty) + assert(dagScheduler.runIdToStageIds.isEmpty) + assert(dagScheduler.idToStage.isEmpty) + assert(dagScheduler.resultStageToJob.isEmpty) + assert(dagScheduler.running.isEmpty) + assert(dagScheduler.shuffleToMapStage.isEmpty) + assert(dagScheduler.waiting.isEmpty) + } +} + +class TaskSchedulerMock(f: (Int) => TaskEndReason ) extends TaskScheduler { + // Listener object to pass upcalls into + var listener: TaskSchedulerListener = null + var taskCount = 0 + + override def start(): Unit = {} + + // Disconnect from the cluster. + override def stop(): Unit = {} + + // Submit a sequence of tasks to run. + override def submitTasks(taskSet: TaskSet): Unit = { + taskSet.tasks.foreach( task => { + val m = new scala.collection.mutable.HashMap[Long, Any]() + m.put(task.stageId, 1) + taskCount += 1 + listener.taskEnded(task, f(taskCount), 1, m, null, task.metrics.getOrElse(null)) // TODO: TaskInfo + }) + } + + // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called. + override def setListener(listener: TaskSchedulerListener) { + this.listener = listener + } + + // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. + override def defaultParallelism(): Int = { + 2 + } } From 25dad26a81a051a5406bad446a067bab8818ef05 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 16:22:34 -0700 Subject: [PATCH 681/696] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-1 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 154d4a69343c0..46ba13c2e052e 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d23ac6fca6799..c2e1d01c3e6b1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 965086fc8eff2..b9c9c9c8b32f8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/pom.xml b/pom.xml index 3ade2004c0f72..be24936854b48 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index a40b602506563..bbaa4d63d54ce 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 733fa5298c6bf..f4ab5faa335a1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 43d3573fa0252..2d1ae474110cb 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml From 7e03d9c54e382ef040968f71527ad85ff4e60116 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 16:22:42 -0700 Subject: [PATCH 682/696] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 46ba13c2e052e..070b1f02f77d8 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index c2e1d01c3e6b1..9ab8b670215b1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index b9c9c9c8b32f8..343514662c07f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index be24936854b48..d654b52ac2bc7 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index bbaa4d63d54ce..95db203505b37 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index f4ab5faa335a1..c34260da93786 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 2d1ae474110cb..d2887afd6860d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml From 2a46a56693c5ca142a2d4114d66f3c535099d511 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 17 May 2013 15:13:52 -0700 Subject: [PATCH 683/696] Specified commons-io version for spark-core --- core/pom.xml | 4 ++++ pom.xml | 10 ++++++++++ 2 files changed, 14 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 9ab8b670215b1..119bd34a77c76 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -15,6 +15,10 @@ http://spark-project.org/ + + commons-io + commons-io + org.eclipse.jetty jetty-server diff --git a/pom.xml b/pom.xml index d654b52ac2bc7..e0512a24f1c3e 100644 --- a/pom.xml +++ b/pom.xml @@ -170,6 +170,11 @@ + + commons-io + commons-io + 2.4 + org.eclipse.jetty jetty-server @@ -195,6 +200,11 @@ jul-to-slf4j ${slf4j.version} + + commons-daemon + commons-daemon + 1.0.10 + com.ning compress-lzf From 74a6c132b32b0a8154a50244804a69bc3c8a271c Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 22 May 2013 15:14:45 -0700 Subject: [PATCH 684/696] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-2 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 070b1f02f77d8..bbd1d42832f07 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 119bd34a77c76..c6a8e9b84a661 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 343514662c07f..1c5a7597cfb09 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/pom.xml b/pom.xml index e0512a24f1c3e..6ea61076244d9 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 pom Spark Project Parent POM http://spark-project.org/ @@ -456,7 +456,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xms64m -Xmx1024m - + diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 95db203505b37..1f21652fc0af9 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index c34260da93786..69eba62dd796a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index d2887afd6860d..59d4540f69f5b 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml From ea16c639556fb1d869dbdf3eb9978793bbc958e8 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 22 May 2013 15:18:41 -0700 Subject: [PATCH 685/696] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index bbd1d42832f07..477ff3f6a0ae6 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index c6a8e9b84a661..839d93570d604 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 1c5a7597cfb09..893d930bc7e97 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 6ea61076244d9..63ad1baa6c7da 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 1f21652fc0af9..abd60ce7cf524 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 69eba62dd796a..b043991e5c38f 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 59d4540f69f5b..a8bc8ef085448 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml From 04f3f3c000d7a0f80dea0674065e333180d635c4 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 28 May 2013 10:54:41 -0700 Subject: [PATCH 686/696] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-3 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 477ff3f6a0ae6..69a9da3d1a753 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 839d93570d604..edf55059edbbd 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 893d930bc7e97..05d0e85b8342f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/pom.xml b/pom.xml index 63ad1baa6c7da..68b89234acbd0 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index abd60ce7cf524..e02c3de28b084 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index b043991e5c38f..0ef9830befea9 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index a8bc8ef085448..b354f9a351565 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml From 51fc0cf4364cc7449db78cb1124f415d6d9d6c6e Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 28 May 2013 10:54:47 -0700 Subject: [PATCH 687/696] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 69a9da3d1a753..f7913ef55e696 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index edf55059edbbd..f96ca23d9295b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 05d0e85b8342f..da4010f34f843 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 68b89234acbd0..07b630ed45050 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index e02c3de28b084..6a920c7bcabc6 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 0ef9830befea9..da3837ea0f919 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index b354f9a351565..7d0a5b74b2e6a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml From 8cf9f48bbbe7c81d991e5203a6a01c30115d5149 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 10:15:09 -0700 Subject: [PATCH 688/696] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-4 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index f7913ef55e696..e9814871eb30f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index f96ca23d9295b..3921b2b509325 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index da4010f34f843..c6cea884913f6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/pom.xml b/pom.xml index 07b630ed45050..020eb67bcec0d 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6a920c7bcabc6..235640a9abe37 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index da3837ea0f919..55d9735e78367 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 7d0a5b74b2e6a..39349d8f52a9e 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml From 24171f2b2a64cc3238233cf6aa0f423b080c83e2 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 10:15:15 -0700 Subject: [PATCH 689/696] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index e9814871eb30f..d40a899344a81 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 3921b2b509325..422be0ba6ab0c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index c6cea884913f6..49d1ddc17079e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 020eb67bcec0d..91aae021b3f4f 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 235640a9abe37..4549d10a746bd 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 55d9735e78367..d3efd8df930ac 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 39349d8f52a9e..8a98d38743c45 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml From aec7efc55486dd9bfedd11ea99715654595ea60c Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 17:21:04 -0700 Subject: [PATCH 690/696] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-5 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index d40a899344a81..aaf259191bf78 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 422be0ba6ab0c..ecfdb2f07c864 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 49d1ddc17079e..98326798b37db 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/pom.xml b/pom.xml index 91aae021b3f4f..e6bc9db609746 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 4549d10a746bd..ccdfa259f3bba 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index d3efd8df930ac..45368f4842799 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8a98d38743c45..e13eca908ecfa 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml From 1dcc2af4b4c694ca75e11b8f3fc86921f36c98af Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 17:21:11 -0700 Subject: [PATCH 691/696] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index aaf259191bf78..e2a8c5057caf8 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ecfdb2f07c864..848567c4de0d8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 98326798b37db..6aabb22e1c1fc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index e6bc9db609746..0f94c4330d931 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index ccdfa259f3bba..a38938778a1b8 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 45368f4842799..9f16ff8f327a2 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index e13eca908ecfa..77b7618f29f9b 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml From 4987795e3e8c9ffaf177685502bec192aa5d1f0d Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 12 Jun 2013 16:01:20 -0700 Subject: [PATCH 692/696] Removed commons-io dependency from core --- core/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 848567c4de0d8..1bfc6ab31cb7b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -15,10 +15,6 @@ http://spark-project.org/ - - commons-io - commons-io - org.eclipse.jetty jetty-server From 62385ee47371b791463ebc59ba338b30e326e0f4 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 14 Jun 2013 11:51:48 -0700 Subject: [PATCH 693/696] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-6 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index e2a8c5057caf8..39e44d2f35213 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1bfc6ab31cb7b..2a50f22d32cb1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 6aabb22e1c1fc..888b5d14e15c8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/pom.xml b/pom.xml index 0f94c4330d931..6dafd4509f59a 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index a38938778a1b8..ec35f6e8eb2ae 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 9f16ff8f327a2..20e7fff224524 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 77b7618f29f9b..a652b2b40c46f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml From 70ea6be56aeec21ca65d59785c554ae6a05f2ac7 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 14 Jun 2013 11:51:55 -0700 Subject: [PATCH 694/696] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 39e44d2f35213..b6ce8d206effe 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2a50f22d32cb1..de487e91b90f2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 888b5d14e15c8..b2958e1f98730 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 6dafd4509f59a..c57fd8d7ae488 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index ec35f6e8eb2ae..902691d8ab91c 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20e7fff224524..a265bc4afc1fa 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index a652b2b40c46f..53703c89aff92 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml From ae96cf837b18fa23ca6b4e51f0c777f7608940ce Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sat, 22 Jun 2013 13:01:52 -0700 Subject: [PATCH 695/696] Removed redundant job.listener.jobFailed notification --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 0bf586bba519e..f9c880558eb15 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -675,7 +675,6 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= resultStage removeStages(job) - job.listener.jobFailed(new SparkException("Job failed: " + reason)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") From 211a5442a65dcdc950a598deaa96a3ee4fd53568 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 27 Jun 2013 10:42:03 -0700 Subject: [PATCH 696/696] Replacement DAGSchedulerSuite is in core/src/test/scala/spark/sheduler --- .../test/scala/spark/DAGSchedulerSuite.scala | 88 ------------------- 1 file changed, 88 deletions(-) delete mode 100644 core/src/test/scala/spark/DAGSchedulerSuite.scala diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala deleted file mode 100644 index 2a3b30ae429b3..0000000000000 --- a/core/src/test/scala/spark/DAGSchedulerSuite.scala +++ /dev/null @@ -1,88 +0,0 @@ -package spark - -import org.scalatest.FunSuite -import scheduler.{DAGScheduler, TaskSchedulerListener, TaskSet, TaskScheduler} -import collection.mutable - -class TaskSchedulerMock(f: (Int) => TaskEndReason ) extends TaskScheduler { - // Listener object to pass upcalls into - var listener: TaskSchedulerListener = null - var taskCount = 0 - - override def start(): Unit = {} - - // Disconnect from the cluster. - override def stop(): Unit = {} - - // Submit a sequence of tasks to run. - override def submitTasks(taskSet: TaskSet): Unit = { - taskSet.tasks.foreach( task => { - val m = new mutable.HashMap[Long, Any]() - m.put(task.stageId, 1) - taskCount += 1 - listener.taskEnded(task, f(taskCount), 1, m) - }) - } - - // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called. - override def setListener(listener: TaskSchedulerListener) { - this.listener = listener - } - - // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. - override def defaultParallelism(): Int = { - 2 - } -} - -class DAGSchedulerSuite extends FunSuite { - def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { - assert(dagScheduler.pendingTasks.isEmpty) - assert(dagScheduler.activeJobs.isEmpty) - assert(dagScheduler.failed.isEmpty) - assert(dagScheduler.runIdToStageIds.isEmpty) - assert(dagScheduler.idToStage.isEmpty) - assert(dagScheduler.resultStageToJob.isEmpty) - assert(dagScheduler.running.isEmpty) - assert(dagScheduler.shuffleToMapStage.isEmpty) - assert(dagScheduler.waiting.isEmpty) - } - - test("oneGoodJob") { - val sc = new SparkContext("local", "test") - val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) - try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) - val func = (tc: TaskContext, iter: Iterator[Int]) => 1 - val callSite = Utils.getSparkCallSite - - val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) - assertDagSchedulerEmpty(dagScheduler) - } finally { - dagScheduler.stop() - sc.stop() - // pause to let dagScheduler stop (separate thread) - Thread.sleep(10) - } - } - - test("manyGoodJobs") { - val sc = new SparkContext("local", "test") - val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) - try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) - val func = (tc: TaskContext, iter: Iterator[Int]) => 1 - val callSite = Utils.getSparkCallSite - - 1.to(100).foreach( v => { - val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) - }) - assertDagSchedulerEmpty(dagScheduler) - } finally { - dagScheduler.stop() - sc.stop() - // pause to let dagScheduler stop (separate thread) - Thread.sleep(10) - } - } -}