From 96130c30d90abf155731346488f79c034bcaaf6a Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 3 Jul 2013 05:49:04 +0000 Subject: [PATCH 01/75] add compression codec trait and snappy compression --- .../scala/spark/storage/BlockManager.scala | 20 +++++++++++++++++-- .../spark/storage/CompressionCodec.scala | 13 ++++++++++++ .../main/scala/spark/storage/DiskStore.scala | 1 - .../spark/storage/LZFCompressionCodec.scala | 16 +++++++++++++++ .../storage/SnappyCompressionCodec.scala | 18 +++++++++++++++++ 5 files changed, 65 insertions(+), 3 deletions(-) create mode 100644 core/src/main/scala/spark/storage/CompressionCodec.scala create mode 100644 core/src/main/scala/spark/storage/LZFCompressionCodec.scala create mode 100644 core/src/main/scala/spark/storage/SnappyCompressionCodec.scala diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9b39d3aadf..2d4a3502c6 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -141,6 +141,8 @@ private[spark] class BlockManager( val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks) initialize() + var compressionCodec: CompressionCodec = null + /** * Construct a BlockManager with a memory limit set based on system properties. */ @@ -902,8 +904,15 @@ private[spark] 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 (compressionCodec == null) { + compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", + "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) + .newInstance().asInstanceOf[CompressionCodec] + } + if (shouldCompress(blockId)) { - (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + //(new LZFOutputStream(s)).setFinishBlockOnFlush(true) + compressionCodec.compressionOutputStream(s) } else { s } @@ -913,7 +922,14 @@ private[spark] class BlockManager( * Wrap an input stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: InputStream): InputStream = { - if (shouldCompress(blockId)) new LZFInputStream(s) else s + if (compressionCodec == null) { + compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", + "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) + .newInstance().asInstanceOf[CompressionCodec] + } + + if (shouldCompress(blockId)) /*new LZFInputStream(s) */ + compressionCodec.compressionInputStream(s) else s } def dataSerialize( diff --git a/core/src/main/scala/spark/storage/CompressionCodec.scala b/core/src/main/scala/spark/storage/CompressionCodec.scala new file mode 100644 index 0000000000..cd80de33f6 --- /dev/null +++ b/core/src/main/scala/spark/storage/CompressionCodec.scala @@ -0,0 +1,13 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + + +/** + * CompressionCodec allows the customization of the compression codec + */ +trait CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream + + def compressionInputStream(s: InputStream): InputStream +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index da859eebcb..221e285192 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -49,7 +49,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def close() { if (initialized) { objOut.close() - bs.close() channel = null bs = null objOut = null diff --git a/core/src/main/scala/spark/storage/LZFCompressionCodec.scala b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala new file mode 100644 index 0000000000..3328b949ef --- /dev/null +++ b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala @@ -0,0 +1,16 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} + +/** + * LZF implementation of [[spark.storage.CompressionCodec]] + */ +class LZFCompressionCodec extends CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream = + (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + + def compressionInputStream(s: InputStream): InputStream = + new LZFInputStream(s) +} diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala new file mode 100644 index 0000000000..dc8546b039 --- /dev/null +++ b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala @@ -0,0 +1,18 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + +import org.xerial.snappy.SnappyOutputStream + +/** + * Snappy implementation of [[spark.storage.CompressionCodec]] + * block size can be configured by spark.snappy.block.size + */ +class SnappyCompressionCodec extends CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream = + new SnappyOutputStream(s, + System.getProperty("spark.snappy.block.size", "32768").toInt) + + def compressionInputStream(s: InputStream): InputStream = + new SnappyInputStream(s) +} From 94238aae57475030f6e88102a83c7809c5835494 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 3 Jul 2013 18:08:38 +0000 Subject: [PATCH 02/75] fix dependencies --- core/src/main/scala/spark/storage/SnappyCompressionCodec.scala | 2 +- project/SparkBuild.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala index dc8546b039..62b00ef3f6 100644 --- a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala +++ b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala @@ -2,7 +2,7 @@ package spark.storage import java.io.{InputStream, OutputStream} -import org.xerial.snappy.SnappyOutputStream +import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} /** * Snappy implementation of [[spark.storage.CompressionCodec]] diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 07572201de..f824826af3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,7 +162,8 @@ object SparkBuild extends Build { "cc.spray" % "spray-json_2.9.2" % "1.1.1" excludeAll(excludeNetty), "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", - "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "org.apache.derby" % "derby" % "10.4.2.0" % "test", + "org.xerial.snappy" % "snappy-java" % "1.0.5" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From 2080e250060975a876a388eb785e7f2b3cf2c0cd Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 12 Jul 2013 14:25:18 +0800 Subject: [PATCH 03/75] Enhance job ui in spark ui system with adding pool information --- core/src/main/scala/spark/SparkContext.scala | 17 ++- .../scala/spark/scheduler/DAGScheduler.scala | 4 +- .../scala/spark/scheduler/JobLogger.scala | 2 +- .../scala/spark/scheduler/SparkListener.scala | 2 +- .../scala/spark/scheduler/TaskScheduler.scala | 7 + .../scheduler/cluster/ClusterScheduler.scala | 11 +- .../cluster/ClusterTaskSetManager.scala | 4 +- .../spark/scheduler/cluster/Schedulable.scala | 6 +- .../scheduler/cluster/SchedulingMode.scala | 4 +- .../scheduler/cluster/TaskSetManager.scala | 1 + .../scheduler/local/LocalScheduler.scala | 10 +- .../scheduler/local/LocalTaskSetManager.scala | 1 + .../main/scala/spark/ui/jobs/IndexPage.scala | 115 ++++---------- .../spark/ui/jobs/JobProgressListener.scala | 140 +++++++++++++++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 112 +++----------- .../main/scala/spark/ui/jobs/PoolPage.scala | 38 +++++ .../main/scala/spark/ui/jobs/PoolTable.scala | 98 ++++++++++++ .../main/scala/spark/ui/jobs/StageTable.scala | 143 ++++++++++++++++++ .../spark/scheduler/DAGSchedulerSuite.scala | 6 + .../spark/scheduler/JobLoggerSuite.scala | 2 +- 20 files changed, 527 insertions(+), 196 deletions(-) create mode 100644 core/src/main/scala/spark/ui/jobs/JobProgressListener.scala create mode 100644 core/src/main/scala/spark/ui/jobs/PoolPage.scala create mode 100644 core/src/main/scala/spark/ui/jobs/PoolTable.scala create mode 100644 core/src/main/scala/spark/ui/jobs/StageTable.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 43e6af2351..b5225d5681 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -10,6 +10,7 @@ import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.HashMap +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.util.DynamicVariable import scala.collection.mutable.{ConcurrentMap, HashMap} @@ -43,13 +44,14 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} +import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -540,6 +542,17 @@ class SparkContext( env.blockManager.master.getStorageStatus } + def getPoolsInfo: ArrayBuffer[Schedulable] = { + taskScheduler.rootPool.schedulableQueue + } + + def getSchedulingMode: SchedulingMode = { + taskScheduler.schedulingMode + } + + def getPoolNameToPool: HashMap[String, Schedulable] = { + taskScheduler.rootPool.schedulableNameToSchedulable + } /** * 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/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..c865743e37 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -472,11 +472,11 @@ class DAGScheduler( } } if (tasks.size > 0) { - sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) + val properties = idToActiveJob(stage.priority).properties + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size, properties))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) - val properties = idToActiveJob(stage.priority).properties taskSched.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority, properties)) if (!stage.submissionTime.isDefined) { diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 6a9d52f356..8e5540873f 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -45,7 +45,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { event match { case SparkListenerJobStart(job, properties) => processJobStartEvent(job, properties) - case SparkListenerStageSubmitted(stage, taskSize) => + case SparkListenerStageSubmitted(stage, taskSize, properties) => processStageSubmittedEvent(stage, taskSize) case StageCompleted(stageInfo) => processStageCompletedEvent(stageInfo) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 8de3aa91a4..94fdad9b98 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -8,7 +8,7 @@ import spark.executor.TaskMetrics sealed trait SparkListenerEvents -case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends SparkListenerEvents +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties = null) extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index 7787b54762..5cdf846032 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -1,5 +1,7 @@ package spark.scheduler +import spark.scheduler.cluster.Pool +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler. * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage, @@ -8,6 +10,11 @@ package spark.scheduler * the TaskSchedulerListener interface. */ private[spark] trait TaskScheduler { + + def rootPool: Pool + + def schedulingMode: SchedulingMode + def start(): Unit // Invoked after system has successfully initialized (typically in spark context). diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3a0c29b27f..1b23fd6cef 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -12,6 +12,7 @@ import spark.scheduler._ import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call @@ -97,6 +98,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null + //default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) override def setListener(listener: TaskSchedulerListener) { this.listener = listener @@ -104,15 +107,13 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context - //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) + rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { - case "FIFO" => + case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) - case "FAIR" => + case SchedulingMode.FAIR => new FairSchedulableBuilder(rootPool) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index fe6420a522..7a6a6b7826 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -90,8 +90,8 @@ private[spark] class ClusterTaskSetManager( var priority = taskSet.priority var stageId = taskSet.stageId var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null - + var parent: Schedulable = null + var schedulableQueue :ArrayBuffer[Schedulable] = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2dd9c0564f..2e4f14c11f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,13 +1,17 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer +import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import scala.collection.mutable.ArrayBuffer /** * An interface for schedulable entities. * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { var parent: Schedulable + //childrens + def schedulableQueue: ArrayBuffer[Schedulable] + def schedulingMode: SchedulingMode def weight: Int def minShare: Int def runningTasks: Int diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6e0c6793e0..c5c7ee3b22 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,7 +1,7 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR","FIFO"){ +object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE"){ type SchedulingMode = Value - val FAIR,FIFO = Value + val FAIR,FIFO,NONE = Value } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index b4dd75d90f..472e01b227 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -6,6 +6,7 @@ import spark.TaskState.TaskState import java.nio.ByteBuffer private[spark] trait TaskSetManager extends Schedulable { + def schedulingMode = SchedulingMode.NONE def taskSet: TaskSet def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index b000e328e6..19a48895e3 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -12,6 +12,7 @@ import spark.TaskState.TaskState import spark.executor.ExecutorURLClassLoader import spark.scheduler._ import spark.scheduler.cluster._ +import spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ /** @@ -63,6 +64,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null + val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) val activeTaskSets = new HashMap[String, TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -70,15 +72,13 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var localActor: ActorRef = null 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) + rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { - case "FIFO" => + case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) - case "FAIR" => + case SchedulingMode.FAIR => new FairSchedulableBuilder(rootPool) } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f12fec41d5..8954f40ea9 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -14,6 +14,7 @@ import spark.scheduler.cluster._ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null + var schedulableQueue :ArrayBuffer[Schedulable] = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1e675ab2cb..e765cecb01 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -6,107 +6,52 @@ import javax.servlet.http.HttpServletRequest import scala.Some import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.storage.StorageLevel +import spark.scheduler.cluster.Schedulable +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode -/** Page showing list of all ongoing and recently finished stages */ +/** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener - val dateFmt = parent.dateFmt + + def stageTable: StageTable = parent.stageTable + + def poolTable: PoolTable = parent.poolTable def render(request: HttpServletRequest): Seq[Node] = { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - /** Special table which merges two header cells. */ - def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - - - - - - - - - - - {rows.map(r => makeRow(r))} - -
Stage IdOriginSubmittedDurationTasks: Complete/TotalShuffle ActivityStored RDD
- } + stageTable.setStagePoolInfo(parent.stagePoolInfo) + poolTable.setPoolSource(parent.stagePagePoolSource) - val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) - val completedStageTable = stageTable(stageRow, completedStages) - val failedStageTable: NodeSeq = stageTable(stageRow, failedStages) + val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + val completedStageNodeSeq = stageTable.toNodeSeq(completedStages) + val failedStageNodeSeq = stageTable.toNodeSeq(failedStages) - val content =

Active Stages

++ activeStageTable ++ -

Completed Stages

++ completedStageTable ++ -

Failed Stages

++ failedStageTable + val content =
+
+
    +
  • Active Stages Number: {activeStages.size}
  • +
  • Completed Stages Number: {completedStages.size}
  • +
  • Failed Stages Number: {failedStages.size}
  • +
  • Scheduling Mode: {parent.sc.getSchedulingMode}
  • +
+
+
++ +

Pools

++ poolTable.toNodeSeq ++ +

Active Stages : {activeStages.size}

++ activeStageNodeSeq++ +

Completed Stages : {completedStages.size}

++ completedStageNodeSeq++ +

Failed Stages : {failedStages.size}

++ failedStageNodeSeq - headerSparkPage(content, parent.sc, "Spark Stages", Jobs) - } - - def getElapsedTime(submitted: Option[Long], completed: Long): String = { - submitted match { - case Some(t) => parent.formatDuration(completed - t) - case _ => "Unknown" - } - } - - def makeProgressBar(completed: Int, total: Int): Seq[Node] = { - val width=130 - val height=15 - val completeWidth = (completed.toDouble / total) * width - - - - - - } - - - def stageRow(s: Stage): Seq[Node] = { - val submissionTime = s.submissionTime match { - case Some(t) => dateFmt.format(new Date(t)) - case None => "Unknown" - } - val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) - val shuffleInfo = (read, write) match { - case (true, true) => "Read/Write" - case (true, false) => "Read" - case (false, true) => "Write" - case _ => "" - } - val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) - val totalTasks = s.numPartitions - - - {s.id} - {s.origin} - {submissionTime} - {getElapsedTime(s.submissionTime, - s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} - {completedTasks} / {totalTasks} - {listener.stageToTasksFailed.getOrElse(s.id, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => - }} - - {shuffleInfo} - {if (s.rdd.getStorageLevel != StorageLevel.NONE) { - - {Option(s.rdd.name).getOrElse(s.rdd.id)} - - }} - - + headerSparkPage(content, parent.sc, "Spark Stages/Pools", Jobs) } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala new file mode 100644 index 0000000000..1244f9538b --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -0,0 +1,140 @@ +package spark.ui.jobs + +import scala.Seq +import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} + +import spark.{ExceptionFailure, SparkContext, Success, Utils} +import spark.scheduler._ +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import collection.mutable + +private[spark] class FairJobProgressListener(val sparkContext: SparkContext) + extends JobProgressListener(sparkContext) { + + val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" + val DEFAULT_POOL_NAME = "default" + + override val stageToPool = HashMap[Stage, String]() + override val poolToActiveStages = HashMap[String, HashSet[Stage]]() + + override def onStageCompleted(stageCompleted: StageCompleted) = { + super.onStageCompleted(stageCompleted) + val stage = stageCompleted.stageInfo.stage + poolToActiveStages(stageToPool(stage)) -= stage + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { + super.onStageSubmitted(stageSubmitted) + val stage = stageSubmitted.stage + var poolName = DEFAULT_POOL_NAME + if (stageSubmitted.properties != null) { + poolName = stageSubmitted.properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) + } + stageToPool(stage) = poolName + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) + stages += stage + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + super.onJobEnd(jobEnd) + jobEnd match { + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + poolToActiveStages(stageToPool(stage)) -= stage + case _ => + } + case _ => + } + } +} + +private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { + // How many stages to remember + val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt + + def stageToPool: HashMap[Stage, String] = null + def poolToActiveStages: HashMap[String, HashSet[Stage]] =null + + val activeStages = HashSet[Stage]() + val completedStages = ListBuffer[Stage]() + val failedStages = ListBuffer[Stage]() + + val stageToTasksComplete = HashMap[Int, Int]() + val stageToTasksFailed = HashMap[Int, Int]() + val stageToTaskInfos = + HashMap[Int, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + + override def onJobStart(jobStart: SparkListenerJobStart) {} + + override def onStageCompleted(stageCompleted: StageCompleted) = { + val stage = stageCompleted.stageInfo.stage + activeStages -= stage + completedStages += stage + trimIfNecessary(completedStages) + } + + /** If stages is too large, remove and garbage collect old stages */ + def trimIfNecessary(stages: ListBuffer[Stage]) { + if (stages.size > RETAINED_STAGES) { + val toRemove = RETAINED_STAGES / 10 + stages.takeRight(toRemove).foreach( s => { + stageToTaskInfos.remove(s.id) + }) + stages.trimEnd(toRemove) + } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = + activeStages += stageSubmitted.stage + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val sid = taskEnd.task.stageId + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = + taskEnd.reason match { + case e: ExceptionFailure => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + (Some(e), e.metrics) + case _ => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + (None, Some(taskEnd.taskMetrics)) + } + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) + stageToTaskInfos(sid) = taskList + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + jobEnd match { + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + activeStages -= stage + failedStages += stage + trimIfNecessary(failedStages) + case _ => + } + case _ => + } + } + + /** Is this stage's input from a shuffle read. */ + def hasShuffleRead(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined + } + return false // No tasks have finished for this stage + } + + /** Is this stage's output to a shuffle write. */ + def hasShuffleWrite(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined + } + return false // No tasks have finished for this stage + } +} diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 84730cc091..e610252242 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -14,9 +14,9 @@ import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} import spark.ui.JettyUtils._ import spark.{ExceptionFailure, SparkContext, Success, Utils} import spark.scheduler._ -import spark.scheduler.cluster.TaskInfo -import spark.executor.TaskMetrics import collection.mutable +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { @@ -24,104 +24,38 @@ private[spark] class JobProgressUI(val sc: SparkContext) { def listener = _listener.get val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) + private val poolPage = new PoolPage(this) + + var stageTable: StageTable = null + var stagePoolInfo: StagePoolInfo = null + var poolTable: PoolTable = null + var stagePagePoolSource: PoolSource = null def start() { - _listener = Some(new JobProgressListener) + sc.getSchedulingMode match { + case SchedulingMode.FIFO => + _listener = Some(new JobProgressListener(sc)) + stagePoolInfo = new FIFOStagePoolInfo() + stagePagePoolSource = new FIFOSource() + case SchedulingMode.FAIR => + _listener = Some(new FairJobProgressListener(sc)) + stagePoolInfo = new FairStagePoolInfo(listener) + stagePagePoolSource = new FairSource(sc) + } + sc.addSparkListener(listener) + stageTable = new StageTable(dateFmt, formatDuration, listener) + poolTable = new PoolTable(listener) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), + ("/stages/pool", (request: HttpServletRequest) => poolPage.render(request)), ("/stages", (request: HttpServletRequest) => indexPage.render(request)) ) } - -private[spark] class JobProgressListener extends SparkListener { - // How many stages to remember - val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt - - val activeStages = HashSet[Stage]() - val completedStages = ListBuffer[Stage]() - val failedStages = ListBuffer[Stage]() - - val stageToTasksComplete = HashMap[Int, Int]() - val stageToTasksFailed = HashMap[Int, Int]() - val stageToTaskInfos = - HashMap[Int, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() - - override def onJobStart(jobStart: SparkListenerJobStart) {} - - override def onStageCompleted(stageCompleted: StageCompleted) = { - val stage = stageCompleted.stageInfo.stage - activeStages -= stage - completedStages += stage - trimIfNecessary(completedStages) - } - - /** If stages is too large, remove and garbage collect old stages */ - def trimIfNecessary(stages: ListBuffer[Stage]) { - if (stages.size > RETAINED_STAGES) { - val toRemove = RETAINED_STAGES / 10 - stages.takeRight(toRemove).foreach( s => { - stageToTaskInfos.remove(s.id) - }) - stages.trimEnd(toRemove) - } - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = - activeStages += stageSubmitted.stage - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val sid = taskEnd.task.stageId - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics) - case _ => - stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) - } - val taskList = stageToTaskInfos.getOrElse( - sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList += ((taskEnd.taskInfo, metrics, failureInfo)) - stageToTaskInfos(sid) = taskList - } - - override def onJobEnd(jobEnd: SparkListenerJobEnd) { - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - activeStages -= stage - failedStages += stage - trimIfNecessary(failedStages) - case _ => - } - case _ => - } - } - - /** Is this stage's input from a shuffle read. */ - def hasShuffleRead(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined - } - return false // No tasks have finished for this stage - } - - /** Is this stage's output to a shuffle write. */ - def hasShuffleWrite(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined - } - return false // No tasks have finished for this stage - } -} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala new file mode 100644 index 0000000000..00703887c3 --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -0,0 +1,38 @@ +package spark.ui.jobs + +import javax.servlet.http.HttpServletRequest + +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashSet + +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ + +/** Page showing specific pool details*/ +private[spark] class PoolPage(parent: JobProgressUI) { + def listener = parent.listener + + def stageTable: StageTable = parent.stageTable + + def poolTable: PoolTable = parent.poolTable + + def render(request: HttpServletRequest): Seq[Node] = { + val poolName = request.getParameter("poolname") + val poolToActiveStages = listener.poolToActiveStages + val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq + val stageToPool = listener.stageToPool + + val poolDetailPoolSource = new PoolDetailSource(parent.sc, poolName) + poolTable.setPoolSource(poolDetailPoolSource) + + stageTable.setStagePoolInfo(parent.stagePoolInfo) + + val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + + val content =

Pool

++ poolTable.toNodeSeq ++ +

Active Stages : {activeStages.size}

++ activeStageNodeSeq + + headerSparkPage(content, parent.sc, "Spark Pool Details", Jobs) + } +} diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala new file mode 100644 index 0000000000..bb8be4b26e --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -0,0 +1,98 @@ +package spark.ui.jobs + +import java.util.Date + +import javax.servlet.http.HttpServletRequest + +import scala.Some +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + +import spark.SparkContext +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ +import spark.storage.StorageLevel +import spark.scheduler.cluster.Schedulable + +/* + * Interface for get pools seq showing on Index or pool detail page + */ + +private[spark] trait PoolSource { + def getPools: Seq[Schedulable] +} + +/* + * Pool source for FIFO scheduler algorithm on Index page + */ +private[spark] class FIFOSource() extends PoolSource{ + def getPools: Seq[Schedulable] = { + Seq[Schedulable]() + } +} + +/* + * Pool source for Fair scheduler algorithm on Index page + */ +private[spark] class FairSource(sc: SparkContext) extends PoolSource{ + def getPools: Seq[Schedulable] = { + sc.getPoolsInfo.toSeq + } +} + +/* + * specific pool info for pool detail page + */ +private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource{ + def getPools: Seq[Schedulable] = { + val pools = HashSet[Schedulable]() + pools += sc.getPoolNameToPool(poolName) + pools.toSeq + } +} + +/** Table showing list of pools */ +private[spark] class PoolTable(listener: JobProgressListener) { + + var poolSource: PoolSource = null + var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages + + def toNodeSeq: Seq[Node] = { + poolTable(poolRow, poolSource.getPools) + } + + def setPoolSource(poolSource: PoolSource) { + this.poolSource = poolSource + } + + //pool tables + def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], rows: Seq[Schedulable]): Seq[Node] = { + + + + + + + + + + + {rows.map(r => makeRow(r, poolToActiveStages))} + +
Pool NameMinimum SharePool WeightActive StagesRunning TasksSchedulingMode
+ } + + def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[Stage]]): Seq[Node] = { + + {p.name} + {p.minShare} + {p.weight} + {poolToActiveStages.getOrElseUpdate(p.name, new HashSet[Stage]()).size} + {p.runningTasks} + {p.schedulingMode} + + } +} + diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala new file mode 100644 index 0000000000..83e566c55b --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -0,0 +1,143 @@ +package spark.ui.jobs + +import java.util.Date +import java.text.SimpleDateFormat + +import javax.servlet.http.HttpServletRequest + +import scala.Some +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap + +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ +import spark.storage.StorageLevel + +/* + * Interface to get stage's pool name + */ +private[spark] trait StagePoolInfo { + def getStagePoolName(s: Stage): String + + def hasHerf: Boolean +} + +/* + * For FIFO scheduler algorithm, just show "N/A" and its link status is false + */ +private[spark] class FIFOStagePoolInfo extends StagePoolInfo { + def getStagePoolName(s: Stage): String = "N/A" + + def hasHerf: Boolean = false +} + +/* + * For Fair scheduler algorithm, show its pool name and pool detail link status is true + */ +private[spark] class FairStagePoolInfo(listener: JobProgressListener) extends StagePoolInfo { + def getStagePoolName(s: Stage): String = { + listener.stageToPool(s) + } + + def hasHerf: Boolean = true +} + +/** Page showing list of all ongoing and recently finished stages */ +private[spark] class StageTable(val dateFmt: SimpleDateFormat, val formatDuration: Long => String, val listener: JobProgressListener) { + + var stagePoolInfo: StagePoolInfo = null + + def toNodeSeq(stages: Seq[Stage]): Seq[Node] = { + stageTable(stageRow, stages) + } + + def setStagePoolInfo(stagePoolInfo: StagePoolInfo) { + this.stagePoolInfo = stagePoolInfo + } + + /** Special table which merges two header cells. */ + def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + + + + + + + + + + + + {rows.map(r => makeRow(r))} + +
Stage IdPool NameOriginSubmittedDurationTasks: Complete/TotalShuffle ActivityStored RDD
+ } + + def getElapsedTime(submitted: Option[Long], completed: Long): String = { + submitted match { + case Some(t) => formatDuration(completed - t) + case _ => "Unknown" + } + } + + def makeProgressBar(completed: Int, total: Int): Seq[Node] = { + val width=130 + val height=15 + val completeWidth = (completed.toDouble / total) * width + + + + + + } + + + def stageRow(s: Stage): Seq[Node] = { + val submissionTime = s.submissionTime match { + case Some(t) => dateFmt.format(new Date(t)) + case None => "Unknown" + } + val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) + val shuffleInfo = (read, write) match { + case (true, true) => "Read/Write" + case (true, false) => "Read" + case (false, true) => "Write" + case _ => "" + } + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) + val totalTasks = s.numPartitions + + val poolName = stagePoolInfo.getStagePoolName(s) + + + {s.id} + {if (stagePoolInfo.hasHerf) { + {poolName} + } else { + {poolName} + }} + {s.origin} + {submissionTime} + {getElapsedTime(s.submissionTime, + s.completionTime.getOrElse(System.currentTimeMillis()))} + {makeProgressBar(completedTasks, totalTasks)} + {completedTasks} / {totalTasks} + {listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => + }} + + {shuffleInfo} + {if (s.rdd.getStorageLevel != StorageLevel.NONE) { + + {Option(s.rdd.name).getOrElse(s.rdd.id)} + + }} + + + } +} diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef950..da72bfbf89 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -22,6 +22,10 @@ import spark.TaskEndReason import spark.{FetchFailed, Success} +import spark.scheduler.cluster.Pool +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode + /** * 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 @@ -39,6 +43,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() val taskScheduler = new TaskScheduler() { + override def rootPool: Pool = null + override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start() = {} override def stop() = {} override def submitTasks(taskSet: TaskSet) = { diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 699901f1a1..328e7e7529 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(SparkListenerStageSubmitted(rootStage, 4)) + joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4, null)) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") joblogger.getRddNameTest(parentRdd) should be ("MyRDD") From 0c391feb73610e56beb542bbf66c56efab01bada Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 16:39:34 -0700 Subject: [PATCH 04/75] Maximum task failures configurable --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d72b0bfc9f..c15aeb5ff0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( 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 + val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail","4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble From 6a47cee72104c9f6d264cb2e61ae8a38523cfdad Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 16:41:37 -0700 Subject: [PATCH 05/75] style --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index c15aeb5ff0..685fc016b2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( 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 = System.getProperty("spark.task.max.fail","4").toInt + val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail", "4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble From f5067abe85922b8075e03f39a45e4ac39267c62e Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Sat, 27 Jul 2013 23:08:00 -0700 Subject: [PATCH 06/75] changes per comments. --- .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- docs/configuration.md | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 685fc016b2..d541234548 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( 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 = System.getProperty("spark.task.max.fail", "4").toInt + val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble diff --git a/docs/configuration.md b/docs/configuration.md index 17fdbf04d1..e217b499c7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -260,6 +260,14 @@ Apart from these, the following properties are also available, and may be useful applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. + + spark.task.maxFailures + 4 + + Number of individual task failures before giving up on the job. + Should greater or equal to 1. Number of allowed retries = this value - 1. + + From 0862494d442199402e37dc968dadf9a5bec9ae78 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Sat, 27 Jul 2013 23:16:20 -0700 Subject: [PATCH 07/75] typo --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index e217b499c7..6ca5cb99f9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -265,7 +265,7 @@ Apart from these, the following properties are also available, and may be useful 4 Number of individual task failures before giving up on the job. - Should greater or equal to 1. Number of allowed retries = this value - 1. + Should be greater than or equal to 1. Number of allowed retries = this value - 1. From 614ee16cc4c63260f13d0c7494fbaafa8a061e95 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 30 Jul 2013 10:57:26 +0800 Subject: [PATCH 08/75] refactor job ui with pool information --- core/src/main/scala/spark/SparkContext.scala | 16 +++-- .../scala/spark/scheduler/SparkListener.scala | 2 +- .../scheduler/cluster/ClusterScheduler.scala | 12 ++-- .../cluster/ClusterTaskSetManager.scala | 7 +-- .../spark/scheduler/cluster/Schedulable.scala | 2 +- .../cluster/SchedulableBuilder.scala | 6 +- .../scheduler/cluster/SchedulingMode.scala | 6 +- .../scheduler/cluster/TaskSetManager.scala | 1 + .../scheduler/local/LocalScheduler.scala | 5 +- .../scheduler/local/LocalTaskSetManager.scala | 7 +-- .../scala/spark/ui/UIWorkloadGenerator.scala | 35 ++++++++--- .../main/scala/spark/ui/jobs/IndexPage.scala | 28 +++------ .../spark/ui/jobs/JobProgressListener.scala | 62 +++++-------------- .../scala/spark/ui/jobs/JobProgressUI.scala | 7 +-- .../main/scala/spark/ui/jobs/PoolPage.scala | 17 ++--- .../main/scala/spark/ui/jobs/PoolTable.scala | 23 +++---- .../main/scala/spark/ui/jobs/StageTable.scala | 16 ++--- 17 files changed, 116 insertions(+), 136 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b5225d5681..375636071d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -45,13 +45,13 @@ import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, +SchedulingMode} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} -import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -542,17 +542,25 @@ class SparkContext( env.blockManager.master.getStorageStatus } - def getPoolsInfo: ArrayBuffer[Schedulable] = { + /** + * Return pools for fair scheduler + * TODO:now, we have not taken nested pools into account + */ + def getPools: ArrayBuffer[Schedulable] = { taskScheduler.rootPool.schedulableQueue } - def getSchedulingMode: SchedulingMode = { + /** + * Return current scheduling mode + */ + def getSchedulingMode: SchedulingMode.SchedulingMode = { taskScheduler.schedulingMode } def getPoolNameToPool: HashMap[String, Schedulable] = { taskScheduler.rootPool.schedulableNameToSchedulable } + /** * 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/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 94fdad9b98..07372ee786 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -8,7 +8,7 @@ import spark.executor.TaskMetrics sealed trait SparkListenerEvents -case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties = null) extends SparkListenerEvents +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties) extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 1b23fd6cef..74b3e43d2b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -98,8 +98,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null - //default scheduler is FIFO - val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) + // default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName( + System.getProperty("spark.cluster.schedulingmode", "FIFO")) override def setListener(listener: TaskSchedulerListener) { this.listener = listener @@ -107,7 +108,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context - //temporarily set rootPool name to empty + // temporarily set rootPool name to empty rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { @@ -254,10 +255,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } var launchedTask = false val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() - for (manager <- sortedTaskSetQueue) - { + + for (manager <- sortedTaskSetQueue) { logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } + for (manager <- sortedTaskSetQueue) { // Split offers based on node local, rack local and off-rack tasks. diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 7a6a6b7826..4d11b0959a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -91,7 +91,6 @@ private[spark] class ClusterTaskSetManager( var stageId = taskSet.stageId var name = "TaskSet_"+taskSet.stageId.toString var parent: Schedulable = null - var schedulableQueue :ArrayBuffer[Schedulable] = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -645,17 +644,17 @@ private[spark] class ClusterTaskSetManager( } } - //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + // 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 + // nothing } override def removeSchedulable(schedulable:Schedulable) { - //nothing + // nothing } override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2e4f14c11f..c410af8af4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -9,7 +9,7 @@ import scala.collection.mutable.ArrayBuffer */ private[spark] trait Schedulable { var parent: Schedulable - //childrens + // child queues def schedulableQueue: ArrayBuffer[Schedulable] def schedulingMode: SchedulingMode def weight: Int diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index 18cc15c2a5..a2fa80aa36 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -27,7 +27,7 @@ private[spark] trait SchedulableBuilder { private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { override def buildPools() { - //nothing + // nothing } override def addTaskSetManager(manager: Schedulable, properties: Properties) { @@ -86,7 +86,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula } } - //finally create "default" pool + // 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) @@ -102,7 +102,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula 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 + // 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( diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index c5c7ee3b22..a7f0f6f393 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,6 +1,10 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE"){ +/** + * "FAIR" and "FIFO" determines which policy is used to order tasks amongst a Schedulable's sub-queues + * "NONE" is used when the a Schedulable has no sub-queues. + */ +object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") { type SchedulingMode = Value val FAIR,FIFO,NONE = Value diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 472e01b227..4e6bc51278 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -6,6 +6,7 @@ import spark.TaskState.TaskState import java.nio.ByteBuffer private[spark] trait TaskSetManager extends Schedulable { + def schedulableQueue = null def schedulingMode = SchedulingMode.NONE def taskSet: TaskSet def slaveOffer(execId: String, hostPort: String, availableCpus: Double, diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 19a48895e3..f4411582f1 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -64,7 +64,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null - val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) + val schedulingMode: SchedulingMode = SchedulingMode.withName( + System.getProperty("spark.cluster.schedulingmode", "FIFO")) val activeTaskSets = new HashMap[String, TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -72,7 +73,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var localActor: ActorRef = null override def start() { - //temporarily set rootPool name to empty + // temporarily set rootPool name to empty rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 8954f40ea9..cc27f1ecca 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -14,7 +14,6 @@ import spark.scheduler.cluster._ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null - var schedulableQueue :ArrayBuffer[Schedulable] = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 @@ -48,11 +47,11 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } def addSchedulable(schedulable: Schedulable): Unit = { - //nothing + // nothing } def removeSchedulable(schedulable: Schedulable): Unit = { - //nothing + // nothing } def getSchedulableByName(name: String): Schedulable = { @@ -60,7 +59,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } def executorLost(executorId: String, host: String): Unit = { - //nothing + // nothing } def checkSpeculatableTasks(): Boolean = { diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 8bbc6ce88e..840ac9773e 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -4,7 +4,8 @@ import scala.util.Random import spark.SparkContext import spark.SparkContext._ - +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Continuously generates jobs that expose various features of the WebUI (internal testing tool). * @@ -15,8 +16,17 @@ private[spark] object UIWorkloadGenerator { val INTER_JOB_WAIT_MS = 500 def main(args: Array[String]) { + if (args.length < 2) { + println("usage: ./run spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + System.exit(1) + } val master = args(0) + val schedulingMode = SchedulingMode.withName(args(1)) val appName = "Spark UI Tester" + + if (schedulingMode == SchedulingMode.FAIR) { + System.setProperty("spark.cluster.schedulingmode", "FAIR") + } val sc = new SparkContext(master, appName) // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, @@ -56,14 +66,21 @@ private[spark] object UIWorkloadGenerator { while (true) { for ((desc, job) <- jobs) { - try { - setName(desc) - job() - println("Job funished: " + desc) - } catch { - case e: Exception => - println("Job Failed: " + desc) - } + new Thread { + override def run() { + if(schedulingMode == SchedulingMode.FAIR) { + sc.addLocalProperties("spark.scheduler.cluster.fair.pool",desc) + } + try { + setName(desc) + job() + println("Job funished: " + desc) + } catch { + case e: Exception => + println("Job Failed: " + desc) + } + } + }.start Thread.sleep(INTER_JOB_WAIT_MS) } } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index e765cecb01..abef683791 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -4,38 +4,30 @@ import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.Some -import scala.xml.{NodeSeq, Node} import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.Some +import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage +import spark.storage.StorageLevel import spark.ui.UIUtils._ import spark.ui.Page._ -import spark.storage.StorageLevel -import spark.scheduler.cluster.Schedulable -import spark.scheduler.cluster.SchedulingMode -import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener - def stageTable: StageTable = parent.stageTable - - def poolTable: PoolTable = parent.poolTable - def render(request: HttpServletRequest): Seq[Node] = { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - stageTable.setStagePoolInfo(parent.stagePoolInfo) - poolTable.setPoolSource(parent.stagePagePoolSource) + val activeStagesTable = new StageTable(activeStages, parent) + val completedStagesTable = new StageTable(completedStages, parent) + val failedStagesTable = new StageTable(failedStages, parent) - val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) - val completedStageNodeSeq = stageTable.toNodeSeq(completedStages) - val failedStageNodeSeq = stageTable.toNodeSeq(failedStages) + val poolTable = new PoolTable(parent.stagePagePoolSource, listener) val content =
@@ -48,9 +40,9 @@ private[spark] class IndexPage(parent: JobProgressUI) {
++

Pools

++ poolTable.toNodeSeq ++ -

Active Stages : {activeStages.size}

++ activeStageNodeSeq++ -

Completed Stages : {completedStages.size}

++ completedStageNodeSeq++ -

Failed Stages : {failedStages.size}

++ failedStageNodeSeq +

Active Stages : {activeStages.size}

++ activeStagesTable.toNodeSeq++ +

Completed Stages : {completedStages.size}

++ completedStagesTable.toNodeSeq++ +

Failed Stages : {failedStages.size}

++ failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages/Pools", Jobs) } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala index 1244f9538b..d4767bea22 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -9,53 +9,13 @@ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics import collection.mutable -private[spark] class FairJobProgressListener(val sparkContext: SparkContext) - extends JobProgressListener(sparkContext) { - - val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" - val DEFAULT_POOL_NAME = "default" - - override val stageToPool = HashMap[Stage, String]() - override val poolToActiveStages = HashMap[String, HashSet[Stage]]() - - override def onStageCompleted(stageCompleted: StageCompleted) = { - super.onStageCompleted(stageCompleted) - val stage = stageCompleted.stageInfo.stage - poolToActiveStages(stageToPool(stage)) -= stage - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { - super.onStageSubmitted(stageSubmitted) - val stage = stageSubmitted.stage - var poolName = DEFAULT_POOL_NAME - if (stageSubmitted.properties != null) { - poolName = stageSubmitted.properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) - } - stageToPool(stage) = poolName - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) - stages += stage - } - - override def onJobEnd(jobEnd: SparkListenerJobEnd) { - super.onJobEnd(jobEnd) - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - poolToActiveStages(stageToPool(stage)) -= stage - case _ => - } - case _ => - } - } -} - private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt + val DEFAULT_POOL_NAME = "default" - def stageToPool: HashMap[Stage, String] = null - def poolToActiveStages: HashMap[String, HashSet[Stage]] =null + val stageToPool = new HashMap[Stage, String]() + val poolToActiveStages = new HashMap[String, HashSet[Stage]]() val activeStages = HashSet[Stage]() val completedStages = ListBuffer[Stage]() @@ -70,6 +30,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList override def onStageCompleted(stageCompleted: StageCompleted) = { val stage = stageCompleted.stageInfo.stage + poolToActiveStages(stageToPool(stage)) -= stage activeStages -= stage completedStages += stage trimIfNecessary(completedStages) @@ -86,8 +47,18 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = - activeStages += stageSubmitted.stage + /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { + val stage = stageSubmitted.stage + activeStages += stage + var poolName = DEFAULT_POOL_NAME + if (stageSubmitted.properties != null) { + poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME) + } + stageToPool(stage) = poolName + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) + stages += stage + } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId @@ -112,6 +83,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList end.jobResult match { case JobFailed(ex, Some(stage)) => activeStages -= stage + poolToActiveStages(stageToPool(stage)) -= stage failedStages += stage trimIfNecessary(failedStages) case _ => diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index e610252242..5703b146df 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -29,26 +29,21 @@ private[spark] class JobProgressUI(val sc: SparkContext) { private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) - var stageTable: StageTable = null var stagePoolInfo: StagePoolInfo = null - var poolTable: PoolTable = null var stagePagePoolSource: PoolSource = null def start() { + _listener = Some(new JobProgressListener(sc)) sc.getSchedulingMode match { case SchedulingMode.FIFO => - _listener = Some(new JobProgressListener(sc)) stagePoolInfo = new FIFOStagePoolInfo() stagePagePoolSource = new FIFOSource() case SchedulingMode.FAIR => - _listener = Some(new FairJobProgressListener(sc)) stagePoolInfo = new FairStagePoolInfo(listener) stagePagePoolSource = new FairSource(sc) } sc.addSparkListener(listener) - stageTable = new StageTable(dateFmt, formatDuration, listener) - poolTable = new PoolTable(listener) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala index 00703887c3..37d4f8fa6b 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -9,30 +9,23 @@ import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ -/** Page showing specific pool details*/ +/** Page showing specific pool details */ private[spark] class PoolPage(parent: JobProgressUI) { def listener = parent.listener - def stageTable: StageTable = parent.stageTable - - def poolTable: PoolTable = parent.poolTable - def render(request: HttpServletRequest): Seq[Node] = { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq - val stageToPool = listener.stageToPool val poolDetailPoolSource = new PoolDetailSource(parent.sc, poolName) - poolTable.setPoolSource(poolDetailPoolSource) + val poolTable = new PoolTable(poolDetailPoolSource, listener) - stageTable.setStagePoolInfo(parent.stagePoolInfo) + val activeStagesTable = new StageTable(activeStages, parent) - val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + val content =

Pool

++ poolTable.toNodeSeq() ++ +

Active Stages : {activeStages.size}

++ activeStagesTable.toNodeSeq() - val content =

Pool

++ poolTable.toNodeSeq ++ -

Active Stages : {activeStages.size}

++ activeStageNodeSeq - headerSparkPage(content, parent.sc, "Spark Pool Details", Jobs) } } diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index bb8be4b26e..8788ed8bc1 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -27,7 +27,7 @@ private[spark] trait PoolSource { /* * Pool source for FIFO scheduler algorithm on Index page */ -private[spark] class FIFOSource() extends PoolSource{ +private[spark] class FIFOSource() extends PoolSource { def getPools: Seq[Schedulable] = { Seq[Schedulable]() } @@ -36,16 +36,16 @@ private[spark] class FIFOSource() extends PoolSource{ /* * Pool source for Fair scheduler algorithm on Index page */ -private[spark] class FairSource(sc: SparkContext) extends PoolSource{ +private[spark] class FairSource(sc: SparkContext) extends PoolSource { def getPools: Seq[Schedulable] = { - sc.getPoolsInfo.toSeq + sc.getPools.toSeq } } /* * specific pool info for pool detail page */ -private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource{ +private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource { def getPools: Seq[Schedulable] = { val pools = HashSet[Schedulable]() pools += sc.getPoolNameToPool(poolName) @@ -54,21 +54,18 @@ private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extend } /** Table showing list of pools */ -private[spark] class PoolTable(listener: JobProgressListener) { +private[spark] class PoolTable(poolSource: PoolSource, listener: JobProgressListener) { - var poolSource: PoolSource = null var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages - def toNodeSeq: Seq[Node] = { + def toNodeSeq(): Seq[Node] = { poolTable(poolRow, poolSource.getPools) } - def setPoolSource(poolSource: PoolSource) { - this.poolSource = poolSource - } - - //pool tables - def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], rows: Seq[Schedulable]): Seq[Node] = { + // pool tables + def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], + rows: Seq[Schedulable] + ): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 83e566c55b..82fb0bd5cc 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -44,18 +44,18 @@ private[spark] class FairStagePoolInfo(listener: JobProgressListener) extends St } /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable(val dateFmt: SimpleDateFormat, val formatDuration: Long => String, val listener: JobProgressListener) { +private[spark] class StageTable( + val stages: Seq[Stage], + val parent: JobProgressUI) { - var stagePoolInfo: StagePoolInfo = null + val listener = parent.listener + val dateFmt = parent.dateFmt + var stagePoolInfo = parent.stagePoolInfo - def toNodeSeq(stages: Seq[Stage]): Seq[Node] = { + def toNodeSeq(): Seq[Node] = { stageTable(stageRow, stages) } - def setStagePoolInfo(stagePoolInfo: StagePoolInfo) { - this.stagePoolInfo = stagePoolInfo - } - /** Special table which merges two header cells. */ def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = {
Pool Name
@@ -77,7 +77,7 @@ private[spark] class StageTable(val dateFmt: SimpleDateFormat, val formatDuratio def getElapsedTime(submitted: Option[Long], completed: Long): String = { submitted match { - case Some(t) => formatDuration(completed - t) + case Some(t) => parent.formatDuration(completed - t) case _ => "Unknown" } } From 5406013997d6b9e9e3f43c09cd3f53ec7b815efb Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 30 Jul 2013 11:41:38 +0800 Subject: [PATCH 09/75] refactor codes less than 100 character per line --- core/src/main/scala/spark/SparkContext.scala | 7 ++++--- .../main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- .../scala/spark/scheduler/SparkListener.scala | 15 ++++++++------- .../scheduler/cluster/ClusterScheduler.scala | 5 +++-- .../scheduler/cluster/ClusterTaskSetManager.scala | 3 ++- .../scheduler/cluster/SchedulableBuilder.scala | 15 ++++++++++----- .../spark/scheduler/cluster/SchedulingMode.scala | 3 ++- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 9 ++++++--- .../scala/spark/ui/jobs/JobProgressListener.scala | 3 ++- 9 files changed, 39 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 375636071d..7747160290 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -44,9 +44,10 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, -SchedulingMode} +import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, + SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, + ClusterScheduler, Schedulable, SchedulingMode} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c865743e37..5fda78e152 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -473,7 +473,8 @@ class DAGScheduler( } if (tasks.size > 0) { val properties = idToActiveJob(stage.priority).properties - sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size, properties))) + sparkListeners.foreach(_.onStageSubmitted( + SparkListenerStageSubmitted(stage, tasks.size, properties))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 07372ee786..49f7c85c29 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -8,17 +8,18 @@ import spark.executor.TaskMetrics sealed trait SparkListenerEvents -case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties) extends SparkListenerEvents +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties) + extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents -case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) +case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) extends SparkListenerEvents -case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) +case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) extends SparkListenerEvents trait SparkListener { @@ -26,12 +27,12 @@ trait SparkListener { * Called when a stage is completed, with information on the completed stage */ def onStageCompleted(stageCompleted: StageCompleted) { } - + /** * Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } - + /** * Called when a task ends */ @@ -41,12 +42,12 @@ trait SparkListener { * Called when a job starts */ def onJobStart(jobStart: SparkListenerJobStart) { } - + /** * Called when a job ends */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } - + } /** diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 74b3e43d2b..20680bbf87 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -9,10 +9,10 @@ import scala.collection.mutable.HashSet import spark._ import spark.TaskState.TaskState import spark.scheduler._ +import spark.scheduler.cluster.SchedulingMode.SchedulingMode import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} -import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call @@ -257,7 +257,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() for (manager <- sortedTaskSetQueue) { - logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + logInfo("parentName:%s, name:%s, runningTasks:%s".format( + manager.parent.name, manager.name, manager.runningTasks)) } for (manager <- sortedTaskSetQueue) { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 4d11b0959a..5e2351bafd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -644,7 +644,8 @@ private[spark] class ClusterTaskSetManager( } } - // TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + // 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 } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index a2fa80aa36..db51b48494 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -24,7 +24,8 @@ private[spark] trait SchedulableBuilder { def addTaskSetManager(manager: Schedulable, properties: Properties) } -private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { +private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) + extends SchedulableBuilder with Logging { override def buildPools() { // nothing @@ -35,7 +36,8 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends Schedula } } -private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { +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" @@ -88,7 +90,8 @@ 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) + 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)) @@ -102,8 +105,10 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula 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) + // 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)) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index a7f0f6f393..cd0642772d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,7 +1,8 @@ package spark.scheduler.cluster /** - * "FAIR" and "FIFO" determines which policy is used to order tasks amongst a Schedulable's sub-queues + * "FAIR" and "FIFO" determines which policy is used + * to order tasks amongst a Schedulable's sub-queues * "NONE" is used when the a Schedulable has no sub-queues. */ object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index abef683791..04651e9c60 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -40,9 +40,12 @@ private[spark] class IndexPage(parent: JobProgressUI) { ++

Pools

++ poolTable.toNodeSeq ++ -

Active Stages : {activeStages.size}

++ activeStagesTable.toNodeSeq++ -

Completed Stages : {completedStages.size}

++ completedStagesTable.toNodeSeq++ -

Failed Stages : {failedStages.size}

++ failedStagesTable.toNodeSeq +

Active Stages : {activeStages.size}

++ + activeStagesTable.toNodeSeq++ +

Completed Stages : {completedStages.size}

++ + completedStagesTable.toNodeSeq++ +

Failed Stages : {failedStages.size}

++ + failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages/Pools", Jobs) } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala index d4767bea22..da767b3c0a 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -53,7 +53,8 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList activeStages += stage var poolName = DEFAULT_POOL_NAME if (stageSubmitted.properties != null) { - poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME) + poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", + DEFAULT_POOL_NAME) } stageToPool(stage) = poolName val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) From f6f46455eb4f1199eb9a464b1a0b45d9926f7ff8 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Tue, 23 Jul 2013 09:33:13 -0400 Subject: [PATCH 10/75] Added property 'spark.executor.uri' for launching on Mesos without requiring Spark to be installed. Using 'make_distribution.sh' a user can put a Spark distribution at a URI supported by Mesos (e.g., 'hdfs://...') and then set that when launching their job. Also added SPARK_EXECUTOR_URI for the REPL. --- .../mesos/CoarseMesosSchedulerBackend.scala | 27 ++++++++++++++----- .../mesos/MesosSchedulerBackend.scala | 17 ++++++++---- make-distribution.sh | 1 + project/SparkBuild.scala | 2 +- .../main/scala/spark/repl/SparkILoop.scala | 2 ++ 5 files changed, 36 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 7bc6040544..f75244a9ac 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -110,12 +110,6 @@ private[spark] class CoarseMesosSchedulerBackend( } def createCommand(offer: Offer, numCores: Int): CommandInfo = { - val runScript = new File(sparkHome, "run").getCanonicalPath - 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, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores) val environment = Environment.newBuilder() sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() @@ -123,7 +117,26 @@ private[spark] class CoarseMesosSchedulerBackend( .setValue(value) .build()) } - return CommandInfo.newBuilder().setValue(command).setEnvironment(environment).build() + val command = CommandInfo.newBuilder() + .setEnvironment(environment) + val driverUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.driver.host"), + System.getProperty("spark.driver.port"), + StandaloneSchedulerBackend.ACTOR_NAME) + val uri = System.getProperty("spark.executor.uri") + if (uri == null) { + val runScript = new File(sparkHome, "run").getCanonicalPath + command.setValue("\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + } else { + // Grab everything to the first '.'. We'll use that and '*' to + // glob the directory "correctly". + val basename = new File(uri).getName().split('.')(0) + command.setValue("cd %s*; ./run spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + } + return command.build() } override def offerRescinded(d: SchedulerDriver, o: OfferID) {} diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 75b8268b55..51b780ac72 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -89,7 +89,6 @@ private[spark] class MesosSchedulerBackend( 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) => environment.addVariables(Environment.Variable.newBuilder() @@ -97,15 +96,23 @@ private[spark] class MesosSchedulerBackend( .setValue(value) .build()) } + val command = CommandInfo.newBuilder() + .setEnvironment(environment) + val uri = System.getProperty("spark.executor.uri") + if (uri == null) { + command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) + } else { + // Grab everything to the first '.'. We'll use that and '*' to + // glob the directory "correctly". + val basename = new File(uri).getName().split('.')(0) + command.setValue("cd %s*; ./spark-executor".format(basename)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + } val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build()) .build() - val command = CommandInfo.newBuilder() - .setValue(execScript) - .setEnvironment(environment) - .build() ExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) diff --git a/make-distribution.sh b/make-distribution.sh index 4374e0e8c4..0a8941c1f8 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -66,6 +66,7 @@ cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/conf" "$DISTDIR" cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" +cp "$FWDIR/spark-executor" "$DISTDIR" if [ "$1" == "tgz" ]; then diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9920e00a67..a2ea9c9694 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -176,7 +176,7 @@ object SparkBuild extends Build { "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", - "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "org.apache.mesos" % "mesos" % "0.12.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 59f9d05683..0bfe7bb743 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -831,6 +831,8 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: var sparkContext: SparkContext = null def createSparkContext(): SparkContext = { + val uri = System.getenv("SPARK_EXECUTOR_URI") + if (uri != null) System.setProperty("spark.executor.uri", uri) val master = this.master match { case Some(m) => m case None => { From 48851d4dd90184fdeee836caef5ad77b0dc643be Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 30 Jul 2013 14:03:15 -0700 Subject: [PATCH 11/75] Add bagel, mllib to SBT assembly. Also add jblas dependency to mllib pom.xml --- mllib/pom.xml | 165 +++++++++++++++++++++++++++++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 6 +- 3 files changed, 170 insertions(+), 2 deletions(-) create mode 100644 mllib/pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml new file mode 100644 index 0000000000..f3928cc73d --- /dev/null +++ b/mllib/pom.xml @@ -0,0 +1,165 @@ + + + + + 4.0.0 + + org.spark-project + spark-parent + 0.8.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-mllib + jar + Spark Project ML Library + http://spark-project.org/ + + + + org.eclipse.jetty + jetty-server + + + org.jblas + jblas + 1.2.3 + + + + org.scalatest + scalatest_${scala.version} + test + + + org.scalacheck + scalacheck_${scala.version} + test + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + 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/pom.xml b/pom.xml index 44729bd422..03b36f2ee8 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ core bagel examples + mllib tools streaming repl diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9920e00a67..0fdd291aac 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -242,14 +242,16 @@ object SparkBuild extends Build { name := "spark-tools" ) - def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") + def bagelSettings = sharedSettings ++ Seq( + name := "spark-bagel" + ) ++ assemblySettings ++ extraAssemblySettings def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) - ) + ) ++ assemblySettings ++ extraAssemblySettings def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", From 7bdafa918ac5850197dacadd3ea56ef544a6a9f8 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Tue, 30 Jul 2013 17:01:00 -0700 Subject: [PATCH 12/75] Format cleanup. --- repl/src/main/scala/spark/repl/SparkILoop.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 0bfe7bb743..0cecbd71ad 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -832,7 +832,9 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def createSparkContext(): SparkContext = { val uri = System.getenv("SPARK_EXECUTOR_URI") - if (uri != null) System.setProperty("spark.executor.uri", uri) + if (uri != null) { + System.setProperty("spark.executor.uri", uri) + } val master = this.master match { case Some(m) => m case None => { From ad7e9d0d64277f616f90f2ca8bf8a5844641883a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:11:54 -0700 Subject: [PATCH 13/75] CompressionCodec cleanup. Moved it to spark.io package. --- .../scala/spark/io/CompressionCodec.scala | 82 +++++++++++++++++++ .../scala/spark/storage/BlockManager.scala | 32 ++------ .../spark/storage/CompressionCodec.scala | 13 --- .../spark/storage/LZFCompressionCodec.scala | 16 ---- .../storage/SnappyCompressionCodec.scala | 18 ---- .../spark/io/CompressionCodecSuite.scala | 0 6 files changed, 91 insertions(+), 70 deletions(-) create mode 100644 core/src/main/scala/spark/io/CompressionCodec.scala delete mode 100644 core/src/main/scala/spark/storage/CompressionCodec.scala delete mode 100644 core/src/main/scala/spark/storage/LZFCompressionCodec.scala delete mode 100644 core/src/main/scala/spark/storage/SnappyCompressionCodec.scala create mode 100644 core/src/test/scala/spark/io/CompressionCodecSuite.scala diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/spark/io/CompressionCodec.scala new file mode 100644 index 0000000000..2ba104a737 --- /dev/null +++ b/core/src/main/scala/spark/io/CompressionCodec.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.io + +import java.io.{InputStream, OutputStream} + +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} + +import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} + + +/** + * CompressionCodec allows the customization of choosing different compression implementations + * to be used in block storage. + */ +trait CompressionCodec { + + def compressionOutputStream(s: OutputStream): OutputStream + + def compressionInputStream(s: InputStream): InputStream +} + + +private[spark] object CompressionCodec { + + def createCodec(): CompressionCodec = { + // Set the default codec to Snappy since the LZF implementation initializes a pretty large + // buffer for every stream, which results in a lot of memory overhead when the number of + // shuffle reduce buckets are large. + createCodec(classOf[SnappyCompressionCodec].getName) + } + + def createCodec(codecName: String): CompressionCodec = { + Class.forName( + System.getProperty("spark.io.compression.codec", codecName), + true, + Thread.currentThread.getContextClassLoader).newInstance().asInstanceOf[CompressionCodec] + } +} + + +/** + * LZF implementation of [[spark.io.CompressionCodec]]. + */ +class LZFCompressionCodec extends CompressionCodec { + + override def compressionOutputStream(s: OutputStream): OutputStream = { + new LZFOutputStream(s).setFinishBlockOnFlush(true) + } + + override def compressionInputStream(s: InputStream): InputStream = new LZFInputStream(s) +} + + +/** + * Snappy implementation of [[spark.io.CompressionCodec]]. + * Block size can be configured by spark.io.compression.snappy.block.size. + */ +class SnappyCompressionCodec extends CompressionCodec { + + override def compressionOutputStream(s: OutputStream): OutputStream = { + val blockSize = System.getProperty("spark.io.snappy.block.size", "32768").toInt + new SnappyOutputStream(s, blockSize) + } + + override def compressionInputStream(s: InputStream): InputStream = new SnappyInputStream(s) +} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 4228c902f8..9ed4c01218 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -27,11 +27,10 @@ import akka.dispatch.{Await, Future} import akka.util.Duration import akka.util.duration._ -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import spark.{Logging, SparkEnv, SparkException, Utils} +import spark.io.CompressionCodec import spark.network._ import spark.serializer.Serializer import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} @@ -158,7 +157,12 @@ private[spark] class BlockManager( val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks) initialize() - var compressionCodec: CompressionCodec = null + // The compression codec to use. Note that the "lazy" val is necessary because we want to delay + // the initialization of the compression codec until it is first used. The reason is that a Spark + // program could be using a user-defined codec in a third party jar, which is loaded in + // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been + // loaded yet. + private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec() /** * Construct a BlockManager with a memory limit set based on system properties. @@ -921,32 +925,14 @@ private[spark] 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 (compressionCodec == null) { - compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", - "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) - .newInstance().asInstanceOf[CompressionCodec] - } - - if (shouldCompress(blockId)) { - //(new LZFOutputStream(s)).setFinishBlockOnFlush(true) - compressionCodec.compressionOutputStream(s) - } else { - s - } + if (shouldCompress(blockId)) compressionCodec.compressionOutputStream(s) else s } /** * Wrap an input stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: InputStream): InputStream = { - if (compressionCodec == null) { - compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", - "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) - .newInstance().asInstanceOf[CompressionCodec] - } - - if (shouldCompress(blockId)) /*new LZFInputStream(s) */ - compressionCodec.compressionInputStream(s) else s + if (shouldCompress(blockId)) compressionCodec.compressionInputStream(s) else s } def dataSerialize( diff --git a/core/src/main/scala/spark/storage/CompressionCodec.scala b/core/src/main/scala/spark/storage/CompressionCodec.scala deleted file mode 100644 index cd80de33f6..0000000000 --- a/core/src/main/scala/spark/storage/CompressionCodec.scala +++ /dev/null @@ -1,13 +0,0 @@ -package spark.storage - -import java.io.{InputStream, OutputStream} - - -/** - * CompressionCodec allows the customization of the compression codec - */ -trait CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream - - def compressionInputStream(s: InputStream): InputStream -} diff --git a/core/src/main/scala/spark/storage/LZFCompressionCodec.scala b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala deleted file mode 100644 index 3328b949ef..0000000000 --- a/core/src/main/scala/spark/storage/LZFCompressionCodec.scala +++ /dev/null @@ -1,16 +0,0 @@ -package spark.storage - -import java.io.{InputStream, OutputStream} - -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - -/** - * LZF implementation of [[spark.storage.CompressionCodec]] - */ -class LZFCompressionCodec extends CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream = - (new LZFOutputStream(s)).setFinishBlockOnFlush(true) - - def compressionInputStream(s: InputStream): InputStream = - new LZFInputStream(s) -} diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala deleted file mode 100644 index 62b00ef3f6..0000000000 --- a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala +++ /dev/null @@ -1,18 +0,0 @@ -package spark.storage - -import java.io.{InputStream, OutputStream} - -import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} - -/** - * Snappy implementation of [[spark.storage.CompressionCodec]] - * block size can be configured by spark.snappy.block.size - */ -class SnappyCompressionCodec extends CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream = - new SnappyOutputStream(s, - System.getProperty("spark.snappy.block.size", "32768").toInt) - - def compressionInputStream(s: InputStream): InputStream = - new SnappyInputStream(s) -} diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/spark/io/CompressionCodecSuite.scala new file mode 100644 index 0000000000..e69de29bb2 From 5227043f84e29bdca9a3be95d03886b0acea4ac6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:12:16 -0700 Subject: [PATCH 14/75] Documentation update for compression codec. --- docs/configuration.md | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 5c06897cae..0bcd73ca99 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,7 +35,7 @@ for these variables. * `SPARK_JAVA_OPTS`, to add JVM options. This includes any system properties that you'd like to pass with `-D`. * `SPARK_CLASSPATH`, to add elements to Spark's classpath. * `SPARK_LIBRARY_PATH`, to add search directories for native libraries. -* `SPARK_MEM`, to set the amount of memory used per node. This should be in the same format as the +* `SPARK_MEM`, to set the amount of memory used per node. This should be in the same format as the JVM's -Xmx option, e.g. `300m` or `1g`. Note that this option will soon be deprecated in favor of the `spark.executor.memory` system property, so we recommend using that in new code. @@ -77,7 +77,7 @@ there are at least five properties that you will commonly want to control: Class to use for serializing objects that will be sent over the network or need to be cached in serialized form. The default of Java serialization works with any Serializable Java object but is quite slow, so we recommend using spark.KryoSerializer - and configuring Kryo serialization when speed is necessary. Can be any subclass of + and configuring Kryo serialization when speed is necessary. Can be any subclass of spark.Serializer). @@ -86,7 +86,7 @@ there are at least five properties that you will commonly want to control:
@@ -180,6 +180,21 @@ Apart from these, the following properties are also available, and may be useful Can save substantial space at the cost of some extra CPU time. + + + + + + + + + + From 56774b176eb7e7a556bb23d9c524621e156c5633 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:12:33 -0700 Subject: [PATCH 15/75] Added unit test for compression codecs. --- .../spark/io/CompressionCodecSuite.scala | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/spark/io/CompressionCodecSuite.scala index e69de29bb2..1b5daf4c97 100644 --- a/core/src/test/scala/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/spark/io/CompressionCodecSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.io + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import org.scalatest.FunSuite + + +class CompressionCodecSuite extends FunSuite { + + def testCodec(codec: CompressionCodec) { + // Write 1000 integers to the output stream, compressed. + val outputStream = new ByteArrayOutputStream() + val out = codec.compressionOutputStream(outputStream) + for (i <- 1 until 1000) { + out.write(i % 256) + } + out.close() + + // Read the 1000 integers back. + val inputStream = new ByteArrayInputStream(outputStream.toByteArray) + val in = codec.compressionInputStream(inputStream) + for (i <- 1 until 1000) { + assert(in.read() === i % 256) + } + in.close() + } + + test("default compression codec") { + val codec = CompressionCodec.createCodec() + assert(codec.getClass === classOf[SnappyCompressionCodec]) + testCodec(codec) + } + + test("lzf compression codec") { + val codec = CompressionCodec.createCodec(classOf[LZFCompressionCodec].getName) + assert(codec.getClass === classOf[LZFCompressionCodec]) + testCodec(codec) + } + + test("snappy compression codec") { + val codec = CompressionCodec.createCodec(classOf[SnappyCompressionCodec].getName) + assert(codec.getClass === classOf[SnappyCompressionCodec]) + testCodec(codec) + } +} From 3b1ced83fbd72703965aabf7f8dc52417e1cf166 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:24:45 -0700 Subject: [PATCH 16/75] Exclude older version of Snappy in streaming and examples. --- project/SparkBuild.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index be3ef1f148..07d7b76901 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -151,6 +151,7 @@ object SparkBuild extends Build { val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") val excludeAsm = ExclusionRule(organization = "asm") + val excludeSnappy = ExclusionRule(organization = "org.xerial.snappy") def coreSettings = sharedSettings ++ Seq( name := "spark-core", @@ -236,6 +237,7 @@ object SparkBuild extends Build { exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro") + excludeAll(excludeSnappy) ) ) @@ -258,7 +260,7 @@ object SparkBuild extends Build { "Akka Repository" at "http://repo.akka.io/releases/" ), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty) From 311aae76a2177629a6afb75a36563b36e3aa7b66 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:25:42 -0700 Subject: [PATCH 17/75] Added Snappy dependency to Maven build files. --- core/pom.xml | 4 ++++ pom.xml | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index f0c936c86a..ba0071f582 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -48,6 +48,10 @@ com.ning compress-lzf + + org.xerial.snappy + snappy-java + org.ow2.asm asm diff --git a/pom.xml b/pom.xml index 44729bd422..f3bca6c40b 100644 --- a/pom.xml +++ b/pom.xml @@ -182,6 +182,11 @@ compress-lzf 0.8.4 + + org.xerial.snappy + snappy-java + 1.0.5 + org.ow2.asm asm From dae12fef9ed473a9dcdfe00b04497eea21bfb96b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:49:31 -0700 Subject: [PATCH 18/75] Updated the configuration option for Snappy block size to be consistent with the documentation. --- core/src/main/scala/spark/io/CompressionCodec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/spark/io/CompressionCodec.scala index 2ba104a737..b4d8481450 100644 --- a/core/src/main/scala/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/spark/io/CompressionCodec.scala @@ -74,7 +74,7 @@ class LZFCompressionCodec extends CompressionCodec { class SnappyCompressionCodec extends CompressionCodec { override def compressionOutputStream(s: OutputStream): OutputStream = { - val blockSize = System.getProperty("spark.io.snappy.block.size", "32768").toInt + val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } From 98024eadc3150a9a509132117875b8d0b18b1d50 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 18:28:46 -0700 Subject: [PATCH 19/75] Renamed compressionOutputStream and compressionInputStream to compressedOutputStream and compressedInputStream. --- core/src/main/scala/spark/io/CompressionCodec.scala | 12 ++++++------ core/src/main/scala/spark/storage/BlockManager.scala | 4 ++-- .../test/scala/spark/io/CompressionCodecSuite.scala | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/spark/io/CompressionCodec.scala index b4d8481450..0adebecadb 100644 --- a/core/src/main/scala/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/spark/io/CompressionCodec.scala @@ -30,9 +30,9 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} */ trait CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream + def compressedOutputStream(s: OutputStream): OutputStream - def compressionInputStream(s: InputStream): InputStream + def compressedInputStream(s: InputStream): InputStream } @@ -59,11 +59,11 @@ private[spark] object CompressionCodec { */ class LZFCompressionCodec extends CompressionCodec { - override def compressionOutputStream(s: OutputStream): OutputStream = { + override def compressedOutputStream(s: OutputStream): OutputStream = { new LZFOutputStream(s).setFinishBlockOnFlush(true) } - override def compressionInputStream(s: InputStream): InputStream = new LZFInputStream(s) + override def compressedInputStream(s: InputStream): InputStream = new LZFInputStream(s) } @@ -73,10 +73,10 @@ class LZFCompressionCodec extends CompressionCodec { */ class SnappyCompressionCodec extends CompressionCodec { - override def compressionOutputStream(s: OutputStream): OutputStream = { + override def compressedOutputStream(s: OutputStream): OutputStream = { val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } - override def compressionInputStream(s: InputStream): InputStream = new SnappyInputStream(s) + override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s) } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9ed4c01218..3a72474419 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -925,14 +925,14 @@ private[spark] 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)) compressionCodec.compressionOutputStream(s) else s + if (shouldCompress(blockId)) compressionCodec.compressedOutputStream(s) else s } /** * Wrap an input stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: InputStream): InputStream = { - if (shouldCompress(blockId)) compressionCodec.compressionInputStream(s) else s + if (shouldCompress(blockId)) compressionCodec.compressedInputStream(s) else s } def dataSerialize( diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/spark/io/CompressionCodecSuite.scala index 1b5daf4c97..1ba82fe2b9 100644 --- a/core/src/test/scala/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/spark/io/CompressionCodecSuite.scala @@ -27,7 +27,7 @@ class CompressionCodecSuite extends FunSuite { def testCodec(codec: CompressionCodec) { // Write 1000 integers to the output stream, compressed. val outputStream = new ByteArrayOutputStream() - val out = codec.compressionOutputStream(outputStream) + val out = codec.compressedOutputStream(outputStream) for (i <- 1 until 1000) { out.write(i % 256) } @@ -35,7 +35,7 @@ class CompressionCodecSuite extends FunSuite { // Read the 1000 integers back. val inputStream = new ByteArrayInputStream(outputStream.toByteArray) - val in = codec.compressionInputStream(inputStream) + val in = codec.compressedInputStream(inputStream) for (i <- 1 until 1000) { assert(in.read() === i % 256) } From 15fd0d619d24caaf0af6ec0c323c983a8daf8faa Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 30 Jul 2013 18:31:11 -0700 Subject: [PATCH 20/75] Add mllib, bagel to repl dependencies Also don't build an assembly jar for them --- project/SparkBuild.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0fdd291aac..f36c4f2655 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,7 +44,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) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn(bagel) dependsOn(mllib) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) @@ -244,14 +244,14 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq( name := "spark-bagel" - ) ++ assemblySettings ++ extraAssemblySettings + ) def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) - ) ++ assemblySettings ++ extraAssemblySettings + ) def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", From fefb03cbd79cd44447df31147e58f32ba0a466a8 Mon Sep 17 00:00:00 2001 From: cdshines Date: Wed, 31 Jul 2013 13:19:42 +0300 Subject: [PATCH 21/75] Eliminated code duplication, refactored to pattern-matching style Partitioner and PythonPartitioner --- core/src/main/scala/spark/Partitioner.scala | 14 +++-------- core/src/main/scala/spark/Utils.scala | 9 +++++++ .../spark/api/python/PythonPartitioner.scala | 25 ++++--------------- 3 files changed, 17 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 660af70d52..6035bc075e 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -65,17 +65,9 @@ object Partitioner { class HashPartitioner(partitions: Int) extends Partitioner { def numPartitions = partitions - def getPartition(key: Any): Int = { - if (key == null) { - return 0 - } else { - val mod = key.hashCode % partitions - if (mod < 0) { - mod + partitions - } else { - mod // Guard against negative hash codes - } - } + def getPartition(key: Any): Int = key match { + case null => 0 + case _ => Utils.nonNegativeMod(key.hashCode, numPartitions) } override def equals(other: Any): Boolean = other match { diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index e6a96a5ec1..c29933dfe9 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -756,4 +756,13 @@ private object Utils extends Logging { } return buf } + + /* Calculates 'x' modulo 'mod', takes to consideration sign of x, + * i.e. if 'x' is negative, than 'x' % 'mod' is negative too + * so function return (x % mod) + mod in that case. + */ + def nonNegativeMod(x: Int, mod: Int): Int = { + val rawMod = x % mod + rawMod + (if (x < 0 && rawMod != 0) mod else 0) + } } diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index 31a719fbff..ac112b8c2c 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -18,7 +18,7 @@ package spark.api.python import spark.Partitioner - +import spark.Utils import java.util.Arrays /** @@ -35,25 +35,10 @@ private[spark] class PythonPartitioner( val pyPartitionFunctionId: Long) extends Partitioner { - override def getPartition(key: Any): Int = { - if (key == null) { - return 0 - } - else { - val hashCode = { - if (key.isInstanceOf[Array[Byte]]) { - Arrays.hashCode(key.asInstanceOf[Array[Byte]]) - } else { - key.hashCode() - } - } - val mod = hashCode % numPartitions - if (mod < 0) { - mod + numPartitions - } else { - mod // Guard against negative hash codes - } - } + override def getPartition(key: Any): Int = key match { + case null => 0 + case key: Array[Byte] => Utils.nonNegativeMod(Arrays.hashCode(key), numPartitions) + case _ => Utils.nonNegativeMod(key.hashCode(), numPartitions) } override def equals(other: Any): Boolean = other match { From 12553e5c5534df1e5522825088b9a4a73d098854 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 08:50:20 -0700 Subject: [PATCH 22/75] Simplified nonNegativeMod to match previous version --- 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 c29933dfe9..ef598ae41b 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -596,7 +596,7 @@ private object Utils extends Logging { output.toString } - /** + /** * A regular expression to match classes of the "core" Spark API that we want to skip when * finding the call site of a method. */ @@ -763,6 +763,6 @@ private object Utils extends Logging { */ def nonNegativeMod(x: Int, mod: Int): Int = { val rawMod = x % mod - rawMod + (if (x < 0 && rawMod != 0) mod else 0) + rawMod + (if (rawMod < 0) mod else 0) } } From 0c6553714ab7b4f0daee59c0450f5004d69bd662 Mon Sep 17 00:00:00 2001 From: Roman Tkalenko Date: Wed, 31 Jul 2013 19:05:46 +0300 Subject: [PATCH 23/75] Refactored Vector.apply(length, initializer) replacing excessive code with library method (also removed unused variable ```ans``` as minor change) --- core/src/main/scala/spark/util/Vector.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala index ed49386f18..a47cac3b96 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/spark/util/Vector.scala @@ -73,7 +73,6 @@ class Vector(val elements: Array[Double]) extends Serializable { def += (other: Vector): Vector = { if (length != other.length) throw new IllegalArgumentException("Vectors of different length") - var ans = 0.0 var i = 0 while (i < length) { elements(i) += other(i) @@ -117,9 +116,7 @@ object Vector { def apply(elements: Double*) = new Vector(elements.toArray) def apply(length: Int, initializer: Int => Double): Vector = { - val elements = new Array[Double](length) - for (i <- 0 until length) - elements(i) = initializer(i) + val elements: Array[Double] = Array.tabulate(length)(initializer) return new Vector(elements) } From 9a815de4bfaf959fe1399d6550e561a0ec9f28c2 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Thu, 1 Aug 2013 00:36:47 +0800 Subject: [PATCH 24/75] write and read generation in ResultTask --- core/src/main/scala/spark/scheduler/ResultTask.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 361b1e6b91..1ced6f9524 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -118,6 +118,7 @@ private[spark] class ResultTask[T, U]( out.write(bytes) out.writeInt(partition) out.writeInt(outputId) + out.writeLong(generation) out.writeObject(split) } } @@ -132,6 +133,7 @@ private[spark] class ResultTask[T, U]( func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] partition = in.readInt() val outputId = in.readInt() + generation = in.readLong() split = in.readObject().asInstanceOf[Partition] } } From 89da9d94b3f8d977360abeb659286ba7c2724825 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 31 Jul 2013 09:47:53 -0700 Subject: [PATCH 25/75] Add JSON path to master index page --- .../main/scala/spark/deploy/master/ui/IndexPage.scala | 9 ++++++++- .../main/scala/spark/deploy/master/ui/MasterWebUI.scala | 1 + 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index b05197c1b9..82a91cba03 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -25,15 +25,22 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import spark.deploy.{RequestMasterState, DeployWebUI, MasterState} +import spark.deploy.{JsonProtocol, RequestMasterState, DeployWebUI, MasterState} import spark.Utils import spark.ui.UIUtils import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import net.liftweb.json.JsonAST.JValue private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master implicit val timeout = parent.timeout + def renderJson(request: HttpServletRequest): JValue = { + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 30 seconds) + JsonProtocol.writeMasterState(state) + } + /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index dabc2d8dc7..31bdb7854e 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -61,6 +61,7 @@ class MasterWebUI(val master: ActorRef, requestedPort: Int) extends Logging { ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), ("/app/json", (request: HttpServletRequest) => applicationPage.renderJson(request)), ("/app", (request: HttpServletRequest) => applicationPage.render(request)), + ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), ("*", (request: HttpServletRequest) => indexPage.render(request)) ) From c61843a69fd50db66b01e9ef0fb2870baf51d351 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 18:54:35 -0700 Subject: [PATCH 26/75] Changed other LZF uses to use the compression codec interface. --- .../scala/spark/broadcast/HttpBroadcast.scala | 36 ++++++++++--------- .../spark/scheduler/ShuffleMapTask.scala | 15 ++------ .../scala/spark/streaming/Checkpoint.scala | 26 +++++++++----- 3 files changed, 39 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala index c565876950..138a8c21bc 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala @@ -17,21 +17,20 @@ package spark.broadcast -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - -import java.io._ -import java.net._ -import java.util.UUID +import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} +import java.net.URL import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import spark._ +import spark.{HttpServer, Logging, SparkEnv, Utils} +import spark.io.CompressionCodec import spark.storage.StorageLevel -import util.{MetadataCleaner, TimeStampedHashSet} +import spark.util.{MetadataCleaner, TimeStampedHashSet} + private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) -extends Broadcast[T](id) with Logging with Serializable { + extends Broadcast[T](id) with Logging with Serializable { def value = value_ @@ -85,6 +84,7 @@ private object HttpBroadcast extends Logging { private val files = new TimeStampedHashSet[String] private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup) + private lazy val compressionCodec = CompressionCodec.createCodec() def initialize(isDriver: Boolean) { synchronized { @@ -122,10 +122,12 @@ private object HttpBroadcast extends Logging { def write(id: Long, value: Any) { val file = new File(broadcastDir, "broadcast-" + id) - val out: OutputStream = if (compress) { - new LZFOutputStream(new FileOutputStream(file)) // Does its own buffering - } else { - new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + val out: OutputStream = { + if (compress) { + compressionCodec.compressedOutputStream(new FileOutputStream(file)) + } else { + new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + } } val ser = SparkEnv.get.serializer.newInstance() val serOut = ser.serializeStream(out) @@ -136,10 +138,12 @@ private object HttpBroadcast extends Logging { def read[T](id: Long): T = { val url = serverUri + "/broadcast-" + id - var in = if (compress) { - new LZFInputStream(new URL(url).openStream()) // Does its own buffering - } else { - new FastBufferedInputStream(new URL(url).openStream(), bufferSize) + val in = { + if (compress) { + compressionCodec.compressedInputStream(new URL(url).openStream()) + } else { + new FastBufferedInputStream(new URL(url).openStream(), bufferSize) + } } val ser = SparkEnv.get.serializer.newInstance() val serIn = ser.deserializeStream(in) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 1c25605f75..e3bb6d1e60 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -18,16 +18,9 @@ package spark.scheduler import java.io._ -import java.util.{HashMap => JHashMap} import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.collection.JavaConversions._ - -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - -import com.ning.compress.lzf.LZFInputStream -import com.ning.compress.lzf.LZFOutputStream +import scala.collection.mutable.HashMap import spark._ import spark.executor.ShuffleWriteMetrics @@ -109,11 +102,7 @@ private[spark] class ShuffleMapTask( preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) } - var split = if (rdd == null) { - null - } else { - rdd.partitions(partition) - } + var split = if (rdd == null) null else rdd.partitions(partition) override def writeExternal(out: ObjectOutput) { RDDCheckpointData.synchronized { diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 1e4c1e3742..070d930b5e 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -17,16 +17,17 @@ package spark.streaming -import spark.{Logging, Utils} - -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 import java.util.concurrent.RejectedExecutionException +import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration + +import spark.Logging +import spark.io.CompressionCodec + + private[streaming] class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { @@ -49,6 +50,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) } } + /** * Convenience class to speed up the writing of graph checkpoint to file */ @@ -66,6 +68,8 @@ class CheckpointWriter(checkpointDir: String) extends Logging { val maxAttempts = 3 val executor = Executors.newFixedThreadPool(1) + private val compressionCodec = CompressionCodec.createCodec() + // Removed code which validates whether there is only one CheckpointWriter per path 'file' since // I did not notice any errors - reintroduce it ? @@ -103,7 +107,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging { def write(checkpoint: Checkpoint) { val bos = new ByteArrayOutputStream() - val zos = new LZFOutputStream(bos) + val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) oos.writeObject(checkpoint) oos.close() @@ -137,6 +141,8 @@ object CheckpointReader extends Logging { val fs = new Path(path).getFileSystem(new Configuration()) val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk")) + val compressionCodec = CompressionCodec.createCodec() + attempts.foreach(file => { if (fs.exists(file)) { logInfo("Attempting to load checkpoint from file '" + file + "'") @@ -147,7 +153,7 @@ 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 zis = new LZFInputStream(fis) + val zis = compressionCodec.compressedInputStream(fis) val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader) val cp = ois.readObject.asInstanceOf[Checkpoint] ois.close() @@ -170,7 +176,9 @@ object CheckpointReader extends Logging { } private[streaming] -class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) extends ObjectInputStream(inputStream_) { +class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) + extends ObjectInputStream(inputStream_) { + override def resolveClass(desc: ObjectStreamClass): Class[_] = { try { return loader.loadClass(desc.getName()) From 49e6344142b9e87cf220d93c41ae6d35965ea88e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 31 Jul 2013 11:17:59 -0700 Subject: [PATCH 27/75] Removed master URL from job UI, reduced heading size of basic spark pages --- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index e33c80282a..18a3bf534c 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -74,7 +74,6 @@ private[spark] object UIUtils {
  • Application: {sc.appName}
  • -
  • Master: {sc.master}
  • Executors: {sc.getExecutorStorageStatus.size}
@@ -117,9 +116,9 @@ private[spark] object UIUtils {
-

+

{title} -

+
{content} From 9a444cffe74374f0d764d1ed8197423e40529f24 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 11:28:39 -0700 Subject: [PATCH 28/75] Use the Char version of split() instead of the String one for efficiency --- mllib/src/main/scala/spark/mllib/util/MLUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index b5e564df6d..25d9673004 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -38,9 +38,9 @@ object MLUtils { */ def loadLabeledData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { sc.textFile(dir).map { line => - val parts = line.split(",") + val parts = line.split(',') val label = parts(0).toDouble - val features = parts(1).trim().split(" ").map(_.toDouble) + val features = parts(1).trim().split(' ').map(_.toDouble) (label, features) } } From c453967f9aa5946a263fa1364587e7fca88aae45 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 31 Jul 2013 11:57:50 -0700 Subject: [PATCH 29/75] Reduced size of heading --- core/src/main/scala/spark/ui/UIUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 18a3bf534c..cff26d3168 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -116,9 +116,9 @@ private[spark] object UIUtils {
-

+

{title} -

+
{content} From 4692ea48920cb7cebd2cf8cacc43382168801722 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Wed, 31 Jul 2013 12:29:42 -0700 Subject: [PATCH 30/75] Used 'uri.split('/').last' instead of 'new File(uri).getName()'. --- .../spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../scala/spark/scheduler/mesos/MesosSchedulerBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index f75244a9ac..6ebbb5ec9b 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -131,7 +131,7 @@ private[spark] class CoarseMesosSchedulerBackend( } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = new File(uri).getName().split('.')(0) + val basename = uri.split('/').last.split('.').head command.setValue("cd %s*; ./run spark.executor.StandaloneExecutorBackend %s %s %s %d".format( basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 51b780ac72..f6069a5775 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -104,7 +104,7 @@ private[spark] class MesosSchedulerBackend( } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = new File(uri).getName().split('.')(0) + val basename = uri.split('/').last.split('.').head command.setValue("cd %s*; ./spark-executor".format(basename)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } From 529ac81195505f59228fd212a5bad154ab316683 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Wed, 31 Jul 2013 12:49:11 -0700 Subject: [PATCH 31/75] Do not try and use 'scala' in 'run' from within a "release". --- run | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/run b/run index 4cffda4708..0a440627a1 100755 --- a/run +++ b/run @@ -72,7 +72,10 @@ esac # hard to kill the child with stuff like Process.destroy(). However, for # the Spark shell, the wrapper is necessary to properly reset the terminal # when we exit, so we allow it to set a variable to launch with scala. -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then +# We still fall back on java for the shell if this is a "release" created +# from make-distribution.sh since it's possible scala is not installed +# but we have everything we need to run the shell. +if [[ "$SPARK_LAUNCH_WITH_SCALA" == "1" && ! -f "$FWDIR/RELEASE" ]]; then if [ "$SCALA_HOME" ]; then RUNNER="${SCALA_HOME}/bin/scala" else From f607ffb9e1f799d73818f1d37c633007a6b900fb Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 14:31:07 -0700 Subject: [PATCH 32/75] Added data generator for K-means Also made it possible to specify the number of runs in KMeans.main(). --- .../scala/spark/mllib/clustering/KMeans.scala | 7 +- .../mllib/util/KMeansDataGenerator.scala | 80 +++++++++++++++++++ 2 files changed, 84 insertions(+), 3 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala index d875d6de50..a2ed42d7a5 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -315,14 +315,15 @@ object KMeans { } def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: KMeans ") + if (args.length < 4) { + println("Usage: KMeans []") System.exit(1) } val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) + val runs = if (args.length >= 5) args(4).toInt else 1 val sc = new SparkContext(master, "KMeans") val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) - val model = KMeans.train(data, k, iters) + val model = KMeans.train(data, k, iters, runs) val cost = model.computeCost(data) println("Cluster centers:") for (c <- model.clusterCenters) { diff --git a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala new file mode 100644 index 0000000000..8f95cf7479 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.util + +import scala.util.Random + +import spark.{RDD, SparkContext} + +object KMeansDataGenerator { + + /** + * Generate an RDD containing test data for KMeans. This function chooses k cluster centers + * from a d-dimensional Gaussian distribution scaled by factor r, then creates a Gaussian + * cluster with scale 1 around each center. + * + * @param sc SparkContext to use for creating the RDD + * @param numPoints Number of points that will be contained in the RDD + * @param k Number of clusters + * @param d Number of dimensions + * @parak r Scaling factor for the distribution of the initial centers + * @param numPartitions Number of partitions of the generated RDD; default 2 + */ + def generateKMeansRDD( + sc: SparkContext, + numPoints: Int, + k: Int, + d: Int, + r: Double, + numPartitions: Int = 2) + : RDD[Array[Double]] = + { + // First, generate some centers + val rand = new Random(42) + val centers = Array.fill(k)(Array.fill(d)(rand.nextGaussian() * r)) + // Then generate points around each center + sc.parallelize(0 until numPoints, numPartitions).map { idx => + val center = centers(idx % k) + val rand2 = new Random(42 + idx) + Array.tabulate(d)(i => center(i) + rand2.nextGaussian()) + } + } + + def main(args: Array[String]) { + if (args.length < 6) { + println("Usage: KMeansGenerator " + + " []") + System.exit(1) + } + + val sparkMaster = args(0) + val outputPath = args(1) + val numPoints = args(2).toInt + val k = args(3).toInt + val d = args(4).toInt + val r = args(5).toDouble + val parts = if (args.length >= 7) args(6).toInt else 2 + + val sc = new SparkContext(sparkMaster, "KMeansDataGenerator") + val data = generateKMeansRDD(sc, numPoints, k, d, r, parts) + data.map(_.mkString(" ")).saveAsTextFile(outputPath) + + System.exit(0) + } +} + From a6f43a97d15e08f0d2c76b8b790f7e5635f5b2e6 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 31 Jul 2013 14:45:53 -0700 Subject: [PATCH 33/75] SPARK-842. Maven assembly is including examples libs and dependencies --- assembly/pom.xml | 24 ++++++------------------ assembly/src/main/assembly/assembly.xml | 2 +- 2 files changed, 7 insertions(+), 19 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 1382539f24..cc5a4875af 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -62,43 +62,31 @@ org.spark-project spark-core ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version}
org.spark-project spark-bagel ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version} org.spark-project - spark-examples + spark-mllib ${classifier.name} - 0.8.0-SNAPSHOT - - - org.spark-project - spark-examples - javadoc - 0.8.0-SNAPSHOT - - - org.spark-project - spark-examples - sources - 0.8.0-SNAPSHOT + ${project.version} org.spark-project spark-repl ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version} org.spark-project spark-streaming ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version} \ No newline at end of file diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index dd05f35f1f..14485b7181 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -49,7 +49,7 @@ org.spark-project:*:jar - org.spark-project:spark-dist:jar + org.spark-project:spark-assembly:jar From 52dba89261ee6dddafff5c746322980567252843 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 23:08:12 -0700 Subject: [PATCH 34/75] Turn on caching in KMeans.main --- mllib/src/main/scala/spark/mllib/clustering/KMeans.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala index a2ed42d7a5..b402c71ed2 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -322,7 +322,7 @@ object KMeans { val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) val runs = if (args.length >= 5) args(4).toInt else 1 val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) + val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)).cache() val model = KMeans.train(data, k, iters, runs) val cost = model.computeCost(data) println("Cluster centers:") From 3b5a11e76563a270d694bdcf8afac0b0a515a631 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 1 Aug 2013 19:37:15 +0800 Subject: [PATCH 35/75] change function name "setName" to "setProperties" as "setName" is also member of Thread class --- .../src/main/scala/spark/ui/UIWorkloadGenerator.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index c738cc4065..4fbb503e5c 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -32,6 +32,8 @@ private[spark] object UIWorkloadGenerator { val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 500 + + def main(args: Array[String]) { if (args.length < 2) { println("usage: ./run spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") @@ -48,12 +50,13 @@ private[spark] object UIWorkloadGenerator { // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, // but we pass it here anyways since it will be useful once we do. - def setName(s: String) = { + def setProperties(s: String) = { if(schedulingMode == SchedulingMode.FAIR) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool",desc) + sc.addLocalProperties("spark.scheduler.cluster.fair.pool", s) } sc.addLocalProperties("spark.job.annotation", s) } + val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) def nextFloat() = (new Random()).nextFloat() @@ -87,9 +90,9 @@ private[spark] object UIWorkloadGenerator { while (true) { for ((desc, job) <- jobs) { new Thread { - override def run() { + override def run() { try { - setName(desc) + setProperties(desc) job() println("Job funished: " + desc) } catch { From d58502a1562bbfb1bb4e517ebcc8239efd639297 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 1 Aug 2013 23:21:41 +0800 Subject: [PATCH 36/75] fix bug of spark "SubmitStage" listener as unit test error --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 88c1b7baff..89c51a44c9 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -510,6 +510,12 @@ class DAGScheduler( tasks += new ResultTask(stage.id, stage.rdd, job.func, partition, locs, id) } } + // must be run listener before possible NotSerializableException + // should be "StageSubmitted" first and then "JobEnded" + val properties = idToActiveJob(stage.priority).properties + sparkListeners.foreach(_.onStageSubmitted( + SparkListenerStageSubmitted(stage, tasks.size, properties))) + if (tasks.size > 0) { // Preemptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception @@ -523,9 +529,7 @@ class DAGScheduler( running -= stage return } - val properties = idToActiveJob(stage.priority).properties - sparkListeners.foreach(_.onStageSubmitted( - SparkListenerStageSubmitted(stage, tasks.size, properties))) + logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) From ffc034e4fbb6d63825626f555b2089c0389d0075 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 10:39:56 -0700 Subject: [PATCH 37/75] Import cleanup --- .../main/scala/spark/deploy/master/ui/IndexPage.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 82a91cba03..df4c00fc97 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -17,19 +17,20 @@ package spark.deploy.master.ui +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node +import net.liftweb.json.JsonAST.JValue import spark.deploy.{JsonProtocol, RequestMasterState, DeployWebUI, MasterState} import spark.Utils import spark.ui.UIUtils import spark.deploy.master.{ApplicationInfo, WorkerInfo} -import net.liftweb.json.JsonAST.JValue private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master From 9177bea2b4274c6772e85a78e9264486ced01893 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 10:42:50 -0700 Subject: [PATCH 38/75] Removing extra imports --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index a15558c05e..79fdb21024 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -25,10 +25,6 @@ import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.duration._ - import net.liftweb.json.JsonAST.JValue import spark.Utils From 5faac7f4f3678d424ea1e40a109996f3caba2142 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 13:09:20 -0700 Subject: [PATCH 39/75] Minor style fixes --- core/src/main/scala/spark/SparkContext.scala | 2 +- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0d1f9fa8d4..2e2a699708 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -579,7 +579,7 @@ class SparkContext( /** * Return pools for fair scheduler - * TODO(xiajunluan):now, we have not taken nested pools into account + * TODO(xiajunluan): We should take nested pools into account */ def getPools: ArrayBuffer[Schedulable] = { taskScheduler.rootPool.schedulableQueue diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 3832c5d33c..dc3aef8f28 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -41,7 +41,6 @@ private[spark] class JobProgressUI(val sc: SparkContext) { def listener = _listener.get val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) From cfcd77b5da7d8c453b525c1862cba158eddc5170 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 13:31:56 -0700 Subject: [PATCH 40/75] Increasing inter job arrival --- core/src/main/scala/spark/ui/UIWorkloadGenerator.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 4fbb503e5c..3ac35085eb 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -30,9 +30,7 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode */ private[spark] object UIWorkloadGenerator { val NUM_PARTITIONS = 100 - val INTER_JOB_WAIT_MS = 500 - - + val INTER_JOB_WAIT_MS = 5000 def main(args: Array[String]) { if (args.length < 2) { From b10199413a1e9193c605dc17de276c78b578e545 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 13:37:22 -0700 Subject: [PATCH 41/75] Slight refactoring to SparkContext functions --- core/src/main/scala/spark/SparkContext.scala | 13 ++++++++----- core/src/main/scala/spark/ui/jobs/PoolTable.scala | 4 ++-- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 2e2a699708..97e1aaf49e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -581,10 +581,17 @@ class SparkContext( * Return pools for fair scheduler * TODO(xiajunluan): We should take nested pools into account */ - def getPools: ArrayBuffer[Schedulable] = { + def getAllPools: ArrayBuffer[Schedulable] = { taskScheduler.rootPool.schedulableQueue } + /** + * Return the pool associated with the given name, if one exists + */ + def getPoolForName(pool: String): Option[Schedulable] = { + taskScheduler.rootPool.schedulableNameToSchedulable.get(pool) + } + /** * Return current scheduling mode */ @@ -592,10 +599,6 @@ class SparkContext( taskScheduler.schedulingMode } - def getPoolNameToPool: HashMap[String, Schedulable] = { - taskScheduler.rootPool.schedulableNameToSchedulable - } - /** * 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/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index 8788ed8bc1..29061199df 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -38,7 +38,7 @@ private[spark] class FIFOSource() extends PoolSource { */ private[spark] class FairSource(sc: SparkContext) extends PoolSource { def getPools: Seq[Schedulable] = { - sc.getPools.toSeq + sc.getAllPools.toSeq } } @@ -48,7 +48,7 @@ private[spark] class FairSource(sc: SparkContext) extends PoolSource { private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource { def getPools: Seq[Schedulable] = { val pools = HashSet[Schedulable]() - pools += sc.getPoolNameToPool(poolName) + pools += sc.getPoolForName(poolName).get pools.toSeq } } From 87fd321a5a12ccea9d5593a4c43aaadfa44855bd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 14:11:18 -0700 Subject: [PATCH 42/75] Minor refactoring and code cleanup --- .../main/scala/spark/ui/jobs/IndexPage.scala | 27 +++++----- .../scala/spark/ui/jobs/JobProgressUI.scala | 12 ----- .../main/scala/spark/ui/jobs/PoolPage.scala | 7 ++- .../main/scala/spark/ui/jobs/PoolTable.scala | 52 ++----------------- .../main/scala/spark/ui/jobs/StageTable.scala | 51 ++++-------------- 5 files changed, 28 insertions(+), 121 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 4ad787565d..b0d057afa1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -17,16 +17,11 @@ package spark.ui.jobs -import java.util.Date - import javax.servlet.http.HttpServletRequest -import scala.Some import scala.xml.{NodeSeq, Node} -import spark.scheduler.cluster.TaskInfo -import spark.scheduler.Stage -import spark.storage.StorageLevel +import spark.scheduler.cluster.SchedulingMode import spark.ui.Page._ import spark.ui.UIUtils._ import spark.Utils @@ -50,7 +45,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStagesTable = new StageTable(completedStages, parent) val failedStagesTable = new StageTable(failedStages, parent) - val poolTable = new PoolTable(parent.stagePagePoolSource, listener) + val poolTable = new PoolTable(listener.sc.getAllPools, listener) val summary: NodeSeq =
    @@ -79,13 +74,17 @@ private[spark] class IndexPage(parent: JobProgressUI) {
val content = summary ++ -

Pools

++ poolTable.toNodeSeq ++ -

Active Stages : {activeStages.size}

++ - activeStagesTable.toNodeSeq++ -

Completed Stages : {completedStages.size}

++ - completedStagesTable.toNodeSeq++ -

Failed Stages : {failedStages.size}

++ - failedStagesTable.toNodeSeq + {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) { +

Pools

++ poolTable.toNodeSeq + } else { + Seq() + }} ++ +

Active Stages : {activeStages.size}

++ + activeStagesTable.toNodeSeq++ +

Completed Stages : {completedStages.size}

++ + completedStagesTable.toNodeSeq++ +

Failed Stages : {failedStages.size}

++ + failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages", Jobs) } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index dc3aef8f28..c83f102ff3 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -45,20 +45,8 @@ private[spark] class JobProgressUI(val sc: SparkContext) { private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) - var stagePoolInfo: StagePoolInfo = null - var stagePagePoolSource: PoolSource = null - def start() { _listener = Some(new JobProgressListener(sc)) - sc.getSchedulingMode match { - case SchedulingMode.FIFO => - stagePoolInfo = new FIFOStagePoolInfo() - stagePagePoolSource = new FIFOSource() - case SchedulingMode.FAIR => - stagePoolInfo = new FairStagePoolInfo(listener) - stagePagePoolSource = new FairSource(sc) - } - sc.addSparkListener(listener) } diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala index 37d4f8fa6b..ee5a6a6a48 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -17,12 +17,11 @@ private[spark] class PoolPage(parent: JobProgressUI) { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq - - val poolDetailPoolSource = new PoolDetailSource(parent.sc, poolName) - val poolTable = new PoolTable(poolDetailPoolSource, listener) - val activeStagesTable = new StageTable(activeStages, parent) + val pool = listener.sc.getPoolForName(poolName).get + val poolTable = new PoolTable(Seq(pool), listener) + val content =

Pool

++ poolTable.toNodeSeq() ++

Active Stages : {activeStages.size}

++ activeStagesTable.toNodeSeq() diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index 29061199df..9cfe0d68f0 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -1,65 +1,19 @@ package spark.ui.jobs -import java.util.Date - -import javax.servlet.http.HttpServletRequest - -import scala.Some -import scala.xml.{NodeSeq, Node} +import scala.xml.Node import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import spark.SparkContext import spark.scheduler.Stage -import spark.ui.UIUtils._ -import spark.ui.Page._ -import spark.storage.StorageLevel import spark.scheduler.cluster.Schedulable -/* - * Interface for get pools seq showing on Index or pool detail page - */ - -private[spark] trait PoolSource { - def getPools: Seq[Schedulable] -} - -/* - * Pool source for FIFO scheduler algorithm on Index page - */ -private[spark] class FIFOSource() extends PoolSource { - def getPools: Seq[Schedulable] = { - Seq[Schedulable]() - } -} - -/* - * Pool source for Fair scheduler algorithm on Index page - */ -private[spark] class FairSource(sc: SparkContext) extends PoolSource { - def getPools: Seq[Schedulable] = { - sc.getAllPools.toSeq - } -} - -/* - * specific pool info for pool detail page - */ -private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource { - def getPools: Seq[Schedulable] = { - val pools = HashSet[Schedulable]() - pools += sc.getPoolForName(poolName).get - pools.toSeq - } -} - /** Table showing list of pools */ -private[spark] class PoolTable(poolSource: PoolSource, listener: JobProgressListener) { +private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) { var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages def toNodeSeq(): Seq[Node] = { - poolTable(poolRow, poolSource.getPools) + poolTable(poolRow, pools) } // pool tables diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index e18b70f0b9..3257f4e360 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -10,51 +10,20 @@ import scala.xml.{NodeSeq, Node} import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import spark.scheduler.cluster.TaskInfo +import spark.scheduler.cluster.{SchedulingMode, TaskInfo} import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.Utils import spark.storage.StorageLevel -/* - * Interface to get stage's pool name - */ -private[spark] trait StagePoolInfo { - def getStagePoolName(s: Stage): String - - def hasHref: Boolean -} - -/* - * For FIFO scheduler algorithm, just show "N/A" and its link status is false - */ -private[spark] class FIFOStagePoolInfo extends StagePoolInfo { - def getStagePoolName(s: Stage): String = "N/A" - - def hasHref: Boolean = false -} - -/* - * For Fair scheduler algorithm, show its pool name and pool detail link status is true - */ -private[spark] class FairStagePoolInfo(listener: JobProgressListener) extends StagePoolInfo { - def getStagePoolName(s: Stage): String = { - listener.stageToPool(s) - } - - def hasHref: Boolean = true -} - /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable( - val stages: Seq[Stage], - val parent: JobProgressUI) { +private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressUI) { val listener = parent.listener val dateFmt = parent.dateFmt - var stagePoolInfo = parent.stagePoolInfo - + val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR + def toNodeSeq(): Seq[Node] = { stageTable(stageRow, stages) } @@ -64,7 +33,7 @@ private[spark] class StageTable(
(none) If you use Kryo serialization, set this class to register your custom classes with Kryo. - You need to set it to a class that extends + You need to set it to a class that extends spark.KryoRegistrator). See the tuning guide for more details.
spark.io.compression.codecspark.io.SnappyCompressionCodec + The compression codec class to use for various compressions. By default, Spark provides two + codecs: spark.io.LZFCompressionCodec and spark.io.SnappyCompressionCodec. +
spark.io.compression.snappy.block.size32768 + Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec is used. +
spark.reducer.maxMbInFlight 48
- + {if (isFairScheduler) {} else {}} @@ -116,15 +85,13 @@ private[spark] class StageTable( val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions - val poolName = stagePoolInfo.getStagePoolName(s) + val poolName = listener.stageToPool.get(s) - + {if (isFairScheduler) { + } + } {if (isFairScheduler) {} else {}} - + @@ -87,13 +87,17 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val poolName = listener.stageToPool.get(s) + val nameLink = {s.name} + val description = listener.stageToDescription.get(s) + .map(d =>
{d}
{nameLink}
).getOrElse(nameLink) + {if (isFairScheduler) { } } - - + + diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 14bb58731b..66fd59e8bb 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -73,7 +73,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { TaskThreadInfo.threadToStarted(threadIndex) = new CountDownLatch(1) new Thread { if (poolName != null) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) + sc.addLocalProperty("spark.scheduler.cluster.fair.pool", poolName) } override def run() { val ans = nums.map(number => { From abfa9e6f708b46894be1972f46efe542578a30f1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 2 Aug 2013 16:17:32 -0700 Subject: [PATCH 50/75] Increase Kryo buffer size in ALS since some arrays become big --- mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 7281b2fcb9..6ecf0151a1 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -418,6 +418,7 @@ object ALS { System.setProperty("spark.serializer", "spark.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) System.setProperty("spark.kryo.referenceTracking", "false") + System.setProperty("spark.kryoserializer.buffer.mb", "8") System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => From 4ab4df5edbc1bded810a8a3e1dfc7f8ae40a7c30 Mon Sep 17 00:00:00 2001 From: Ginger Smith Date: Fri, 2 Aug 2013 22:22:36 -0700 Subject: [PATCH 51/75] adding matrix factorization data generator --- .../spark/mllib/util/MFDataGenerator.scala | 105 ++++++++++++++++++ 1 file changed, 105 insertions(+) create mode 100644 mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala new file mode 100644 index 0000000000..8637d27cd0 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.recommendation + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + + +object MFDataGenerator{ + + /** + * Generate RDD(s) containing data for Matrix Factorization. This function chooses + * positive labels with probability `probOne` and scales positive examples by `eps`. + * + * @param sc SparkContext to use for creating the RDD. + * @param outputPath Directory to save output. + * @param m Number of rows in data matrix. + * @param n Number of columns in data matrix. + * @param rank Underlying rank of data matrix. + * @param tr_samp_fact Oversampling factor. + * @param noise Boolean value - whether to add gaussian noise to training data. + * @param sigma Standard deviation of added gaussian noise. + * @param test Boolean value - whether to create testing RDD. + * @param te_samp_fact Percentage of training data to use as test data. + */ + + def main(args: Array[String]) { + if (args.length != 10) { + println("Usage: MFGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val m: Int = if (args.length > 2) args(2).toInt else 100 + val n: Int = if (args.length > 3) args(3).toInt else 100 + val rank: Int = if (args.length > 4) args(4).toInt else 10 + val tr_samp_fact: Double = if (args.length > 5) args(5).toDouble else 1.0 + val noise: Boolean = if (args.length > 6) args(6).toBoolean else false + val sigma: Double = if (args.length > 7) args(7).toDouble else 0.1 + val test: Boolean = if (args.length > 8) args(8).toBoolean else false + val te_samp_fact: Double = if (args.length > 9) args(9).toDouble else 0.1 + + val sc = new SparkContext(sparkMaster, "MFDataGenerator") + + val A = DoubleMatrix.randn(m,rank) + val B = DoubleMatrix.randn(rank,n) + val z = 1/(scala.math.sqrt(scala.math.sqrt(rank))) + A.mmuli(z) + B.mmuli(z) + val fullData = A.mmul(B) + + val df = rank*(m+n-rank) + val sampsize = scala.math.min(scala.math.round(tr_samp_fact*df), scala.math.round(.99*m*n)).toInt + val rand = new Random() + val mn = m*n + val shuffled = rand.shuffle(1 to mn toIterable) + + val omega = shuffled.slice(0,sampsize) + val ordered = omega.sortWith(_ < _).toArray + val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) + .map(x => (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + + // optionally add gaussian noise + if(noise){ + trainData.map(x => (x._1,x._2,x._3+rand.nextGaussian*sigma)) + } + + trainData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) + + // optionally generate testing data + if(test){ + val test_sampsize = scala.math + .min(scala.math.round(sampsize*te_samp_fact),scala.math.round(mn-sampsize)) + .toInt + val test_omega = shuffled.slice(sampsize,sampsize+test_sampsize) + val test_ordered = test_omega.sortWith(_ < _).toArray + val testData: RDD[(Int, Int, Double)] = sc.parallelize(test_ordered) + .map(x=> (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) + } + + sc.stop() + } +} \ No newline at end of file From a6826373016f415169316024373c604089d2be31 Mon Sep 17 00:00:00 2001 From: stayhf Date: Fri, 2 Aug 2013 02:02:31 +0000 Subject: [PATCH 52/75] Simple PageRank algorithm implementation in Java for SPARK-760 --- .../java/spark/examples/JavaPageRank.java | 129 ++++++++++++++++++ 1 file changed, 129 insertions(+) create mode 100644 examples/src/main/java/spark/examples/JavaPageRank.java diff --git a/examples/src/main/java/spark/examples/JavaPageRank.java b/examples/src/main/java/spark/examples/JavaPageRank.java new file mode 100644 index 0000000000..c491c86a6f --- /dev/null +++ b/examples/src/main/java/spark/examples/JavaPageRank.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.examples; + +import scala.Tuple2; +import spark.api.java.JavaPairRDD; +import spark.api.java.JavaRDD; +import spark.api.java.JavaSparkContext; +import spark.api.java.function.FlatMapFunction; +import spark.api.java.function.Function; +import spark.api.java.function.PairFunction; + +import java.util.List; +import java.util.ArrayList; + +/** + * Computes the PageRank of URLs from an input file. Input file should + * be in format of: + * URL neighbor URL + * URL neighbor URL + * URL neighbor URL + * ... + * where URL and their neighbors are separated by space(s). + */ +public class JavaPageRank { + private static double sum(List numbers) { + double out = 0.0; + for (double number : numbers) { + out += number; + } + return out; + } + + public static double round(double value, int places) { + if (places < 0) throw new IllegalArgumentException(); + + long factor = (long) Math.pow(10, places); + value = value * factor; + long tmp = Math.round(value); + return (double) tmp / factor; + } + + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: JavaPageRank "); + System.exit(1); + } + + JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + + // Loads in input file. It should be in format of: + // URL neighbor URL + // URL neighbor URL + // URL neighbor URL + // ... + JavaRDD lines = ctx.textFile(args[1], 1).cache(); + + // Loads all URLs from input file and initialize their neighbors. + JavaPairRDD> links = lines.map(new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s.split("\\s+")[0], s.split("\\s+")[1]); + } + }).distinct().groupByKey(); + + // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. + JavaPairRDD ranks = lines.map(new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s.split("\\s+")[1], 1.0); + } + }).distinct(); + + + // Calculates and updates URL ranks continuously using PageRank algorithm. + for (int current = 0; current < Integer.parseInt(args[2]); current++) { + // Calculates URL contributions to the rank of other URLs. + JavaPairRDD contribs = links.join(ranks).values() + .flatMap(new FlatMapFunction, Double>, Tuple2>() { + @Override + public Iterable> call(Tuple2, Double> s) { + List> results = new ArrayList>(); + for (String n : s._1) { + results.add(new Tuple2(n, s._2 / s._1.size())); + } + + return results; + } + }).map(new PairFunction, String, Double>() { + @Override + public Tuple2 call(Tuple2 s) { + return s; + } + }); + + // Re-calculates URL ranks based on neighbor contributions. + ranks = contribs.groupByKey().mapValues(new Function, Double>() { + @Override + public Double call(List cs) throws Exception { + return 0.15 + sum(cs) * 0.85; + } + }); + } + + // Collects all URL ranks and dump them to console. + List> output = ranks.collect(); + for (Tuple2 tuple : output) { + System.out.println(tuple._1 + " has rank: " + round((Double)(tuple._2), 2) + "."); + } + + System.exit(0); + } +} From 98fd62605daaf4e74dfffef1e7bc57ac131a61bb Mon Sep 17 00:00:00 2001 From: stayhf Date: Mon, 5 Aug 2013 00:30:28 +0000 Subject: [PATCH 53/75] Updated code with reviewer's suggestions --- .../java/spark/examples/JavaPageRank.java | 86 +++++++++---------- 1 file changed, 43 insertions(+), 43 deletions(-) diff --git a/examples/src/main/java/spark/examples/JavaPageRank.java b/examples/src/main/java/spark/examples/JavaPageRank.java index c491c86a6f..b57797a9fb 100644 --- a/examples/src/main/java/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/spark/examples/JavaPageRank.java @@ -61,47 +61,47 @@ public class JavaPageRank { System.exit(1); } - JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); - // Loads in input file. It should be in format of: - // URL neighbor URL - // URL neighbor URL - // URL neighbor URL - // ... - JavaRDD lines = ctx.textFile(args[1], 1).cache(); + // Loads in input file. It should be in format of: + // URL neighbor URL + // URL neighbor URL + // URL neighbor URL + // ... + JavaRDD lines = ctx.textFile(args[1], 1); - // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.map(new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s.split("\\s+")[0], s.split("\\s+")[1]); - } - }).distinct().groupByKey(); + // Loads all URLs from input file and initialize their neighbors. + JavaPairRDD> links = lines.map(new PairFunction() { + @Override + public Tuple2 call(String s) { + String[] parts = s.split("\\s+"); + return new Tuple2(parts[0], parts[1]); + } + }).distinct().groupByKey().cache(); - // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - JavaPairRDD ranks = lines.map(new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s.split("\\s+")[1], 1.0); - } - }).distinct(); + // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. + JavaPairRDD ranks = links.mapValues(new Function, Double>() { + @Override + public Double call(List rs) throws Exception { + return 1.0; + } + }); + // Calculates and updates URL ranks continuously using PageRank algorithm. + for (int current = 0; current < Integer.parseInt(args[2]); current++) { + // Calculates URL contributions to the rank of other URLs. + JavaPairRDD contribs = links.join(ranks).values() + .flatMap(new FlatMapFunction, Double>, Tuple2>() { + @Override + public Iterable> call(Tuple2, Double> s) { + List> results = new ArrayList>(); + for (String n : s._1) { + results.add(new Tuple2(n, s._2 / s._1.size())); + } - // Calculates and updates URL ranks continuously using PageRank algorithm. - for (int current = 0; current < Integer.parseInt(args[2]); current++) { - // Calculates URL contributions to the rank of other URLs. - JavaPairRDD contribs = links.join(ranks).values() - .flatMap(new FlatMapFunction, Double>, Tuple2>() { - @Override - public Iterable> call(Tuple2, Double> s) { - List> results = new ArrayList>(); - for (String n : s._1) { - results.add(new Tuple2(n, s._2 / s._1.size())); + return results; } - - return results; - } }).map(new PairFunction, String, Double>() { @Override public Tuple2 call(Tuple2 s) { @@ -115,15 +115,15 @@ public class JavaPageRank { public Double call(List cs) throws Exception { return 0.15 + sum(cs) * 0.85; } - }); - } + }); + } - // Collects all URL ranks and dump them to console. - List> output = ranks.collect(); - for (Tuple2 tuple : output) { - System.out.println(tuple._1 + " has rank: " + round((Double)(tuple._2), 2) + "."); - } + // Collects all URL ranks and dump them to console. + List> output = ranks.collect(); + for (Tuple2 tuple : output) { + System.out.println(tuple._1 + " has rank: " + round((Double)(tuple._2), 2) + "."); + } - System.exit(0); + System.exit(0); } } From 39e4fda76f704f61924b10618a6f22bd93d8000f Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Mon, 5 Aug 2013 02:09:54 -0700 Subject: [PATCH 54/75] [HOTFIX] Extend thread safety for SparkEnv.get() A ThreadLocal SparkEnv.env is facing various situations leading to NullPointerExceptions, where SparkEnv.env set in one thread is not gettable in another thread, but often assumed to be available. See, e.g., https://groups.google.com/forum/#!topic/spark-developers/GLx8yunSj0A This hotfixes SparkEnv.env to return either (a) the ThreadLocal value if non-null, or (b) the previously set value in any thread. This approach preserves SparkEnv.set() thread safety needed by RDD.compute() and possibly other places. A refactoring that parameterizes SparkEnv should be addressed subsequently. On branch adatao-global-SparkEnv Changes to be committed: modified: core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkEnv.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 4a1d341f5d..9ac59918cc 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -97,13 +97,26 @@ class SparkEnv ( object SparkEnv extends Logging { private val env = new ThreadLocal[SparkEnv] + private var lastSetSparkEnv : SparkEnv = _ def set(e: SparkEnv) { + lastSetSparkEnv = e env.set(e) } + /** + * Returns the ThreadLocal SparkEnv, if non-null. Else returns the SparkEnv + * previously set in any thread. + */ def get: SparkEnv = { - env.get() + Option(env.get()).getOrElse(lastSetSparkEnv) + } + + /** + * Returns the ThreadLocal SparkEnv. + */ + def getThreadLocal : SparkEnv = { + env.get() } def createFromSystemProperties( From d93d5fcaacc22f837c861bddbb54928b8121bdfb Mon Sep 17 00:00:00 2001 From: Bill Zhao Date: Mon, 5 Aug 2013 10:09:03 -0700 Subject: [PATCH 55/75] SPARK-850: Give better error message on the console --- .../main/scala/spark/scheduler/cluster/ClusterScheduler.scala | 3 ++- project/SparkBuild.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index d92dd3de8b..96568e0d27 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -206,7 +206,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def run() { if (!hasLaunchedTask) { logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered") + "check your cluster UI to ensure that workers are registered " + + "and have sufficient memory") } else { this.cancel() } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c822f49e78..4a9f3de8c9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -156,7 +156,7 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/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 8c8947e2b66169dddb828b801ffaa43cc400b8a5 Mon Sep 17 00:00:00 2001 From: Ginger Smith Date: Mon, 5 Aug 2013 11:22:18 -0700 Subject: [PATCH 56/75] fixing formatting --- .../spark/mllib/util/MFDataGenerator.scala | 39 +++++++++++-------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala index 8637d27cd0..1d2b5c89f0 100644 --- a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala @@ -24,25 +24,32 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils +/** +* Generate RDD(s) containing data for Matrix Factorization. +* +* This method samples training entries according to the oversampling factor +* 'tr_samp_fact', which is a multiplicative factor of the number of +* degrees of freedom of the matrix: rank*(m+n-rank). +* +* It optionally samples entries for a testing matrix using +* 'te_samp_fact', the percentage of the number of training entries +* to use for testing. +* +* This method takes the following inputs: +* sparkMaster (String) The master URL. +* outputPath (String) Directory to save output. +* m (Int) Number of rows in data matrix. +* n (Int) Number of columns in data matrix. +* rank (Int) Underlying rank of data matrix. +* tr_samp_fact (Double) Oversampling factor. +* noise (Boolean) Whether to add gaussian noise to training data. +* sigma (Double) Standard deviation of added gaussian noise. +* test (Boolean) Whether to create testing RDD. +* te_samp_fact (Double) Percentage of training data to use as test data. +*/ object MFDataGenerator{ - /** - * Generate RDD(s) containing data for Matrix Factorization. This function chooses - * positive labels with probability `probOne` and scales positive examples by `eps`. - * - * @param sc SparkContext to use for creating the RDD. - * @param outputPath Directory to save output. - * @param m Number of rows in data matrix. - * @param n Number of columns in data matrix. - * @param rank Underlying rank of data matrix. - * @param tr_samp_fact Oversampling factor. - * @param noise Boolean value - whether to add gaussian noise to training data. - * @param sigma Standard deviation of added gaussian noise. - * @param test Boolean value - whether to create testing RDD. - * @param te_samp_fact Percentage of training data to use as test data. - */ - def main(args: Array[String]) { if (args.length != 10) { println("Usage: MFGenerator " + From 87134b3648c3e01aa2f1fff28751f3625c6434a2 Mon Sep 17 00:00:00 2001 From: Bill Zhao Date: Mon, 5 Aug 2013 11:55:35 -0700 Subject: [PATCH 57/75] SPARK-850: give better console message --- .../main/scala/spark/scheduler/cluster/ClusterScheduler.scala | 3 ++- 1 file changed, 2 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 d92dd3de8b..96568e0d27 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -206,7 +206,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def run() { if (!hasLaunchedTask) { logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered") + "check your cluster UI to ensure that workers are registered " + + "and have sufficient memory") } else { this.cancel() } From 33b9b155defe915d69de25d00da5d2dbdbb71eb5 Mon Sep 17 00:00:00 2001 From: Bill Zhao Date: Mon, 5 Aug 2013 12:02:36 -0700 Subject: [PATCH 58/75] JBoss repository working now --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4a9f3de8c9..c822f49e78 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -156,7 +156,7 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - // "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/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 95025afdecdf5b21ab3d73c6786be9f3c6d579ca Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 5 Aug 2013 13:04:56 -0700 Subject: [PATCH 59/75] Made most small fixes for SPARK-849 except for table sort, task progress overlay --- core/src/main/scala/spark/ui/UIUtils.scala | 17 ++++++------- .../main/scala/spark/ui/jobs/IndexPage.scala | 24 ++++++++++++------- .../main/scala/spark/ui/jobs/StageTable.scala | 9 +------ .../main/scala/spark/ui/storage/RDDPage.scala | 5 ++-- 4 files changed, 27 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index cff26d3168..226fe49aaf 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -28,14 +28,14 @@ private[spark] object UIUtils { /** Returns a spark page with correctly formatted headers */ def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) : Seq[Node] = { - val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • - } val jobs = page match { case Jobs =>
  • Jobs
  • case _ =>
  • Jobs
  • } + val storage = page match { + case Storage =>
  • Storage
  • + case _ =>
  • Storage
  • + } val environment = page match { case Environment =>
  • Environment
  • case _ =>
  • Environment
  • @@ -65,17 +65,14 @@ private[spark] object UIUtils { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 2da2155e09..cb0119bcce 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -69,11 +69,19 @@ private[spark] class IndexPage(parent: JobProgressUI) { {Utils.memoryBytesToString(listener.totalShuffleWrite)} } -
  • Active Stages Number: {activeStages.size}
  • -
  • Completed Stages Number: {completedStages.size}
  • -
  • Failed Stages Number: {failedStages.size}
  • -
  • Scheduling Mode: {parent.sc.getSchedulingMode}
  • - +
  • + Active Stages Number: + {activeStages.size} +
  • +
  • + Completed Stages Number: + {completedStages.size} +
  • +
  • + Failed Stages Number: + {failedStages.size} +
  • +
  • Scheduling Mode: {parent.sc.getSchedulingMode}
  • @@ -83,11 +91,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { } else { Seq() }} ++ -

    Active Stages : {activeStages.size}

    ++ +

    Active Stages : {activeStages.size}

    ++ activeStagesTable.toNodeSeq++ -

    Completed Stages : {completedStages.size}

    ++ +

    Completed Stages : {completedStages.size}

    ++ completedStagesTable.toNodeSeq++ -

    Failed Stages : {failedStages.size}

    ++ +

    Failed Stages : {failedStages.size}

    ++ failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages", Jobs) diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 3257f4e360..b1b8743a97 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -37,10 +37,9 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
    - + - {rows.map(r => makeRow(r))} @@ -105,12 +104,6 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU - } } diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 003be54ad8..cd828778a6 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -83,18 +83,19 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
    +

    Data Distribution Summary

    {workerTable}

    -

    RDD Summary

    +

    Partitions

    {blockTable}
    ; - headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Jobs) + headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Storage) } def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = { From e8bec8365f6598affd0335eae82b093acf4671da Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Wed, 31 Jul 2013 15:52:05 -0700 Subject: [PATCH 60/75] Only reduce the number of cores once when removing an executor --- .../main/scala/spark/deploy/master/ApplicationInfo.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 79687df614..6dd2f06126 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -52,8 +52,10 @@ private[spark] class ApplicationInfo( } def removeExecutor(exec: ExecutorInfo) { - executors -= exec.id - coresGranted -= exec.cores + if (executors.contains(exec.id)) { + executors -= exec.id + coresGranted -= exec.cores + } } def coresLeft: Int = desc.maxCores - coresGranted From cdd1af562ef3fb480f2e98300e3d463657c09681 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 1 Aug 2013 15:14:39 -0700 Subject: [PATCH 61/75] Timeout zombie workers --- .../scala/spark/deploy/master/Master.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 0aed4b9802..b50613f866 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -39,7 +39,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt - + val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt + var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] val idToWorker = new HashMap[String, WorkerInfo] @@ -337,12 +338,17 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act /** 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 + val currentTime = System.currentTimeMillis() + val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT).toArray for (worker <- toRemove) { - logWarning("Removing %s because we got no heartbeat in %d seconds".format( - worker.id, WORKER_TIMEOUT)) - removeWorker(worker) + if (worker.state != WorkerState.DEAD) { + logWarning("Removing %s because we got no heartbeat in %d seconds".format( + worker.id, WORKER_TIMEOUT)) + removeWorker(worker) + } else { + if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) + workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it + } } } } From 37ccf9301a427095d5bf6a35447f5871e12cdb35 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 1 Aug 2013 15:33:42 -0700 Subject: [PATCH 62/75] milliseconds -> seconds in timeOutDeadWorkers logging --- 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 b50613f866..bd7924c71d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -343,7 +343,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act for (worker <- toRemove) { if (worker.state != WorkerState.DEAD) { logWarning("Removing %s because we got no heartbeat in %d seconds".format( - worker.id, WORKER_TIMEOUT)) + worker.id, WORKER_TIMEOUT/1000)) removeWorker(worker) } else { if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) From 35d8f5ee521dc1873548a978d27b10644076a0c0 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sun, 4 Aug 2013 09:13:41 -0700 Subject: [PATCH 63/75] Moved handling of timed out workers within the Master actor --- core/src/main/scala/spark/deploy/DeployMessage.scala | 6 ++++++ core/src/main/scala/spark/deploy/master/Master.scala | 6 +++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 7c37a16615..31861f3ac2 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -109,6 +109,7 @@ private[deploy] object DeployMessages { } // WorkerWebUI to Worker + case object RequestWorkerState // Worker to WorkerWebUI @@ -120,4 +121,9 @@ private[deploy] object DeployMessages { Utils.checkHost(host, "Required hostname") assert (port > 0) } + + // Actor System to Master + + case object CheckForWorkerTimeOut + } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index bd7924c71d..4a4d9908a0 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,7 +80,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() - context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) + context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() @@ -176,6 +176,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case RequestMasterState => { sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray) } + + case CheckForWorkerTimeOut => { + timeOutDeadWorkers() + } } /** From b1bbbe699cd23b937e6fa5c90d5ea0fcc9801681 Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Mon, 5 Aug 2013 17:22:27 -0700 Subject: [PATCH 64/75] [HOTFIX] Mark lastSetSparkEnv @volatile in case it gets HotSpot-cached On branch adatao-global-SparkEnv Changes to be committed: modified: core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkEnv.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 9ac59918cc..0adbf1d96e 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -97,7 +97,7 @@ class SparkEnv ( object SparkEnv extends Logging { private val env = new ThreadLocal[SparkEnv] - private var lastSetSparkEnv : SparkEnv = _ + @volatile private var lastSetSparkEnv : SparkEnv = _ def set(e: SparkEnv) { lastSetSparkEnv = e From bf7033f3ebf9315ccf9aba09a6e702c3a671fd8d Mon Sep 17 00:00:00 2001 From: Ginger Smith Date: Mon, 5 Aug 2013 21:26:24 -0700 Subject: [PATCH 65/75] fixing formatting, style, and input --- .../spark/mllib/util/MFDataGenerator.scala | 73 ++++++++++--------- 1 file changed, 37 insertions(+), 36 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala index 1d2b5c89f0..88992cde0c 100644 --- a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala @@ -28,32 +28,32 @@ import spark.mllib.util.MLUtils * Generate RDD(s) containing data for Matrix Factorization. * * This method samples training entries according to the oversampling factor -* 'tr_samp_fact', which is a multiplicative factor of the number of +* 'trainSampFact', which is a multiplicative factor of the number of * degrees of freedom of the matrix: rank*(m+n-rank). * * It optionally samples entries for a testing matrix using -* 'te_samp_fact', the percentage of the number of training entries +* 'testSampFact', the percentage of the number of training entries * to use for testing. * * This method takes the following inputs: -* sparkMaster (String) The master URL. -* outputPath (String) Directory to save output. -* m (Int) Number of rows in data matrix. -* n (Int) Number of columns in data matrix. -* rank (Int) Underlying rank of data matrix. -* tr_samp_fact (Double) Oversampling factor. -* noise (Boolean) Whether to add gaussian noise to training data. -* sigma (Double) Standard deviation of added gaussian noise. -* test (Boolean) Whether to create testing RDD. -* te_samp_fact (Double) Percentage of training data to use as test data. +* sparkMaster (String) The master URL. +* outputPath (String) Directory to save output. +* m (Int) Number of rows in data matrix. +* n (Int) Number of columns in data matrix. +* rank (Int) Underlying rank of data matrix. +* trainSampFact (Double) Oversampling factor. +* noise (Boolean) Whether to add gaussian noise to training data. +* sigma (Double) Standard deviation of added gaussian noise. +* test (Boolean) Whether to create testing RDD. +* testSampFact (Double) Percentage of training data to use as test data. */ object MFDataGenerator{ def main(args: Array[String]) { - if (args.length != 10) { - println("Usage: MFGenerator " + - " ") + if (args.length < 2) { + println("Usage: MFDataGenerator " + + " [m] [n] [rank] [trainSampFact] [noise] [sigma] [test] [testSampFact]") System.exit(1) } @@ -62,51 +62,52 @@ object MFDataGenerator{ val m: Int = if (args.length > 2) args(2).toInt else 100 val n: Int = if (args.length > 3) args(3).toInt else 100 val rank: Int = if (args.length > 4) args(4).toInt else 10 - val tr_samp_fact: Double = if (args.length > 5) args(5).toDouble else 1.0 + val trainSampFact: Double = if (args.length > 5) args(5).toDouble else 1.0 val noise: Boolean = if (args.length > 6) args(6).toBoolean else false val sigma: Double = if (args.length > 7) args(7).toDouble else 0.1 val test: Boolean = if (args.length > 8) args(8).toBoolean else false - val te_samp_fact: Double = if (args.length > 9) args(9).toDouble else 0.1 + val testSampFact: Double = if (args.length > 9) args(9).toDouble else 0.1 val sc = new SparkContext(sparkMaster, "MFDataGenerator") - val A = DoubleMatrix.randn(m,rank) - val B = DoubleMatrix.randn(rank,n) - val z = 1/(scala.math.sqrt(scala.math.sqrt(rank))) + val A = DoubleMatrix.randn(m, rank) + val B = DoubleMatrix.randn(rank, n) + val z = 1 / (scala.math.sqrt(scala.math.sqrt(rank))) A.mmuli(z) B.mmuli(z) val fullData = A.mmul(B) - val df = rank*(m+n-rank) - val sampsize = scala.math.min(scala.math.round(tr_samp_fact*df), scala.math.round(.99*m*n)).toInt + val df = rank * (m + n - rank) + val sampSize = scala.math.min(scala.math.round(trainSampFact * df), + scala.math.round(.99 * m * n)).toInt val rand = new Random() - val mn = m*n + val mn = m * n val shuffled = rand.shuffle(1 to mn toIterable) - val omega = shuffled.slice(0,sampsize) + val omega = shuffled.slice(0, sampSize) val ordered = omega.sortWith(_ < _).toArray val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) - .map(x => (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) // optionally add gaussian noise - if(noise){ - trainData.map(x => (x._1,x._2,x._3+rand.nextGaussian*sigma)) + if (noise) { + trainData.map(x => (x._1, x._2, x._3 + rand.nextGaussian * sigma)) } trainData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) // optionally generate testing data - if(test){ - val test_sampsize = scala.math - .min(scala.math.round(sampsize*te_samp_fact),scala.math.round(mn-sampsize)) - .toInt - val test_omega = shuffled.slice(sampsize,sampsize+test_sampsize) - val test_ordered = test_omega.sortWith(_ < _).toArray - val testData: RDD[(Int, Int, Double)] = sc.parallelize(test_ordered) - .map(x=> (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + if (test) { + val testSampSize = scala.math + .min(scala.math.round(sampSize * testSampFact),scala.math.round(mn - sampSize)).toInt + val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) + val testOrdered = testOmega.sortWith(_ < _).toArray + val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) + .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) } - sc.stop() + sc.stop() + } } \ No newline at end of file From a30866438bf71c83575a3e83887bd4bf33c7cdde Mon Sep 17 00:00:00 2001 From: Alexander Pivovarov Date: Mon, 5 Aug 2013 21:48:43 -0700 Subject: [PATCH 66/75] SHARK-94 Log the files computed by HadoopRDD and NewHadoopRDD --- core/src/main/scala/spark/rdd/HadoopRDD.scala | 1 + core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index d0fdeb741e..fd00d59c77 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -88,6 +88,7 @@ class HadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext) = new NextIterator[(K, V)] { val split = theSplit.asInstanceOf[HadoopPartition] + logInfo("Input split: " + split.inputSplit) var reader: RecordReader[K, V] = null val conf = confBroadcast.value.value diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 17fe805fd4..0b71608169 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -73,6 +73,7 @@ class NewHadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopPartition] + logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) From 5ed5b73026748223c44a83ac4e189e2cd5ab05aa Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 6 Aug 2013 13:59:53 -0700 Subject: [PATCH 67/75] Sorts first column of env tables --- core/src/main/scala/spark/ui/env/EnvironmentUI.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 5ae7935ed4..5845a41552 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -46,7 +46,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { ("Scala Home", Properties.scalaHome) ) def jvmRow(kv: (String, String)) = - def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation) + def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation.sorted) val properties = System.getProperties.iterator.toSeq val classPathProperty = properties @@ -58,8 +58,8 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val propertyHeaders = Seq("Name", "Value") def propertyRow(kv: (String, String)) = - val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties) - val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties) + val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties.sorted) + val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties.sorted) val classPathEntries = classPathProperty._2 .split(System.getProperty("path.separator", ":")) @@ -71,7 +71,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val classPathHeaders = Seq("Resource", "Source") def classPathRow(data: (String, String)) = - val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath) + val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath.sorted) val content = From 326a7a82e0fc7155d10af696ccca2e385134c144 Mon Sep 17 00:00:00 2001 From: stayhf Date: Tue, 6 Aug 2013 21:03:24 +0000 Subject: [PATCH 68/75] changes as reviewer requested --- .../src/main/java/spark/examples/JavaPageRank.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/examples/src/main/java/spark/examples/JavaPageRank.java b/examples/src/main/java/spark/examples/JavaPageRank.java index b57797a9fb..8e58bf2433 100644 --- a/examples/src/main/java/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/spark/examples/JavaPageRank.java @@ -46,15 +46,6 @@ public class JavaPageRank { return out; } - public static double round(double value, int places) { - if (places < 0) throw new IllegalArgumentException(); - - long factor = (long) Math.pow(10, places); - value = value * factor; - long tmp = Math.round(value); - return (double) tmp / factor; - } - public static void main(String[] args) throws Exception { if (args.length < 3) { System.err.println("Usage: JavaPageRank "); @@ -121,7 +112,7 @@ public class JavaPageRank { // Collects all URL ranks and dump them to console. List> output = ranks.collect(); for (Tuple2 tuple : output) { - System.out.println(tuple._1 + " has rank: " + round((Double)(tuple._2), 2) + "."); + System.out.println(tuple._1 + " has rank: " + tuple._2 + "."); } System.exit(0); From 254a93073067c4febeb4903a06a1b7e26e17c67e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 6 Aug 2013 14:18:38 -0700 Subject: [PATCH 69/75] Reverse sorts StageTable by submitted time --- core/src/main/scala/spark/ui/jobs/StageTable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index b1b8743a97..658b25138d 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -25,7 +25,7 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR def toNodeSeq(): Seq[Node] = { - stageTable(stageRow, stages) + stageTable(stageRow, stages.sortBy(_.submissionTime).reverse) } /** Special table which merges two header cells. */ From 882baee48927e7c9ee4c3536282c79f553500dbc Mon Sep 17 00:00:00 2001 From: stayhf Date: Tue, 6 Aug 2013 21:34:39 +0000 Subject: [PATCH 70/75] Got rid of unnecessary map function --- examples/src/main/java/spark/examples/JavaPageRank.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/examples/src/main/java/spark/examples/JavaPageRank.java b/examples/src/main/java/spark/examples/JavaPageRank.java index 8e58bf2433..9d90ef9174 100644 --- a/examples/src/main/java/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/spark/examples/JavaPageRank.java @@ -23,6 +23,7 @@ import spark.api.java.JavaRDD; import spark.api.java.JavaSparkContext; import spark.api.java.function.FlatMapFunction; import spark.api.java.function.Function; +import spark.api.java.function.PairFlatMapFunction; import spark.api.java.function.PairFunction; import java.util.List; @@ -83,7 +84,7 @@ public class JavaPageRank { for (int current = 0; current < Integer.parseInt(args[2]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMap(new FlatMapFunction, Double>, Tuple2>() { + .flatMap(new PairFlatMapFunction, Double>, String, Double>() { @Override public Iterable> call(Tuple2, Double> s) { List> results = new ArrayList>(); @@ -93,11 +94,6 @@ public class JavaPageRank { return results; } - }).map(new PairFunction, String, Double>() { - @Override - public Tuple2 call(Tuple2 s) { - return s; - } }); // Re-calculates URL ranks based on neighbor contributions. From 099528b6c47e29b5635e878fbb43814be8f3d05c Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 6 Aug 2013 14:52:12 -0700 Subject: [PATCH 71/75] Pre-sorts stage/env tables, changes text/link of stage summaries --- .../scala/spark/ui/env/EnvironmentUI.scala | 16 +++++----- .../main/scala/spark/ui/jobs/IndexPage.scala | 30 +++++++++---------- .../main/scala/spark/ui/jobs/PoolPage.scala | 2 +- .../main/scala/spark/ui/jobs/StageTable.scala | 2 +- 4 files changed, 25 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 5845a41552..e98302611e 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -44,22 +44,22 @@ private[spark] class EnvironmentUI(sc: SparkContext) { ("Java Home", Properties.javaHome), ("Scala Version", Properties.versionString), ("Scala Home", Properties.scalaHome) - ) + ).sorted def jvmRow(kv: (String, String)) = - def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation.sorted) + def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation) val properties = System.getProperties.iterator.toSeq val classPathProperty = properties .filter{case (k, v) => k.contains("java.class.path")} .headOption .getOrElse("", "") - val sparkProperties = properties.filter(_._1.startsWith("spark")) - val otherProperties = properties.diff(sparkProperties :+ classPathProperty) + val sparkProperties = properties.filter(_._1.startsWith("spark")).sorted + val otherProperties = properties.diff(sparkProperties :+ classPathProperty).sorted val propertyHeaders = Seq("Name", "Value") def propertyRow(kv: (String, String)) = - val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties.sorted) - val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties.sorted) + val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties) + val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties) val classPathEntries = classPathProperty._2 .split(System.getProperty("path.separator", ":")) @@ -67,11 +67,11 @@ private[spark] class EnvironmentUI(sc: SparkContext) { .map(e => (e, "System Classpath")) val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")} - val classPath = addedJars ++ addedFiles ++ classPathEntries + val classPath = (addedJars ++ addedFiles ++ classPathEntries).sorted val classPathHeaders = Seq("Resource", "Source") def classPathRow(data: (String, String)) = - val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath.sorted) + val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath) val content = diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index cb0119bcce..b611b0fe85 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -41,9 +41,9 @@ private[spark] class IndexPage(parent: JobProgressUI) { activeTime += t.timeRunning(now) } - val activeStagesTable = new StageTable(activeStages, parent) - val completedStagesTable = new StageTable(completedStages, parent) - val failedStagesTable = new StageTable(failedStages, parent) + val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) + val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) val poolTable = new PoolTable(listener.sc.getAllPools, listener) val summary: NodeSeq = @@ -69,18 +69,18 @@ private[spark] class IndexPage(parent: JobProgressUI) { {Utils.memoryBytesToString(listener.totalShuffleWrite)} } -
  • - Active Stages Number: - {activeStages.size} -
  • -
  • - Completed Stages Number: - {completedStages.size} -
  • -
  • - Failed Stages Number: - {failedStages.size} -
  • +
  • + Active Stages: + {activeStages.size} +
  • +
  • + Completed Stages: + {completedStages.size} +
  • +
  • + Failed Stages: + {failedStages.size} +
  • Scheduling Mode: {parent.sc.getSchedulingMode}
  • diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala index ee5a6a6a48..647c6d2ae3 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -17,7 +17,7 @@ private[spark] class PoolPage(parent: JobProgressUI) { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq - val activeStagesTable = new StageTable(activeStages, parent) + val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) val pool = listener.sc.getPoolForName(poolName).get val poolTable = new PoolTable(Seq(pool), listener) diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 658b25138d..b1b8743a97 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -25,7 +25,7 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR def toNodeSeq(): Seq[Node] = { - stageTable(stageRow, stages.sortBy(_.submissionTime).reverse) + stageTable(stageRow, stages) } /** Special table which merges two header cells. */ From ca1903ea63e71706456c9bc19b22393c9d02c896 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 6 Aug 2013 15:45:42 -0700 Subject: [PATCH 72/75] Overlays progress text on top of bar --- .../main/scala/spark/ui/jobs/StageTable.scala | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index b1b8743a97..40514b9f40 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -37,7 +37,7 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
    - + @@ -54,11 +54,14 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU } } - def makeProgressBar(started: Int, completed: Int, total: Int): Seq[Node] = { + def makeProgressBar(started: Int, completed: Int, failed: String, total: Int): Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) -
    +
    + + {completed}/{total} {failed} +
    @@ -82,6 +85,10 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val startedTasks = listener.stageToTasksActive.getOrElse(s.id, HashSet[TaskInfo]()).size val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) + val failedTasks = listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => "" + } val totalTasks = s.numPartitions val poolName = listener.stageToPool.get(s) @@ -95,12 +102,8 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
    - - From 8bc497fa10044e732d2912755b1f110d1682d740 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 6 Aug 2013 16:33:05 -0700 Subject: [PATCH 73/75] Lightened color of progress bars --- .../main/resources/spark/ui/static/webui.css | 25 +++++++++++++++++++ .../main/scala/spark/ui/jobs/StageTable.scala | 4 +-- 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index f7537bb766..d918790187 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -47,3 +47,28 @@ padding-top: 7px; padding-left: 4px; } + +.progress-completed .bar, +.progress .bar-completed { + background-color: #7ccde5; + background-image: -moz-linear-gradient(top, #7ccde5, #75c3d9); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#7ccde5), to(#75c3d9)); + background-image: -webkit-linear-gradient(top, #7ccde5, #75c3d9); + background-image: -o-linear-gradient(top, #7ccde5, #75c3d9); + background-image: linear-gradient(to bottom, #7ccde5, #75c3d9); + background-repeat: repeat-x; + filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff7ccde5', endColorstr='#ff75c3d9', GradientType=0); +} + +.progress-running .bar, +.progress .bar-running { + background-color: #cdecf5; + background-image: -moz-linear-gradient(top, #cdecf5, #c2e4ee); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#cdecf5), to(#c2e4ee)); + background-image: -webkit-linear-gradient(top, #cdecf5, #c2e4ee); + background-image: -o-linear-gradient(top, #cdecf5, #c2e4ee); + background-image: linear-gradient(to bottom, #cdecf5, #c2e4ee); + background-repeat: repeat-x; + filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffcdecf5', endColorstr='#ffc2e4ee', GradientType=0); +} + diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 40514b9f40..666437b556 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -62,8 +62,8 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU {completed}/{total} {failed} -
    -
    +
    +
    } From afc2c80fdb17606a5be328cecfd9179f7cb82f4f Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 7 Aug 2013 00:44:43 +0100 Subject: [PATCH 74/75] Update spark-standalone.md --- docs/spark-standalone.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 3986c0c79d..7463844a4e 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -43,7 +43,7 @@ Finally, the following configuration options can be passed to the master and wor
    - + From 908032e79b954a6a9b57ce270ddf0a4d1ba2c8f4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 6 Aug 2013 16:52:21 -0700 Subject: [PATCH 75/75] Used saturated colors for progress bars --- .../main/resources/spark/ui/static/webui.css | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index d918790187..33fc51155f 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -50,25 +50,25 @@ .progress-completed .bar, .progress .bar-completed { - background-color: #7ccde5; - background-image: -moz-linear-gradient(top, #7ccde5, #75c3d9); - background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#7ccde5), to(#75c3d9)); - background-image: -webkit-linear-gradient(top, #7ccde5, #75c3d9); - background-image: -o-linear-gradient(top, #7ccde5, #75c3d9); - background-image: linear-gradient(to bottom, #7ccde5, #75c3d9); + background-color: #b3def9; + background-image: -moz-linear-gradient(top, #addfff, #badcf2); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#addfff), to(#badcf2)); + background-image: -webkit-linear-gradient(top, #addfff, #badcf2); + background-image: -o-linear-gradient(top, #addfff, #badcf2); + background-image: linear-gradient(to bottom, #addfff, #badcf2); background-repeat: repeat-x; - filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff7ccde5', endColorstr='#ff75c3d9', GradientType=0); + filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffaddfff', endColorstr='#ffbadcf2', GradientType=0); } .progress-running .bar, .progress .bar-running { - background-color: #cdecf5; - background-image: -moz-linear-gradient(top, #cdecf5, #c2e4ee); - background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#cdecf5), to(#c2e4ee)); - background-image: -webkit-linear-gradient(top, #cdecf5, #c2e4ee); - background-image: -o-linear-gradient(top, #cdecf5, #c2e4ee); - background-image: linear-gradient(to bottom, #cdecf5, #c2e4ee); + background-color: #c2ebfa; + background-image: -moz-linear-gradient(top, #bdedff, #c7e8f5); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#bdedff), to(#c7e8f5)); + background-image: -webkit-linear-gradient(top, #bdedff, #c7e8f5); + background-image: -o-linear-gradient(top, #bdedff, #c7e8f5); + background-image: linear-gradient(to bottom, #bdedff, #c7e8f5); background-repeat: repeat-x; - filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffcdecf5', endColorstr='#ffc2e4ee', GradientType=0); + filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffbdedff', endColorstr='#ffc7e8f5', GradientType=0); }
    Stage IdPool NamePool NameOrigin Submitted Duration
    {s.id}{if (stagePoolInfo.hasHref) { - {poolName} - } else { - {poolName} - }}{poolName.get}{s.name} {submissionTime} {getElapsedTime(s.submissionTime, From 37bc64a20535215bd4638a928431ebd0e45f3a1a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 11:57:44 -0700 Subject: [PATCH 43/75] Adding application-level metrics. This adds metrics for applications in the deploy Master. --- .../spark/deploy/master/ApplicationInfo.scala | 1 + .../deploy/master/ApplicationSource.scala | 24 +++++++++++++++++++ .../scala/spark/deploy/master/Master.scala | 22 +++++++++++++---- .../scala/spark/metrics/MetricsSystem.scala | 10 +++++++- 4 files changed, 51 insertions(+), 6 deletions(-) create mode 100644 core/src/main/scala/spark/deploy/master/ApplicationSource.scala diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 15ff919738..79687df614 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -34,6 +34,7 @@ private[spark] class ApplicationInfo( var executors = new mutable.HashMap[Int, ExecutorInfo] var coresGranted = 0 var endTime = -1L + val appSource = new ApplicationSource(this) private var nextExecutorId = 0 diff --git a/core/src/main/scala/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/spark/deploy/master/ApplicationSource.scala new file mode 100644 index 0000000000..4df2b6bfdd --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ApplicationSource.scala @@ -0,0 +1,24 @@ +package spark.deploy.master + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +class ApplicationSource(val application: ApplicationInfo) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "%s.%s.%s".format("application", application.desc.name, + System.currentTimeMillis()) + + metricRegistry.register(MetricRegistry.name("status"), new Gauge[String] { + override def getValue: String = application.state.toString + }) + + metricRegistry.register(MetricRegistry.name("runtime_ms"), new Gauge[Long] { + override def getValue: Long = application.duration + }) + + metricRegistry.register(MetricRegistry.name("cores", "number"), new Gauge[Int] { + override def getValue: Int = application.coresGranted + }) + +} diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 202d5bcdb7..36c22d0873 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -38,6 +38,7 @@ import spark.util.AkkaUtils 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 + val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retained_applications", "1000").toInt var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -59,7 +60,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val metricsSystem = MetricsSystem.createMetricsSystem("master") + val masterMetricsSystem = MetricsSystem.createMetricsSystem("master") + val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications") val masterSource = new MasterSource(this) val masterPublicAddress = { @@ -79,13 +81,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - metricsSystem.registerSource(masterSource) - metricsSystem.start() + masterMetricsSystem.registerSource(masterSource) + masterMetricsSystem.start() + applicationMetricsSystem.start() } override def postStop() { webUi.stop() - metricsSystem.stop() + masterMetricsSystem.stop() + applicationMetricsSystem.stop() } override def receive = { @@ -275,6 +279,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val now = System.currentTimeMillis() val date = new Date(now) val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl) + applicationMetricsSystem.registerSource(app.appSource) apps += app idToApp(app.id) = app actorToApp(driver) = app @@ -300,7 +305,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act idToApp -= app.id actorToApp -= app.driver addressToApp -= app.driver.path.address - completedApps += app // Remember it in our history + if (completedApps.size >= RETAINED_APPLICATIONS) { + val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) + completedApps.take(toRemove).foreach( a => { + applicationMetricsSystem.removeSource(a.appSource) + }) + completedApps.trimStart(toRemove) + } + completedApps += app // Remember it in our history waitingApps -= app for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index fabddfb947..5c98f892e1 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -17,7 +17,7 @@ package spark.metrics -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} +import com.codahale.metrics._ import java.util.Properties import java.util.concurrent.TimeUnit @@ -93,6 +93,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } } + def removeSource(source: Source) { + sources -= source + println("Removing source: " + source.sourceName) + registry.removeMatching(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name.startsWith(source.sourceName) + }) + } + def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) From 12d9c82c9b2a5040a2d1e5ab23ab44b41cbe1807 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 13:17:23 -0700 Subject: [PATCH 44/75] Small style fix --- core/src/main/scala/spark/metrics/MetricsSystem.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 5c98f892e1..1dacafa135 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -17,7 +17,7 @@ package spark.metrics -import com.codahale.metrics._ +import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit @@ -95,7 +95,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def removeSource(source: Source) { sources -= source - println("Removing source: " + source.sourceName) registry.removeMatching(new MetricFilter { def matches(name: String, metric: Metric): Boolean = name.startsWith(source.sourceName) }) From f1d2ad550edb83319fe3e187dad2151a8d0aeae1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 21:40:56 -0700 Subject: [PATCH 45/75] under_scores --> camelCase for config options --- 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 36c22d0873..0aed4b9802 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -38,7 +38,7 @@ import spark.util.AkkaUtils 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 - val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retained_applications", "1000").toInt + val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] From d3c37ff120f42f4291a5970fb842af79e0f4a866 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 22:09:22 -0700 Subject: [PATCH 46/75] Improving documentation in config file example --- conf/metrics.properties.template | 67 +++++++++++++++----------------- 1 file changed, 32 insertions(+), 35 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 0486ca4c79..63a5a2093e 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,48 +1,45 @@ -# syntax: [instance].[sink|source].[name].[options] +# syntax: [instance].sink|source.[name].[options]=[value] -# "instance" specify "who" (the role) use metrics system. In spark there are -# several roles like master, worker, executor, driver, these roles will -# create metrics system for monitoring. So instance represents these roles. -# Currently in Spark, several instances have already implemented: master, -# worker, executor, driver. +# This file configures Spark's internal metrics system. The metrics system is +# divided into instances which correspond to internal components. +# Each instance can be configured to report its metrics to one or more sinks. +# Accepted values for [instance] are "master", "worker", "executor", "driver", +# and "applications". A wild card "*" can be used as an instance name, in +# which case all instances will inherit the supplied property. # -# [instance] field can be "master", "worker", "executor", "driver", which means -# only the specified instance has this property. -# a wild card "*" can be used to represent instance name, which means all the -# instances will have this property. +# Within an instance, a "source" specifies a particular set of grouped metrics. +# there are two kinds of sources: +# 1. Spark internal sources, like MasterSource, WorkerSource, etc, which will +# collect a Spark component's internal state. Each instance is paired with a +# Spark source that is added automatically. +# 2. Common sources, like JvmSource, which will collect low level state. +# These can be added through configuration options and are then loaded +# using reflection. # -# "source" specify "where" (source) to collect metrics data. In metrics system, -# there exists two kinds of source: -# 1. Spark internal source, like MasterSource, WorkerSource, etc, which will -# collect Spark component's internal state, these sources are related to -# instance and will be added after specific metrics system is created. -# 2. Common source, like JvmSource, which will collect low level state, is -# configured by configuration and loaded through reflection. +# A "sink" specifies where metrics are delivered to. Each instance can be +# assigned one or more sinks. # -# "sink" specify "where" (destination) to output metrics data to. Several sinks -# can be coexisted and flush metrics to all these sinks. +# The sink|source field specifies whether the property relates to a sink or +# source. # -# [sink|source] field specify this property is source related or sink, this -# field can only be source or sink. +# The [name] field specifies the name of source or sink. # -# [name] field specify the name of source or sink, this is custom defined. -# -# [options] field is the specific property of this source or sink, this source -# or sink is responsible for parsing this property. +# The [options] field is the specific property of this source or sink. The +# source or sink is responsible for parsing this property. # # Notes: -# 1. Sinks should be added through configuration, like console sink, class -# full name should be specified by class property. -# 2. Some sinks can specify polling period, like console sink, which is 10 seconds, -# it should be attention minimal polling period is 1 seconds, any period -# below than 1s is illegal. -# 3. Wild card property can be overlapped by specific instance property, for -# example, *.sink.console.period can be overlapped by master.sink.console.period. +# 1. To add a new sink, set the "class" option to a fully qualified class +# name (see examples below). +# 2. Some sinks involve a polling period. The minimum allowed polling period +# is 1 second. +# 3. Wild card properties can be overridden by more specific properties. +# For example, master.sink.console.period takes precedence over +# *.sink.console.period. # 4. A metrics specific configuration # "spark.metrics.conf=${SPARK_HOME}/conf/metrics.properties" should be -# added to Java property using -Dspark.metrics.conf=xxx if you want to -# customize metrics system, or you can put it in ${SPARK_HOME}/conf, -# metrics system will search and load it automatically. +# added to Java properties using -Dspark.metrics.conf=xxx if you want to +# customize metrics system. You can also put the file in ${SPARK_HOME}/conf +# and it will be loaded automatically. # Enable JmxSink for all instances by class name #*.sink.jmx.class=spark.metrics.sink.JmxSink From e466a55a6b803a5295e6dcc106a4abef917e7058 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 1 Aug 2013 15:45:21 -0700 Subject: [PATCH 47/75] Revert Mesos version to 0.9 since the 0.12 artifact has target Java 7 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bb96ad4ae3..c822f49e78 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -178,7 +178,7 @@ object SparkBuild extends Build { "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", - "org.apache.mesos" % "mesos" % "0.12.0-incubating", + "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", From b3ae5b25d5e20c422049dc0cada397e51f52cb32 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 2 Aug 2013 13:25:14 -0700 Subject: [PATCH 48/75] Shows time the app has been running --- core/src/main/scala/spark/SparkContext.scala | 2 ++ core/src/main/scala/spark/ui/jobs/IndexPage.scala | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0d1f9fa8d4..ca5ea30d22 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -128,6 +128,8 @@ class SparkContext( private[spark] val ui = new SparkUI(this) ui.bind() + val startTime = System.currentTimeMillis() + // Add each JAR given through the constructor if (jars != null) { jars.foreach { addJar(_) } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 4ad787565d..12166f10c2 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -54,7 +54,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { val summary: NodeSeq =
      -
    • +
    • + Duration: + {parent.formatDuration(now - listener.sc.startTime)} +
    • +
    • CPU time: {parent.formatDuration(listener.totalTime + activeTime)}
    • From 5b3784a79c4e6069ace17c5dddc0ad0046909c8b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 2 Aug 2013 15:46:09 -0700 Subject: [PATCH 49/75] Show user-defined job name in UI --- .../main/resources/spark/ui/static/webui.css | 4 ++++ core/src/main/scala/spark/SparkContext.scala | 11 +++++++++-- .../main/scala/spark/scheduler/JobLogger.scala | 5 +++-- .../scala/spark/ui/UIWorkloadGenerator.scala | 6 ++---- .../spark/ui/jobs/JobProgressListener.scala | 18 +++++++++++++----- .../main/scala/spark/ui/jobs/StageTable.scala | 10 +++++++--- .../spark/scheduler/LocalSchedulerSuite.scala | 2 +- 7 files changed, 39 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index f7537bb766..8b9f4ee938 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -47,3 +47,7 @@ padding-top: 7px; padding-left: 4px; } + +.table td { + vertical-align: middle !important; +} diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 97e1aaf49e..039f5522b7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -265,12 +265,18 @@ class SparkContext( localProperties.value = new Properties() } - def addLocalProperties(key: String, value: String) { + def addLocalProperty(key: String, value: String) { if(localProperties.value == null) { localProperties.value = new Properties() } localProperties.value.setProperty(key,value) } + + /** Set a human readable description of the current job. */ + def setDescription(value: String) { + addLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value) + } + // Post init taskScheduler.postStartHook() @@ -841,6 +847,7 @@ class SparkContext( * various Spark features. */ object SparkContext { + val SPARK_JOB_DESCRIPTION = "spark.job.description" implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 @@ -958,7 +965,6 @@ object SparkContext { } } - /** * A class encapsulating how to convert some type T to Writable. It stores both the Writable class * corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion. @@ -970,3 +976,4 @@ private[spark] class WritableConverter[T]( val writableClass: ClassManifest[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable + diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index c7e8f8a9a1..ad2efcec63 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -26,6 +26,7 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{Map, HashMap, ListBuffer} import scala.io.Source import spark._ +import spark.SparkContext import spark.executor.TaskMetrics import spark.scheduler.cluster.TaskInfo @@ -317,8 +318,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { protected def recordJobProperties(jobID: Int, properties: Properties) { if(properties != null) { - val annotation = properties.getProperty("spark.job.annotation", "") - jobLogInfo(jobID, annotation, false) + val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") + jobLogInfo(jobID, description, false) } } diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 3ac35085eb..97ea644021 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -46,13 +46,11 @@ private[spark] object UIWorkloadGenerator { } val sc = new SparkContext(master, appName) - // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, - // but we pass it here anyways since it will be useful once we do. def setProperties(s: String) = { if(schedulingMode == SchedulingMode.FAIR) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool", s) + sc.addLocalProperty("spark.scheduler.cluster.fair.pool", s) } - sc.addLocalProperties("spark.job.annotation", s) + sc.addLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, s) } val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala index 200e13cf99..f22c4e39e3 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -15,6 +15,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val DEFAULT_POOL_NAME = "default" val stageToPool = new HashMap[Stage, String]() + val stageToDescription = new HashMap[Stage, String]() val poolToActiveStages = new HashMap[String, HashSet[Stage]]() val activeStages = HashSet[Stage]() @@ -57,6 +58,8 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList stageToTasksActive.remove(s.id) stageToTasksComplete.remove(s.id) stageToTasksFailed.remove(s.id) + stageToPool.remove(s) + if (stageToDescription.contains(s)) {stageToDescription.remove(s)} }) stages.trimEnd(toRemove) } @@ -66,12 +69,17 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { val stage = stageSubmitted.stage activeStages += stage - var poolName = DEFAULT_POOL_NAME - if (stageSubmitted.properties != null) { - poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", - DEFAULT_POOL_NAME) - } + + val poolName = Option(stageSubmitted.properties).map { + p => p.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME) + }.getOrElse(DEFAULT_POOL_NAME) stageToPool(stage) = poolName + + val description = Option(stageSubmitted.properties).flatMap { + p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) + } + description.map(d => stageToDescription(stage) = d) + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) stages += stage } diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 3257f4e360..38fa3bcbcd 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -34,7 +34,7 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
    Stage IdPool NameOriginDescription Submitted Duration Tasks: Complete/Total
    {s.id}{poolName.get}{s.name}{submissionTime}{description}{submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} {makeProgressBar(startedTasks, completedTasks, totalTasks)}Origin Submitted DurationTasks: Complete/TotalTasks: Succeeded/Total Shuffle Read Shuffle WriteStored RDD
    {shuffleRead} {shuffleWrite}{if (s.rdd.getStorageLevel != StorageLevel.NONE) { - - {Option(s.rdd.name).getOrElse(s.rdd.id)} - - }} -
    {kv._1}{kv._2}
    {kv._1}{kv._2}
    {data._1}{data._2}
    {kv._1}{kv._2}
    {kv._1}{kv._2}
    {data._1}{data._2}
    Origin Submitted DurationTasks: Succeeded/TotalTasks: Succeeded/Total Shuffle Read Shuffle Write {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))}{makeProgressBar(startedTasks, completedTasks, totalTasks)}{completedTasks} / {totalTasks} - {listener.stageToTasksFailed.getOrElse(s.id, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => - }} + + {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} {shuffleRead} {shuffleWrite}
    -p PORT, --port PORTIP address or DNS name to listen on (default: 7077 for master, random for worker)Port for service to listen on (default: 7077 for master, random for worker)
    --webui-port PORT