From 56cc7fbcaf04a5aab88296d20da2cfc5b84a7651 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 28 Mar 2014 14:45:46 -0700 Subject: [PATCH 01/26] First cut implementation of Streaming UI. --- .../spark/streaming/StreamingContext.scala | 4 + .../spark/streaming/ui/StreamingUI.scala | 131 ++++++++++++++++++ .../apache/spark/streaming/ui/UIUtils.scala | 72 ++++++++++ 3 files changed, 207 insertions(+) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e198c69470c1f..d45cdac5bef41 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -40,6 +40,7 @@ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receivers._ import org.apache.spark.streaming.scheduler._ import org.apache.hadoop.conf.Configuration +import org.apache.spark.streaming.ui.StreamingUI /** * Main entry point for Spark Streaming functionality. It provides methods used to create @@ -158,6 +159,9 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter + private[streaming] val ui = new StreamingUI(this) + ui.bind() + /** * Return the associated Spark context */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala new file mode 100644 index 0000000000000..e9f8d21faab45 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import scala.collection.mutable.SynchronizedQueue +import scala.xml.Node + +import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.servlet.ServletContextHandler + +import org.apache.spark.Logging +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.scheduler.{BatchInfo, StreamingListener, StreamingListenerBatchCompleted} +import org.apache.spark.ui.{ServerInfo, SparkUI} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.{Distribution, Utils} + +private[spark] class StreamingUIListener() extends StreamingListener { + + private val batchInfos = new SynchronizedQueue[BatchInfo] + private val maxBatchInfos = 100 + + override def onBatchCompleted(batchStarted: StreamingListenerBatchCompleted) { + batchInfos.enqueue(batchStarted.batchInfo) + if (batchInfos.size > maxBatchInfos) batchInfos.dequeue() + } + + def processingDelayDistribution = extractDistribution(_.processingDelay) + + def schedulingDelayDistribution = extractDistribution(_.schedulingDelay) + + def totalDelay = extractDistribution(_.totalDelay) + + def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { + Distribution(batchInfos.flatMap(getMetric(_)).map(_.toDouble)) + } + + def numBatchInfos = batchInfos.size +} + +private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { + + private val sc = ssc.sparkContext + private val conf = sc.conf + private val appName = sc.appName + private val bindHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) + private val port = conf.getInt("spark.streaming.ui.port", StreamingUI.DEFAULT_PORT) + private val securityManager = sc.env.securityManager + private val listener = new StreamingUIListener() + private val handlers: Seq[ServletContextHandler] = { + Seq( + createServletHandler("/", + (request: HttpServletRequest) => render(request), securityManager), + createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static") + ) + } + + private var serverInfo: Option[ServerInfo] = None + + ssc.addStreamingListener(listener) + + def bind() { + try { + serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) + logInfo("Started Spark Streaming Web UI at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Spark JettyUtils", e) + System.exit(1) + } + } + + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + private def render(request: HttpServletRequest): Seq[Node] = { + val batchStatsTable = generateBatchStatsTable() + val content = batchStatsTable + UIUtils.headerStreamingPage(content, "", appName, "Spark Streaming Overview") + } + + private def generateBatchStatsTable(): Seq[Node] = { + def getQuantiles(timeDistributionOption: Option[Distribution]) = { + timeDistributionOption.get.getQuantiles().map { ms => Utils.msDurationToString(ms.toLong) } + } + val numBatches = listener.numBatchInfos + val table = if (numBatches > 0) { + val processingDelayQuantilesRow = + "Processing Times" +: getQuantiles(listener.processingDelayDistribution) + val schedulingDelayQuantilesRow = + "Scheduling Delay:" +: getQuantiles(listener.processingDelayDistribution) + val totalDelayQuantilesRow = + "End-to-end Delay:" +: getQuantiles(listener.totalDelay) + + val headerRow = Seq("Metric", "Min", "25th percentile", + "Median", "75th percentile", "Max") + val dataRows: Seq[Seq[String]] = Seq( + processingDelayQuantilesRow, + schedulingDelayQuantilesRow, + totalDelayQuantilesRow + ) + Some(UIUtils.listingTable(headerRow, dataRows, fixedWidth = true)) + } else { + None + } + + val content = +

Batch Processing Statistics

++ +
{table.getOrElse("No statistics have been generated yet.")}
+ content + } +} + +object StreamingUI { + val DEFAULT_PORT = 6060 +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala new file mode 100644 index 0000000000000..62e95135fa5c5 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala @@ -0,0 +1,72 @@ +package org.apache.spark.streaming.ui + +import scala.xml.Node +import org.apache.spark.ui.Page + +private[spark] object UIUtils { + + import org.apache.spark.ui.UIUtils.prependBaseUri + + def headerStreamingPage( + content: => Seq[Node], + basePath: String, + appName: String, + title: String): Seq[Node] = { + val overview = { +
  • Overview
  • + } + + + + + + + + {appName} - {title} + + + + +
    +
    +
    +

    + {title} +

    +
    +
    + {content} +
    + + + } + + def listingTable[T]( + headers: Seq[String], + makeRow: T => Seq[Node], + rows: Seq[T], + fixedWidth: Boolean = false): Seq[Node] = { + org.apache.spark.ui.UIUtils.listingTable(headers, makeRow, rows, fixedWidth) + } + + def listingTable[T]( + headers: Seq[String], + rows: Seq[Seq[String]], + fixedWidth: Boolean = false + ): Seq[Node] = { + def makeRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} + org.apache.spark.ui.UIUtils.listingTable(headers, makeRow, rows, fixedWidth) + } +} From 93f1c69e067fb02bcbb1dcab93d1dff4905c2e17 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 31 Mar 2014 16:31:48 -0700 Subject: [PATCH 02/26] Added network receiver information to the Streaming UI. --- .../spark/streaming/dstream/DStream.scala | 9 - .../dstream/NetworkInputDStream.scala | 61 +++++-- .../spark/streaming/scheduler/BatchInfo.scala | 1 + .../streaming/scheduler/JobGenerator.scala | 10 +- .../streaming/scheduler/JobScheduler.scala | 11 +- .../spark/streaming/scheduler/JobSet.scala | 7 +- .../scheduler/NetworkInputTracker.scala | 68 ++++--- .../scheduler/StreamingListener.scala | 13 +- .../spark/streaming/ui/StreamingUI.scala | 171 +++++++++++++++--- .../spark/streaming/InputStreamsSuite.scala | 6 +- .../org/apache/spark/streaming/UISuite.scala | 52 ++++++ 11 files changed, 301 insertions(+), 108 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d48b51aa69565..36aae958019ce 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -351,15 +351,6 @@ abstract class DStream[T: ClassTag] ( dependencies.foreach(_.clearMetadata(time)) } - /* Adds metadata to the Stream while it is running. - * This method should be overwritten by sublcasses of InputDStream. - */ - private[streaming] def addMetadata(metadata: Any) { - if (metadata != null) { - logInfo("Dropping Metadata: " + metadata.toString) - } - } - /** * Refresh the list of checkpointed RDDs that will be saved along with checkpoint of * this stream. This is an internal method that should not be called directly. This is diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 0dc6704603f82..8da4309daf4ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,23 +17,23 @@ package org.apache.spark.streaming.dstream -import java.util.concurrent.ArrayBlockingQueue import java.nio.ByteBuffer +import java.util.concurrent.ArrayBlockingQueue -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.Await import scala.concurrent.duration._ import scala.reflect.ClassTag -import akka.actor.{Props, Actor} +import akka.actor.{Actor, Props} import akka.pattern.ask -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import org.apache.spark.streaming._ import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.rdd.{RDD, BlockRDD} +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} -import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.scheduler.{ReceivedBlockInfo, AddBlocks, DeregisterReceiver, RegisterReceiver} +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -48,8 +48,10 @@ import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, Regi abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - // This is an unique identifier that is used to match the network receiver with the - // corresponding network input stream. + /** Keeps all received blocks information */ + private val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] + + /** This is an unique identifier for the network input stream. */ val id = ssc.getNewNetworkStreamId() /** @@ -64,23 +66,45 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte def stop() {} + /** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */ override def compute(validTime: Time): Option[RDD[T]] = { // If this is called for any time before the start time of the context, // then this returns an empty RDD. This may happen when recovering from a // master failure if (validTime >= graph.startTime) { - val blockIds = ssc.scheduler.networkInputTracker.getBlockIds(id, validTime) + val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id) + receivedBlockInfo(validTime) = blockInfo + val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) Some(new BlockRDD[T](ssc.sc, blockIds)) } else { Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) } } + + /** Get information on received blocks. */ + private[streaming] def getReceivedBlockInfo(time: Time) = { + receivedBlockInfo(time) + } + + /** + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This + * implementation overrides the default implementation to clear received + * block information. + */ + private[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) + val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) + receivedBlockInfo --= oldReceivedBlocks.keys + logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + } } private[streaming] sealed trait NetworkReceiverMessage private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage -private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) +private[streaming] case class ReportBlock(blockId: StreamBlockId, numRecords: Long, metadata: Any) extends NetworkReceiverMessage private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage @@ -156,21 +180,20 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging actor ! ReportError(e.toString) } - /** * Pushes a block (as an ArrayBuffer filled with data) into the block manager. */ - def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { + def pushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - actor ! ReportBlock(blockId, metadata) + actor ! ReportBlock(blockId, arrayBuffer.size, metadata) } /** * Pushes a block (as bytes) into the block manager. */ - def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { + def pushBlock(blockId: StreamBlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { env.blockManager.putBytes(blockId, bytes, level) - actor ! ReportBlock(blockId, metadata) + actor ! ReportBlock(blockId, -1 , metadata) } /** A helper actor that communicates with the NetworkInputTracker */ @@ -188,8 +211,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging } override def receive() = { - case ReportBlock(blockId, metadata) => - tracker ! AddBlocks(streamId, Array(blockId), metadata) + case ReportBlock(blockId, numRecords, metadata) => + tracker ! AddBlocks(ReceivedBlockInfo(streamId, blockId, numRecords, metadata)) case ReportError(msg) => tracker ! DeregisterReceiver(streamId, msg) case StopReceiver(msg) => @@ -211,7 +234,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging class BlockGenerator(storageLevel: StorageLevel) extends Serializable with Logging { - case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) + case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 7f3cd2f8eb1fd..9c69a2a4e21f5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -29,6 +29,7 @@ import org.apache.spark.streaming.Time */ case class BatchInfo( batchTime: Time, + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], submissionTime: Long, processingStartTime: Option[Long], processingEndTime: Option[Long] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index c7306248b1950..80888c755c6bc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -147,7 +147,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " + timesToReschedule.mkString(", ")) timesToReschedule.foreach(time => - jobScheduler.runJobs(time, graph.generateJobs(time)) + jobScheduler.submitJobSet(JobSet(time, graph.generateJobs(time))) ) // Restart the timer @@ -159,7 +159,13 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { private def generateJobs(time: Time) { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { - case Success(jobs) => jobScheduler.runJobs(time, jobs) + case Success(jobs) => + val receivedBlockInfo = graph.getNetworkInputStreams.map { stream => + val streamId = stream.id + val receivedBlockInfo = stream.getReceivedBlockInfo(time) + (streamId, receivedBlockInfo) + }.toMap + jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfo)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } eventActor ! DoCheckpoint(time) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index de675d3c7fb94..ae99454cf8b86 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -82,14 +82,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } } - def runJobs(time: Time, jobs: Seq[Job]) { - if (jobs.isEmpty) { - logInfo("No jobs added for time " + time) + def submitJobSet(jobSet: JobSet) { + if (jobSet.jobs.isEmpty) { + logInfo("No jobs added for time " + jobSet.time) } else { - val jobSet = new JobSet(time, jobs) - jobSets.put(time, jobSet) + jobSets.put(jobSet.time, jobSet) jobSet.jobs.foreach(job => executor.execute(new JobHandler(job))) - logInfo("Added jobs for time " + time) + logInfo("Added jobs for time " + jobSet.time) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index fcf303aee6cd7..a69d74362173e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -24,7 +24,11 @@ import org.apache.spark.streaming.Time * belong to the same batch. */ private[streaming] -case class JobSet(time: Time, jobs: Seq[Job]) { +case class JobSet( + time: Time, + jobs: Seq[Job], + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty + ) { private val incompleteJobs = new HashSet[Job]() private val submissionTime = System.currentTimeMillis() // when this jobset was submitted @@ -60,6 +64,7 @@ case class JobSet(time: Time, jobs: Seq[Job]) { def toBatchInfo: BatchInfo = { new BatchInfo( time, + receivedBlockInfo, submissionTime, if (processingStartTime >= 0 ) Some(processingStartTime) else None, if (processingEndTime >= 0 ) Some(processingEndTime) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index e4fa163f2e069..74a7644d1c7ad 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -17,26 +17,33 @@ package org.apache.spark.streaming.scheduler -import org.apache.spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} -import org.apache.spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} -import org.apache.spark.{SparkException, Logging, SparkEnv} -import org.apache.spark.SparkContext._ - -import scala.collection.mutable.HashMap -import scala.collection.mutable.Queue -import scala.concurrent.duration._ +import scala.collection.mutable.{HashMap, SynchronizedQueue, SynchronizedMap} import akka.actor._ -import akka.pattern.ask -import akka.dispatch._ -import org.apache.spark.storage.BlockId -import org.apache.spark.streaming.{Time, StreamingContext} + +import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} import org.apache.spark.util.AkkaUtils +/** Information about block received by the network receiver */ +case class ReceivedBlockInfo( + streamId: Int, + blockId: StreamBlockId, + numRecords: Long, + metadata: Any + ) + +/** + * Messages used by the NetworkReceiver and the NetworkInputTracker to communicate + * with each other. + */ private[streaming] sealed trait NetworkInputTrackerMessage private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage -private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) +private[streaming] case class AddBlocks(receivedBlockInfo: ReceivedBlockInfo) extends NetworkInputTrackerMessage private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage @@ -53,9 +60,10 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverExecutor() val receiverInfo = new HashMap[Int, ActorRef] - val receivedBlockIds = new HashMap[Int, Queue[BlockId]] + val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] + with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] val timeout = AkkaUtils.askTimeout(ssc.conf) - + val listenerBus = ssc.scheduler.listenerBus // actor is created when generator starts. // This not being null means the tracker has been started and not stopped @@ -87,15 +95,14 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } /** Return all the blocks received from a receiver. */ - def getBlockIds(receiverId: Int, time: Time): Array[BlockId] = synchronized { - val queue = receivedBlockIds.synchronized { - receivedBlockIds.getOrElse(receiverId, new Queue[BlockId]()) - } - val result = queue.synchronized { - queue.dequeueAll(x => true) - } - logInfo("Stream " + receiverId + " received " + result.size + " blocks") - result.toArray + def getReceivedBlockInfo(streamId: Int): Array[ReceivedBlockInfo] = { + val receivedBlockInfo = getReceivedBlockInfoQueue(streamId).dequeueAll(x => true) + logInfo("Stream " + streamId + " received " + receivedBlockInfo.size + " blocks") + receivedBlockInfo.toArray + } + + private def getReceivedBlockInfoQueue(streamId: Int) = { + receivedBlockInfo.getOrElseUpdate(streamId, new SynchronizedQueue[ReceivedBlockInfo]) } /** Actor to receive messages from the receivers. */ @@ -110,17 +117,8 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { + sender.path.address) sender ! true } - case AddBlocks(streamId, blockIds, metadata) => { - val tmp = receivedBlockIds.synchronized { - if (!receivedBlockIds.contains(streamId)) { - receivedBlockIds += ((streamId, new Queue[BlockId])) - } - receivedBlockIds(streamId) - } - tmp.synchronized { - tmp ++= blockIds - } - networkInputStreamMap(streamId).addMetadata(metadata) + case AddBlocks(receivedBlockInfo) => { + getReceivedBlockInfoQueue(receivedBlockInfo.streamId) += receivedBlockInfo } case DeregisterReceiver(streamId, msg) => { receiverInfo -= streamId diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 461ea3506477f..0c1edff9c8616 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -23,6 +23,7 @@ import org.apache.spark.util.Distribution /** Base trait for events related to StreamingListener */ sealed trait StreamingListenerEvent +case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent @@ -34,14 +35,14 @@ private[scheduler] case object StreamingListenerShutdown extends StreamingListen * computation. */ trait StreamingListener { - /** - * Called when processing of a batch has completed - */ + + /** Called when a batch of jobs has been submitted for processing. */ + def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { } + + /** Called when processing of a batch of jobs has completed. */ def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } - /** - * Called when processing of a batch has started - */ + /** Called when processing of a batch of jobs has started. */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala index e9f8d21faab45..b574cb103f766 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala @@ -17,40 +17,86 @@ package org.apache.spark.streaming.ui -import scala.collection.mutable.SynchronizedQueue +import scala.collection.mutable.{HashMap, Queue} import scala.xml.Node import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.Logging -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.scheduler.{BatchInfo, StreamingListener, StreamingListenerBatchCompleted} +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.scheduler._ import org.apache.spark.ui.{ServerInfo, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{Distribution, Utils} +import java.util.Locale -private[spark] class StreamingUIListener() extends StreamingListener { +private[spark] class StreamingUIListener(conf: SparkConf) extends StreamingListener { - private val batchInfos = new SynchronizedQueue[BatchInfo] - private val maxBatchInfos = 100 + private val waitingBatchInfos = new HashMap[Time, BatchInfo] + private val runningBatchInfos = new HashMap[Time, BatchInfo] + private val completedaBatchInfos = new Queue[BatchInfo] + private val batchInfoLimit = conf.getInt("spark.steaming.ui.maxBatches", 100) - override def onBatchCompleted(batchStarted: StreamingListenerBatchCompleted) { - batchInfos.enqueue(batchStarted.batchInfo) - if (batchInfos.size > maxBatchInfos) batchInfos.dequeue() + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) = synchronized { + runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo } - def processingDelayDistribution = extractDistribution(_.processingDelay) + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { + runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo + waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { + waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) + runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) + completedaBatchInfos.enqueue(batchCompleted.batchInfo) + if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + } + + def numNetworkReceivers: Int = synchronized { + completedaBatchInfos.headOption.map(_.receivedBlockInfo.size).getOrElse(0) + } + + def waitingBatches: Seq[BatchInfo] = synchronized { + waitingBatchInfos.values.toSeq + } + + def runningBatches: Seq[BatchInfo] = synchronized { + runningBatchInfos.values.toSeq + } - def schedulingDelayDistribution = extractDistribution(_.schedulingDelay) + def completedBatches: Seq[BatchInfo] = synchronized { + completedaBatchInfos.toSeq + } - def totalDelay = extractDistribution(_.totalDelay) + def processingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.processingDelay) + } - def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { - Distribution(batchInfos.flatMap(getMetric(_)).map(_.toDouble)) + def schedulingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.schedulingDelay) } - def numBatchInfos = batchInfos.size + def totalDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.totalDelay) + } + + def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { + val allBatcheInfos = waitingBatchInfos.values.toSeq ++ + runningBatchInfos.values.toSeq ++ completedaBatchInfos + val latestBatchInfos = allBatcheInfos.sortBy(_.batchTime)(Time.ordering).reverse.take(batchInfoLimit) + val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) + (0 until numNetworkReceivers).map { receiverId => + val blockInfoOfParticularReceiver = latestBlockInfos.map(_.get(receiverId).getOrElse(Array.empty)) + val distributionOption = Distribution(blockInfoOfParticularReceiver.map(_.map(_.numRecords).sum.toDouble)) + (receiverId, distributionOption) + }.toMap + } + + private def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { + Distribution(completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) + } } private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { @@ -62,7 +108,7 @@ private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) private val port = conf.getInt("spark.streaming.ui.port", StreamingUI.DEFAULT_PORT) private val securityManager = sc.env.securityManager - private val listener = new StreamingUIListener() + private val listener = new StreamingUIListener(conf) private val handlers: Seq[ServletContextHandler] = { Seq( createServletHandler("/", @@ -89,23 +135,19 @@ private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) private def render(request: HttpServletRequest): Seq[Node] = { - val batchStatsTable = generateBatchStatsTable() - val content = batchStatsTable + val content = generateBatchStatsTable() ++ generateNetworkStatsTable() UIUtils.headerStreamingPage(content, "", appName, "Spark Streaming Overview") } - private def generateBatchStatsTable(): Seq[Node] = { - def getQuantiles(timeDistributionOption: Option[Distribution]) = { - timeDistributionOption.get.getQuantiles().map { ms => Utils.msDurationToString(ms.toLong) } - } - val numBatches = listener.numBatchInfos + private def generateBatchStatsTable(): Seq[Node] = { + val numBatches = listener.completedBatches.size val table = if (numBatches > 0) { val processingDelayQuantilesRow = "Processing Times" +: getQuantiles(listener.processingDelayDistribution) val schedulingDelayQuantilesRow = - "Scheduling Delay:" +: getQuantiles(listener.processingDelayDistribution) + "Scheduling Delay:" +: getQuantiles(listener.schedulingDelayDistribution) val totalDelayQuantilesRow = - "End-to-end Delay:" +: getQuantiles(listener.totalDelay) + "End-to-end Delay:" +: getQuantiles(listener.totalDelayDistribution) val headerRow = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") @@ -119,11 +161,86 @@ private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { None } + val batchCounts = + + + val batchStats = + + val content =

    Batch Processing Statistics

    ++ -
    {table.getOrElse("No statistics have been generated yet.")}
    +
    {batchCounts}
    ++ +
    {batchStats}
    + content } + + private def generateNetworkStatsTable(): Seq[Node] = { + val receivedRecordDistributions = listener.receivedRecordsDistributions + val numNetworkReceivers = receivedRecordDistributions.size + val table = if (receivedRecordDistributions.size > 0) { + val headerRow = Seq("Receiver", "Min", "25th percentile", + "Median", "75th percentile", "Max") + val dataRows = (0 until numNetworkReceivers).map { receiverId => + val receiverName = s"Receiver-$receiverId" + val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => + d.getQuantiles().map(r => numberToString(r.toLong) + " records/batch") + }.getOrElse { + Seq("-", "-", "-", "-", "-") + } + receiverName +: receivedRecordStats + } + Some(UIUtils.listingTable(headerRow, dataRows, fixedWidth = true)) + } else { + None + } + + val content = +

    Network Input Statistics

    ++ +
    {table.getOrElse("No network receivers")}
    + + content + } + + private def getQuantiles(timeDistributionOption: Option[Distribution]) = { + timeDistributionOption.get.getQuantiles().map { ms => Utils.msDurationToString(ms.toLong) } + } + + private def numberToString(records: Double): String = { + val trillion = 1e12 + val billion = 1e9 + val million = 1e6 + val thousand = 1e3 + + val (value, unit) = { + if (records >= 2*trillion) { + (records / trillion, "T") + } else if (records >= 2*billion) { + (records / billion, "B") + } else if (records >= 2*million) { + (records / million, "M") + } else if (records >= 2*thousand) { + (records / thousand, "K") + } else { + (records, "") + } + } + "%.1f%s".formatLocal(Locale.US, value, unit) + } } object StreamingUI { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 74e73ebb342fe..723ea18e91dbf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -238,11 +238,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { /** This is a server to test the network input stream */ -class TestServer() extends Logging { +class TestServer(portToBind: Int = 0) extends Logging { val queue = new ArrayBlockingQueue[String](100) - val serverSocket = new ServerSocket(0) + val serverSocket = new ServerSocket(portToBind) val servingThread = new Thread() { override def run() { @@ -281,7 +281,7 @@ class TestServer() extends Logging { def start() { servingThread.start() } - def send(msg: String) { queue.add(msg) } + def send(msg: String) { queue.put(msg) } def stop() { servingThread.interrupt() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala new file mode 100644 index 0000000000000..204041def7dfc --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -0,0 +1,52 @@ +package org.apache.spark.streaming + +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite} +import org.apache.spark.streaming.dstream.InputDStream +import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import scala.util.Random + +class UISuite extends FunSuite with BeforeAndAfterAll { + + test("Testing") { + runStreaming(1000000) + } + + def runStreaming(duration: Long) { + val ssc = new StreamingContext("local[10]", "test", Seconds(1)) + val servers = (1 to 5).map { i => new TestServer(10000 + i) } + + val inputStream = ssc.union(servers.map(server => ssc.socketTextStream("localhost", server.port))) + inputStream.count.print + + ssc.start() + servers.foreach(_.start()) + val startTime = System.currentTimeMillis() + while (System.currentTimeMillis() - startTime < duration) { + servers.map(_.send(Random.nextString(10) + "\n")) + Thread.sleep(1) + } + ssc.stop() + servers.foreach(_.stop()) + } +} + +class FunctionBasedInputDStream[T: ClassTag]( + ssc_ : StreamingContext, + function: (StreamingContext, Time) => Option[RDD[T]] + ) extends InputDStream[T](ssc_) { + + def start(): Unit = {} + + def stop(): Unit = {} + + def compute(validTime: Time): Option[RDD[T]] = function(ssc, validTime) +} + + + + + + + + From 4d86e985cb7bbc7f4f125e52d72f4e4bd560677e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 1 Apr 2014 11:02:23 -0700 Subject: [PATCH 03/26] Added basic stats to the StreamingUI and refactored the UI to a Page to make it easier to transition to using SparkUI later. --- .../spark/streaming/ui/StreamingUI.scala | 186 +++++++++++++----- .../apache/spark/streaming/ui/UIUtils.scala | 9 +- .../org/apache/spark/streaming/UISuite.scala | 2 +- 3 files changed, 147 insertions(+), 50 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala index b574cb103f766..545c5cb8e3f61 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala @@ -29,14 +29,17 @@ import org.apache.spark.streaming.scheduler._ import org.apache.spark.ui.{ServerInfo, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{Distribution, Utils} -import java.util.Locale +import java.util.{Calendar, Locale} -private[spark] class StreamingUIListener(conf: SparkConf) extends StreamingListener { +private[ui] class StreamingUIListener(ssc: StreamingContext) extends StreamingListener { private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] private val completedaBatchInfos = new Queue[BatchInfo] - private val batchInfoLimit = conf.getInt("spark.steaming.ui.maxBatches", 100) + private val batchInfoLimit = ssc.conf.getInt("spark.steaming.ui.maxBatches", 100) + private var totalBatchesCompleted = 0L + + val batchDuration = ssc.graph.batchDuration.milliseconds override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) = synchronized { runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo @@ -52,6 +55,11 @@ private[spark] class StreamingUIListener(conf: SparkConf) extends StreamingListe runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) completedaBatchInfos.enqueue(batchCompleted.batchInfo) if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + totalBatchesCompleted += 1L + } + + def numTotalBatchesCompleted: Long = synchronized { + totalBatchesCompleted } def numNetworkReceivers: Int = synchronized { @@ -89,7 +97,8 @@ private[spark] class StreamingUIListener(conf: SparkConf) extends StreamingListe val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) (0 until numNetworkReceivers).map { receiverId => val blockInfoOfParticularReceiver = latestBlockInfos.map(_.get(receiverId).getOrElse(Array.empty)) - val distributionOption = Distribution(blockInfoOfParticularReceiver.map(_.map(_.numRecords).sum.toDouble)) + val recordsOfParticularReceiver = blockInfoOfParticularReceiver.map(_.map(_.numRecords).sum.toDouble * 1000 / batchDuration) + val distributionOption = Distribution(recordsOfParticularReceiver) (receiverId, distributionOption) }.toMap } @@ -99,44 +108,42 @@ private[spark] class StreamingUIListener(conf: SparkConf) extends StreamingListe } } -private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { - private val sc = ssc.sparkContext - private val conf = sc.conf - private val appName = sc.appName - private val bindHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) - private val port = conf.getInt("spark.streaming.ui.port", StreamingUI.DEFAULT_PORT) - private val securityManager = sc.env.securityManager - private val listener = new StreamingUIListener(conf) - private val handlers: Seq[ServletContextHandler] = { - Seq( - createServletHandler("/", - (request: HttpServletRequest) => render(request), securityManager), - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static") - ) - } +private[ui] class StreamingPage(parent: StreamingUI) extends Logging { - private var serverInfo: Option[ServerInfo] = None + private val listener = parent.listener + private val calendar = Calendar.getInstance() + private val startTime = calendar.getTime() - ssc.addStreamingListener(listener) - def bind() { - try { - serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) - logInfo("Started Spark Streaming Web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark JettyUtils", e) - System.exit(1) - } - } + def render(request: HttpServletRequest): Seq[Node] = { - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + val content = + generateBasicStats() ++ +

    Statistics over last {listener.completedBatches.size} processed batches

    ++ + generateNetworkStatsTable() ++ + generateBatchStatsTable() + UIUtils.headerStreamingPage(content, "", parent.appName, "Spark Streaming Overview") + } - private def render(request: HttpServletRequest): Seq[Node] = { - val content = generateBatchStatsTable() ++ generateNetworkStatsTable() - UIUtils.headerStreamingPage(content, "", appName, "Spark Streaming Overview") + private def generateBasicStats(): Seq[Node] = { + + val timeSinceStart = System.currentTimeMillis() - startTime.getTime +
      +
    • + Started at: {startTime.toString} +
    • +
    • + Time since start: {msDurationToString(timeSinceStart)} +
    • +
    • + Batch interval: {msDurationToString(listener.batchDuration)} +
    • +
    • + Processed batches: {listener.numTotalBatchesCompleted} +
    • +
    • +
    } private def generateBatchStatsTable(): Seq[Node] = { @@ -173,18 +180,12 @@ private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { val batchStats =
      -
    • -
      Statistics over last {numBatches} processed batches
      -
    • -
    • - {table.getOrElse("No statistics have been generated yet.")} -
    • + {table.getOrElse("No statistics have been generated yet.")}
    val content = -

    Batch Processing Statistics

    ++ -
    {batchCounts}
    ++ -
    {batchStats}
    +
    Batch Processing Statistics
    ++ +
    {batchStats}
    content } @@ -198,7 +199,7 @@ private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { val dataRows = (0 until numNetworkReceivers).map { receiverId => val receiverName = s"Receiver-$receiverId" val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => - d.getQuantiles().map(r => numberToString(r.toLong) + " records/batch") + d.getQuantiles().map(r => numberToString(r.toLong) + " records/second") }.getOrElse { Seq("-", "-", "-", "-", "-") } @@ -210,8 +211,8 @@ private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { } val content = -

    Network Input Statistics

    ++ -
    {table.getOrElse("No network receivers")}
    +
    Network Input Statistics
    ++ +
    {table.getOrElse("No network receivers")}
    content } @@ -241,6 +242,95 @@ private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { } "%.1f%s".formatLocal(Locale.US, value, unit) } + + /** + * Returns a human-readable string representing a duration such as "5 second 35 ms" + */ + private def msDurationToString(ms: Long): String = { + try { + val second = 1000L + val minute = 60 * second + val hour = 60 * minute + val day = 24 * hour + val week = 7 * day + val year = 365 * day + + def toString(num: Long, unit: String): String = { + if (num == 0) { + "" + } else if (num == 1) { + s"$num $unit" + } else { + s"$num ${unit}s" + } + } + + val millisecondsString = if (ms % second == 0) "" else s"${ms % second} ms" + val secondString = toString((ms % minute) / second, "second") + val minuteString = toString((ms % hour) / minute, "minute") + val hourString = toString((ms % day) / hour, "hour") + val dayString = toString((ms % week) / day, "day") + val weekString = toString((ms % year) / week, "week") + val yearString = toString(ms / year, "year") + + Seq( + second -> millisecondsString, + minute -> s"$secondString $millisecondsString", + hour -> s"$minuteString $secondString", + day -> s"$hourString $minuteString $secondString", + week -> s"$dayString $hourString $minuteString", + year -> s"$weekString $dayString $hourString" + ).foreach { + case (durationLimit, durationString) if (ms < durationLimit) => + return durationString + case e: Any => // matcherror is thrown without this + } + return s"$yearString $weekString $dayString" + } catch { + case e: Exception => + logError("Error converting time to string", e) + return "" + } + } +} + + +private[spark] class StreamingUI(val ssc: StreamingContext) extends Logging { + + val sc = ssc.sparkContext + val conf = sc.conf + val appName = sc.appName + val listener = new StreamingUIListener(ssc) + val overviewPage = new StreamingPage(this) + + private val bindHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) + private val port = conf.getInt("spark.streaming.ui.port", StreamingUI.DEFAULT_PORT) + private val securityManager = sc.env.securityManager + private val handlers: Seq[ServletContextHandler] = { + Seq( + createServletHandler("/", + (request: HttpServletRequest) => overviewPage.render(request), securityManager), + createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static") + ) + } + + private var serverInfo: Option[ServerInfo] = None + + ssc.addStreamingListener(listener) + + def bind() { + try { + serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) + logInfo("Started Spark Streaming Web UI at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Spark JettyUtils", e) + System.exit(1) + } + } + + private def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) } object StreamingUI { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala index 62e95135fa5c5..b87bba87129b5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala @@ -25,8 +25,15 @@ private[spark] object UIUtils { type="text/css" /> {appName} - {title} + - + -
    -
    - {execTable} -
    -
    ; +
    +
    + {execTable} +
    +
    ; UIUtils.headerSparkPage( content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) @@ -158,55 +140,3 @@ private[ui] class ExecutorsUI(parent: SparkUI) { execFields.zip(execValues).toMap } } - -/** - * A SparkListener that prepares information to be displayed on the ExecutorsUI - */ -private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) - extends SparkListener { - - val executorToTasksActive = HashMap[String, Int]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - def storageStatusList = storageStatusListener.storageStatusList - - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val eid = formatExecutorId(taskStart.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val info = taskEnd.taskInfo - if (info != null) { - val eid = formatExecutorId(info.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - } - - // Update shuffle read/write - val metrics = taskEnd.taskMetrics - if (metrics != null) { - metrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead - } - metrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten - } - } - } - } - - // This addresses executor ID inconsistencies in the local mode - private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) -} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 73861ae6746da..31173e48d7a1e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -20,11 +20,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable import scala.xml.Node +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing executor summary */ -private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { - private lazy val listener = parent.listener +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { + private lazy val listener = parent.jobProgressListener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -69,7 +70,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {parent.formatDuration(v.taskTime)} + {UIUtils.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 70d62b66a4829..c600e58af004d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -23,23 +23,23 @@ import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{UIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class IndexPage(parent: JobProgressUI) { +private[ui] class IndexPage(parent: JobProgressTab) extends UIPage("") { private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener + private lazy val listener = parent.jobProgressListener private lazy val isFairScheduler = parent.isFairScheduler - def render(request: HttpServletRequest): Seq[Node] = { + override def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) val completedStagesTable = @@ -57,7 +57,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { // Total duration is not meaningful unless the UI is live
  • Total Duration: - {parent.formatDuration(now - sc.startTime)} + {UIUtils.formatDuration(now - sc.startTime)}
  • }}
  • diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala similarity index 50% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index b2c67381cc3da..c40b75d684510 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -17,44 +17,29 @@ package org.apache.spark.ui.jobs -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.{SparkUI, UITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressUI(parent: SparkUI) { +private[ui] class JobProgressTab(parent: SparkUI) extends UITab("stages") { val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc - lazy val listener = _listener.get - lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) - private val poolPage = new PoolPage(this) - private var _listener: Option[JobProgressListener] = None - def start() { val conf = if (live) sc.conf else new SparkConf - _listener = Some(new JobProgressListener(conf)) + listener = Some(new JobProgressListener(conf)) + attachPage(new IndexPage(this)) + attachPage(new StagePage(this)) + attachPage(new PoolPage(this)) } - def formatDuration(ms: Long) = Utils.msDurationToString(ms) + def jobProgressListener = { + assert(listener.isDefined, "JobProgressTab has not started yet!") + listener.get.asInstanceOf[JobProgressListener] + } - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/stages/stage", - (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), - createServletHandler("/stages/pool", - (request: HttpServletRequest) => poolPage.render(request), parent.securityManager, basePath), - createServletHandler("/stages", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) + def isFairScheduler = jobProgressListener.schedulingMode.exists(_ == SchedulingMode.FAIR) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index bd33182b70059..53200ecdd4fee 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -23,17 +23,17 @@ import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{UIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressUI) { +private[ui] class PoolPage(parent: JobProgressTab) extends UIPage("pool") { private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener + private lazy val listener = parent.jobProgressListener - def render(request: HttpServletRequest): Seq[Node] = { + override def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index c5c8d8668740b..bb7a9c14f7761 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,10 +24,9 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { private val basePath = parent.basePath - private val poolToActiveStages = listener.poolToActiveStages - private lazy val listener = parent.listener + private lazy val listener = parent.jobProgressListener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -48,7 +47,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { SchedulingMode - {rows.map(r => makeRow(r, poolToActiveStages))} + {rows.map(r => makeRow(r, listener.poolToActiveStages))} } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0c55f2ee7e944..bd3d878b1567f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -23,16 +23,16 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.ui.Page._ -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{UIPage, UIUtils} import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressUI) { +private[ui] class StagePage(parent: JobProgressTab) extends UIPage("stage") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener + private lazy val listener = parent.jobProgressListener - def render(request: HttpServletRequest): Seq[Node] = { + override def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt @@ -58,7 +58,7 @@ private[ui] class StagePage(parent: JobProgressUI) { val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val tasksActive = listener.stageIdToTasksActive(stageId).values tasksActive.foreach(activeTime += _.timeRunning(now)) @@ -68,7 +68,7 @@ private[ui] class StagePage(parent: JobProgressUI) {
    • Total task time across all tasks: - {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)} + {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
    • {if (hasShuffleRead)
    • @@ -119,13 +119,13 @@ private[ui] class StagePage(parent: JobProgressUI) { } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). - get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) + get.getQuantiles().map(ms => UIUtils.formatDuration(ms.toLong)) val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() - .map(ms => parent.formatDuration(ms.toLong)) + .map(ms => UIUtils.formatDuration(ms.toLong)) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { @@ -136,7 +136,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val gettingResultQuantiles = "Time spent fetching task results" +: Distribution(gettingResultTimes).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, @@ -153,7 +153,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val schedulerDelayQuantiles = "Scheduler delay" +: Distribution(schedulerDelays).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } def getQuantileCols(data: Seq[Double]) = @@ -217,8 +217,8 @@ private[ui] class StagePage(parent: JobProgressUI) { taskData match { case TaskUIData(info, metrics, exception) => val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) + else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) @@ -233,8 +233,8 @@ private[ui] class StagePage(parent: JobProgressUI) { val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => - if (ms == 0) "" else parent.formatDuration(ms) + val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else UIUtils.formatDuration(ms) }.getOrElse("") val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) @@ -252,15 +252,15 @@ private[ui] class StagePage(parent: JobProgressUI) { {info.status} {info.taskLocality} {info.host} - {WebUI.formatDate(new Date(info.launchTime))} + {UIUtils.formatDate(new Date(info.launchTime))} {formatDuration} - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + {if (serializationTime > 0) UIUtils.formatDuration(serializationTime) else ""} {if (shuffleRead) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index ac61568af52d2..e41e6c0ab7da6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,9 +27,9 @@ import org.apache.spark.ui.{WebUI, UIUtils} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { +private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressTab) { private val basePath = parent.basePath - private lazy val listener = parent.listener + private lazy val listener = parent.jobProgressListener private lazy val isFairScheduler = parent.isFairScheduler def toNodeSeq: Seq[Node] = { @@ -81,14 +81,14 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
      {d}
      {nameLink}
      ).getOrElse(nameLink) val submissionTime = s.submissionTime match { - case Some(t) => WebUI.formatDate(new Date(t)) + case Some(t) => UIUtils.formatDate(new Date(t)) case None => "Unknown" } val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) val duration = s.submissionTime.map { t => if (finishTime > t) finishTime - t else System.currentTimeMillis - t } - val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala similarity index 78% rename from core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala rename to core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala index a7b24ff695214..d7fcd823a242f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala @@ -17,38 +17,27 @@ package org.apache.spark.ui.storage -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class BlockManagerUI(parent: SparkUI) { +private[ui] class BlockManagerTab(parent: SparkUI) extends UITab("storage") { val appName = parent.appName val basePath = parent.basePath - private val indexPage = new IndexPage(this) - private val rddPage = new RDDPage(this) - private var _listener: Option[BlockManagerListener] = None - - lazy val listener = _listener.get - def start() { - _listener = Some(new BlockManagerListener(parent.storageStatusListener)) + listener = Some(new BlockManagerListener(parent.storageStatusListener)) + attachPage(new IndexPage(this)) + attachPage(new RddPage(this)) } - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/storage/rdd", - (request: HttpServletRequest) => rddPage.render(request), parent.securityManager, basePath), - createServletHandler("/storage", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) + def blockManagerListener = { + assert(listener.isDefined, "BlockManagerTab has not started yet!") + listener.get.asInstanceOf[BlockManagerListener] + } } /** diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index b2732de51058a..f06fdf7ce4d4c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -23,16 +23,16 @@ import scala.xml.Node import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{UIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class IndexPage(parent: BlockManagerUI) { +private[ui] class IndexPage(parent: BlockManagerTab) extends UIPage("") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener + private lazy val listener = parent.blockManagerListener - def render(request: HttpServletRequest): Seq[Node] = { + override def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 3f42eba4ece00..cb21754550494 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -23,16 +23,16 @@ import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{UIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RDDPage(parent: BlockManagerUI) { +private[ui] class RddPage(parent: BlockManagerTab) extends UIPage("rdd") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener + private lazy val listener = parent.blockManagerListener - def render(request: HttpServletRequest): Seq[Node] = { + override def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { From a37ad4f55c38716bdda5921ae83daff6187c04ba Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 2 Apr 2014 20:41:33 -0700 Subject: [PATCH 07/26] Comments, imports and formatting (minor) --- .../org/apache/spark/deploy/master/Master.scala | 2 +- .../apache/spark/deploy/master/ui/MasterWebUI.scala | 1 + .../org/apache/spark/deploy/worker/Worker.scala | 2 +- .../apache/spark/deploy/worker/ui/WorkerWebUI.scala | 1 + core/src/main/scala/org/apache/spark/ui/WebUI.scala | 13 +++++++++---- .../org/apache/spark/ui/env/EnvironmentTab.scala | 2 +- .../org/apache/spark/ui/exec/ExecutorsTab.scala | 2 +- .../scala/org/apache/spark/ui/exec/IndexPage.scala | 10 +++++----- .../scala/org/apache/spark/ui/jobs/IndexPage.scala | 2 +- .../org/apache/spark/ui/jobs/JobProgressTab.scala | 2 +- .../scala/org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../scala/org/apache/spark/ui/jobs/StageTable.scala | 2 +- .../apache/spark/ui/storage/BlockManagerTab.scala | 2 +- .../org/apache/spark/ui/storage/IndexPage.scala | 2 +- .../scala/org/apache/spark/ui/storage/RDDPage.scala | 2 +- 16 files changed, 28 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 4c12ab192079f..a58e14bdcdd34 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -85,7 +85,6 @@ private[spark] class Master( val masterSource = new MasterSource(this) val webUi = new MasterWebUI(this, webUiPort) - webUi.start() val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -116,6 +115,7 @@ private[spark] class Master( logInfo("Starting Spark master at " + masterUrl) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + webUi.start() webUi.bind() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index fbcc76b3cc150..c8d51e44a4dff 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -35,6 +35,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) + /** Initialize all components of the server. Must be called before bind(). */ def start() { attachPage(new ApplicationPage(this)) attachPage(new IndexPage(this)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 97b5a37f1439c..e1921bcc0d025 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -122,10 +122,10 @@ private[spark] class Worker( host, port, cores, Utils.megabytesToString(memory))) logInfo("Spark home: " + sparkHome) createWorkDir() + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() webUi.bind() - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) registerWithMaster() metricsSystem.registerSource(workerSource) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index e0a60121cb65a..ae1b7ab014e6e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -38,6 +38,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) val timeout = AkkaUtils.askTimeout(worker.conf) + /** Initialize all components of the server. Must be called before bind(). */ def start() { val logPage = new LogPage(this) attachPage(logPage) diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 6f7385086b534..f20aec893c787 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -19,15 +19,16 @@ package org.apache.spark.ui import javax.servlet.http.HttpServletRequest +import scala.collection.mutable.ArrayBuffer +import scala.xml.Node + import org.eclipse.jetty.servlet.ServletContextHandler +import org.json4s.JsonAST.{JNothing, JValue} import org.apache.spark.SecurityManager +import org.apache.spark.scheduler.SparkListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils -import scala.collection.mutable.ArrayBuffer -import org.apache.spark.scheduler.SparkListener -import scala.xml.Node -import org.json4s.JsonAST.{JNothing, JValue} /** * The top level component of the UI hierarchy that contains the server. @@ -70,6 +71,9 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: /** Return a list of handlers attached to this UI. */ def getHandlers = handlers.toSeq + /** Initialize all components of the server. Must be called before bind(). */ + def start() + /** * Bind to the HTTP server behind this web interface. * Overridden implementation should set serverInfo. @@ -101,6 +105,7 @@ private[spark] abstract class UITab(val prefix: String) { pages += page } + /** Initialize listener and attach pages. */ def start() } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index dd4ea2a2332a2..ad355e59a37aa 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -29,7 +29,7 @@ private[ui] class EnvironmentTab(parent: SparkUI) extends UITab("environment") { attachPage(new IndexPage(this)) } - def environmentListener = { + def environmentListener: EnvironmentListener = { assert(listener.isDefined, "EnvironmentTab has not started yet!") listener.get.asInstanceOf[EnvironmentListener] } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 2b833c58c8e44..3941a1849b182 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -33,7 +33,7 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends UITab("executors") { attachPage(new IndexPage(this)) } - def executorsListener = { + def executorsListener: ExecutorsListener = { assert(listener.isDefined, "ExecutorsTab has not started yet!") listener.get.asInstanceOf[ExecutorsListener] } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala index fbbba2f63878f..9a711d773ae01 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala @@ -50,11 +50,11 @@ private[ui] class IndexPage(parent: ExecutorsTab) extends UIPage("") {
    -
    -
    - {execTable} -
    -
    ; +
    +
    + {execTable} +
    +
    ; UIUtils.headerSparkPage( content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index c600e58af004d..5c752793d2342 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -22,8 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable -import org.apache.spark.ui.Page._ import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.Page.Stages /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class IndexPage(parent: JobProgressTab) extends UIPage("") { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index c40b75d684510..611febe10c1aa 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -36,7 +36,7 @@ private[ui] class JobProgressTab(parent: SparkUI) extends UITab("stages") { attachPage(new PoolPage(this)) } - def jobProgressListener = { + def jobProgressListener: JobProgressListener = { assert(listener.isDefined, "JobProgressTab has not started yet!") listener.get.asInstanceOf[JobProgressListener] } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 53200ecdd4fee..9382251d6e612 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -22,8 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} -import org.apache.spark.ui.Page._ import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.Page.Stages /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressTab) extends UIPage("pool") { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index bd3d878b1567f..a4f6e5d69c515 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,8 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.Page._ import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.Page.Stages import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index e41e6c0ab7da6..d918feafd97d0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.HashMap import scala.xml.Node import org.apache.spark.scheduler.{StageInfo, TaskInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala index d7fcd823a242f..2b9cf35fcde55 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala @@ -34,7 +34,7 @@ private[ui] class BlockManagerTab(parent: SparkUI) extends UITab("storage") { attachPage(new RddPage(this)) } - def blockManagerListener = { + def blockManagerListener: BlockManagerListener = { assert(listener.isDefined, "BlockManagerTab has not started yet!") listener.get.asInstanceOf[BlockManagerListener] } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index f06fdf7ce4d4c..7bfcf13c69c6b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -22,8 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.Page._ import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.Page.Storage import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index cb21754550494..35ccfd505a3ef 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -22,8 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.Page._ import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.Page.Storage import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ From ed25dfc704af544994a4a3bfe98ecaef9fff5bdb Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 2 Apr 2014 21:13:45 -0700 Subject: [PATCH 08/26] Generalize SparkUI header to display tabs dynamically --- .../main/scala/org/apache/spark/ui/Page.scala | 22 ------------ .../scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../scala/org/apache/spark/ui/UIUtils.scala | 36 +++++-------------- .../scala/org/apache/spark/ui/WebUI.scala | 11 +++--- .../apache/spark/ui/env/EnvironmentTab.scala | 2 ++ .../org/apache/spark/ui/env/IndexPage.scala | 3 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 2 ++ .../org/apache/spark/ui/exec/IndexPage.scala | 5 ++- .../org/apache/spark/ui/jobs/IndexPage.scala | 3 +- .../apache/spark/ui/jobs/JobProgressTab.scala | 2 ++ .../org/apache/spark/ui/jobs/PoolPage.scala | 5 ++- .../org/apache/spark/ui/jobs/StagePage.scala | 9 +++-- .../spark/ui/storage/BlockManagerTab.scala | 2 ++ .../apache/spark/ui/storage/IndexPage.scala | 3 +- .../org/apache/spark/ui/storage/RDDPage.scala | 8 ++--- 15 files changed, 37 insertions(+), 78 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/ui/Page.scala diff --git a/core/src/main/scala/org/apache/spark/ui/Page.scala b/core/src/main/scala/org/apache/spark/ui/Page.scala deleted file mode 100644 index b2a069a37552d..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/Page.scala +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ui - -private[spark] object Page extends Enumeration { - val Stages, Storage, Environment, Executors = Value -} diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index d8ea1b13362e3..c333dd3784bb7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -55,8 +55,8 @@ private[spark] class SparkUI( /** Initialize all components of the server. Must be called before bind(). */ def start() { - attachTab(new BlockManagerTab(this)) attachTab(new JobProgressTab(this)) + attachTab(new BlockManagerTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index de4216849dc7d..7cf16b5ed29b1 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -25,8 +25,6 @@ import scala.xml.Node /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils { - import Page._ - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") @@ -62,26 +60,13 @@ private[spark] object UIUtils { basePath: String, appName: String, title: String, - page: Page.Value) : Seq[Node] = { - val jobs = page match { - case Stages => -
  • Stages
  • - case _ =>
  • Stages
  • - } - val storage = page match { - case Storage => -
  • Storage
  • - case _ =>
  • Storage
  • - } - val environment = page match { - case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • - } - val executors = page match { - case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • + tabs: Seq[UITab], + activeTab: UITab) : Seq[Node] = { + + val header = tabs.map { tab => +
  • + {tab.name} +
  • } @@ -100,12 +85,7 @@ private[spark] object UIUtils { - + diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index f20aec893c787..0b847a9a471f0 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -42,6 +42,10 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: protected val handlers = ArrayBuffer[ServletContextHandler]() protected var serverInfo: Option[ServerInfo] = None + def getTabs: Seq[UITab] = tabs.toSeq + def getHandlers: Seq[ServletContextHandler] = handlers.toSeq + def getListeners: Seq[SparkListener] = tabs.flatMap(_.listener) + /** Attach a tab to this UI, along with all of its attached pages. Only valid before bind(). */ def attachTab(tab: UITab) { tab.start() @@ -65,12 +69,6 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: handlers += handler } - /** Return a list of listeners attached to this UI. */ - def getListeners = tabs.flatMap(_.listener) - - /** Return a list of handlers attached to this UI. */ - def getHandlers = handlers.toSeq - /** Initialize all components of the server. Must be called before bind(). */ def start() @@ -98,6 +96,7 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: private[spark] abstract class UITab(val prefix: String) { val pages = ArrayBuffer[UIPage]() var listener: Option[SparkListener] = None + var name = prefix.capitalize /** Attach a page to this tab. This prepends the page's prefix with the tab's own prefix. */ def attachPage(page: UIPage) { diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index ad355e59a37aa..6a2304f1ad42f 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -33,6 +33,8 @@ private[ui] class EnvironmentTab(parent: SparkUI) extends UITab("environment") { assert(listener.isDefined, "EnvironmentTab has not started yet!") listener.get.asInstanceOf[EnvironmentListener] } + + def headerTabs: Seq[UITab] = parent.getTabs } /** diff --git a/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala index bf1872f18d54e..bde672909bbcc 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala @@ -22,7 +22,6 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.ui.{UIUtils, UIPage} -import org.apache.spark.ui.Page.Environment private[ui] class IndexPage(parent: EnvironmentTab) extends UIPage("") { private val appName = parent.appName @@ -46,7 +45,7 @@ private[ui] class IndexPage(parent: EnvironmentTab) extends UIPage("") {

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, basePath, appName, "Environment", Environment) + UIUtils.headerSparkPage(content, basePath, appName, "Environment", parent.headerTabs, parent) } private def propertyHeader = Seq("Name", "Value") diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 3941a1849b182..c1f5ca856ffe1 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -37,6 +37,8 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends UITab("executors") { assert(listener.isDefined, "ExecutorsTab has not started yet!") listener.get.asInstanceOf[ExecutorsListener] } + + def headerTabs: Seq[UITab] = parent.getTabs } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala index 9a711d773ae01..1956b6c63929e 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala @@ -22,7 +22,6 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.ui.{UIPage, UIUtils} -import org.apache.spark.ui.Page.Executors import org.apache.spark.util.Utils private[ui] class IndexPage(parent: ExecutorsTab) extends UIPage("") { @@ -56,8 +55,8 @@ private[ui] class IndexPage(parent: ExecutorsTab) extends UIPage("") { ; - UIUtils.headerSparkPage( - content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, basePath, appName, "Executors (" + execInfo.size + ")", + parent.headerTabs, parent) } /** Header fields for the executors table */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 5c752793d2342..2b54603af104e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -23,7 +23,6 @@ import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.{UIPage, UIUtils} -import org.apache.spark.ui.Page.Stages /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class IndexPage(parent: JobProgressTab) extends UIPage("") { @@ -92,7 +91,7 @@ private[ui] class IndexPage(parent: JobProgressTab) extends UIPage("") {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 611febe10c1aa..93d26f7dd3632 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -42,4 +42,6 @@ private[ui] class JobProgressTab(parent: SparkUI) extends UITab("stages") { } def isFairScheduler = jobProgressListener.schedulingMode.exists(_ == SchedulingMode.FAIR) + + def headerTabs: Seq[UITab] = parent.getTabs } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 9382251d6e612..7fffe2affb0f2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -23,7 +23,6 @@ import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.{UIPage, UIUtils} -import org.apache.spark.ui.Page.Stages /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressTab) extends UIPage("pool") { @@ -51,8 +50,8 @@ private[ui] class PoolPage(parent: JobProgressTab) extends UIPage("pool") {

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage( - content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Fair Scheduler Pool: " + poolName, + parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index a4f6e5d69c515..372210919cd91 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -23,7 +23,6 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.ui.{UIPage, UIUtils} -import org.apache.spark.ui.Page.Stages import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ @@ -42,8 +41,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends UIPage("stage") {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet - return UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage(content, basePath, appName, + "Details for Stage %s".format(stageId), parent.headerTabs, parent) } val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -204,8 +203,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends UIPage("stage") {

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++

    Tasks

    ++ taskTable - UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), + parent.headerTabs, parent) } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala index 2b9cf35fcde55..ac83f71ed31de 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala @@ -38,6 +38,8 @@ private[ui] class BlockManagerTab(parent: SparkUI) extends UITab("storage") { assert(listener.isDefined, "BlockManagerTab has not started yet!") listener.get.asInstanceOf[BlockManagerListener] } + + def headerTabs: Seq[UITab] = parent.getTabs } /** diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 7bfcf13c69c6b..96b08d07d48d2 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -23,7 +23,6 @@ import scala.xml.Node import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.{UIPage, UIUtils} -import org.apache.spark.ui.Page.Storage import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ @@ -35,7 +34,7 @@ private[ui] class IndexPage(parent: BlockManagerTab) extends UIPage("") { override def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent) } /** Header fields for the RDD table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 35ccfd505a3ef..a65ba0a020bcd 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -23,7 +23,6 @@ import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} import org.apache.spark.ui.{UIPage, UIUtils} -import org.apache.spark.ui.Page.Storage import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ @@ -37,7 +36,8 @@ private[ui] class RddPage(parent: BlockManagerTab) extends UIPage("rdd") { val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", Storage) + return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", + parent.headerTabs, parent) } // Worker table @@ -95,8 +95,8 @@ private[ui] class RddPage(parent: BlockManagerTab) extends UIPage("rdd") { ; - UIUtils.headerSparkPage( - content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, basePath, appName, "RDD Storage Info for " + rddInfo.name, + parent.headerTabs, parent) } /** Header fields for the worker table */ From 53be2c54fd2d0fd55efe60e61a3daae5a52f8c5f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 3 Apr 2014 15:39:31 -0700 Subject: [PATCH 09/26] Minor style updates. --- .../dstream/NetworkInputDStream.scala | 14 ++++++- .../spark/streaming/ui/StreamingUI.scala | 37 +++++++++++++------ .../apache/spark/streaming/ui/UIUtils.scala | 2 +- .../org/apache/spark/streaming/UISuite.scala | 12 +----- 4 files changed, 40 insertions(+), 25 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 8309b82d0bbbd..a3ee1213200e9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -184,7 +184,12 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** * Pushes a block (as an ArrayBuffer filled with data) into the block manager. */ - def pushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + arrayBuffer: ArrayBuffer[T], + metadata: Any, + level: StorageLevel + ) { env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) actor ! ReportBlock(blockId, arrayBuffer.size, metadata) } @@ -192,7 +197,12 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** * Pushes a block (as bytes) into the block manager. */ - def pushBlock(blockId: StreamBlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + bytes: ByteBuffer, + metadata: Any, + level: StorageLevel + ) { env.blockManager.putBytes(blockId, bytes, level) actor ! ReportBlock(blockId, -1 , metadata) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala index 86427ca171489..9a3cd8058e338 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala @@ -105,8 +105,13 @@ private[ui] class StreamingUIListener(ssc: StreamingContext) extends StreamingLi val latestBatchInfos = allBatches.reverse.take(batchInfoLimit) val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) (0 until numNetworkReceivers).map { receiverId => - val blockInfoOfParticularReceiver = latestBlockInfos.map(_.get(receiverId).getOrElse(Array.empty)) - val recordsOfParticularReceiver = blockInfoOfParticularReceiver.map(_.map(_.numRecords).sum.toDouble * 1000 / batchDuration) + val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => + batchInfo.get(receiverId).getOrElse(Array.empty) + } + val recordsOfParticularReceiver = blockInfoOfParticularReceiver.map { blockInfo => + // calculate records per second for each batch + blockInfo.map(_.numRecords).sum.toDouble * 1000 / batchDuration + } val distributionOption = Distribution(recordsOfParticularReceiver) (receiverId, distributionOption) }.toMap @@ -231,16 +236,24 @@ private[ui] class StreamingPage(parent: StreamingUI) extends Logging { val numBatches = listener.completedBatches.size val lastCompletedBatch = listener.lastCompletedBatch val table = if (numBatches > 0) { - val processingDelayQuantilesRow = - Seq("Processing Time", msDurationToString(lastCompletedBatch.flatMap(_.processingDelay))) ++ - getQuantiles(listener.processingDelayDistribution) - val schedulingDelayQuantilesRow = - Seq("Scheduling Delay", msDurationToString(lastCompletedBatch.flatMap(_.schedulingDelay))) ++ - getQuantiles(listener.schedulingDelayDistribution) - val totalDelayQuantilesRow = - Seq("Total Delay", msDurationToString(lastCompletedBatch.flatMap(_.totalDelay))) ++ - getQuantiles(listener.totalDelayDistribution) - + val processingDelayQuantilesRow = { + Seq( + "Processing Time", + msDurationToString(lastCompletedBatch.flatMap(_.processingDelay)) + ) ++ getQuantiles(listener.processingDelayDistribution) + } + val schedulingDelayQuantilesRow = { + Seq( + "Scheduling Delay", + msDurationToString(lastCompletedBatch.flatMap(_.schedulingDelay)) + ) ++ getQuantiles(listener.schedulingDelayDistribution) + } + val totalDelayQuantilesRow = { + Seq( + "Total Delay", + msDurationToString(lastCompletedBatch.flatMap(_.totalDelay)) + ) ++ getQuantiles(listener.totalDelayDistribution) + } val headerRow = Seq("Metric", "Last batch", "Minimum", "25th percentile", "Median", "75th percentile", "Maximum") val dataRows: Seq[Seq[String]] = Seq( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala index 401ee1fe63590..51b11a29cbd98 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala @@ -45,7 +45,7 @@ private[spark] object UIUtils { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 1ca8b259c9622..a4e42189f3309 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -25,7 +25,7 @@ import scala.util.Random class UISuite extends FunSuite with BeforeAndAfterAll { - test("Testing") { + ignore("Testing") { runStreaming(1000000) } @@ -58,12 +58,4 @@ class FunctionBasedInputDStream[T: ClassTag]( def stop(): Unit = {} def compute(validTime: Time): Option[RDD[T]] = function(ssc, validTime) -} - - - - - - - - +} \ No newline at end of file From 9a48fa1de7b357f6ffdaad8e93af7b2b7e39bc06 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 3 Apr 2014 15:53:48 -0700 Subject: [PATCH 10/26] Allow adding tabs to SparkUI dynamically + add example An example of how this is done is in org.apache.spark.ui.FooTab. Run it through bin/spark-class to see what it looks like (which should more or less match your expectations...). --- .../spark/deploy/master/ui/MasterWebUI.scala | 18 +-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- .../scala/org/apache/spark/ui/FooTab.scala | 105 ++++++++++++++++++ .../scala/org/apache/spark/ui/SparkUI.scala | 13 ++- .../scala/org/apache/spark/ui/WebUI.scala | 26 ++++- 5 files changed, 139 insertions(+), 25 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/FooTab.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index c8d51e44a4dff..3a30919a70584 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -35,7 +35,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) - /** Initialize all components of the server. Must be called before bind(). */ + /** Initialize all components of the server. */ def start() { attachPage(new ApplicationPage(this)) attachPage(new IndexPage(this)) @@ -59,25 +59,13 @@ class MasterWebUI(val master: Master, requestedPort: Int) /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ def attachUI(ui: SparkUI) { assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.getHandlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } + ui.getHandlers.foreach(attachHandler) } /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ def detachUI(ui: SparkUI) { assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.getHandlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } + ui.getHandlers.foreach(detachHandler) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index ae1b7ab014e6e..490a383be42e1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -38,7 +38,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) val timeout = AkkaUtils.askTimeout(worker.conf) - /** Initialize all components of the server. Must be called before bind(). */ + /** Initialize all components of the server. */ def start() { val logPage = new LogPage(this) attachPage(logPage) diff --git a/core/src/main/scala/org/apache/spark/ui/FooTab.scala b/core/src/main/scala/org/apache/spark/ui/FooTab.scala new file mode 100644 index 0000000000000..1e30fa75a263d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/FooTab.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 org.apache.spark.ui + +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable +import scala.xml.Node + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} + +/* + * This is an example of how to extend the SparkUI by adding new tabs to it. It is intended + * only as a demonstration and should be removed before merging into master! + * + * bin/spark-class org.apache.spark.ui.FooTab + */ + +/** A tab that displays basic information about jobs seen so far. */ +private[spark] class FooTab(parent: SparkUI) extends UITab("foo") { + val appName = parent.appName + val basePath = parent.basePath + + def start() { + listener = Some(new FooListener) + attachPage(new IndexPage(this)) + } + + def fooListener: FooListener = { + assert(listener.isDefined, "ExecutorsTab has not started yet!") + listener.get.asInstanceOf[FooListener] + } + + def headerTabs: Seq[UITab] = parent.getTabs +} + +/** A foo page. Enough said. */ +private[spark] class IndexPage(parent: FooTab) extends UIPage("") { + private val appName = parent.appName + private val basePath = parent.basePath + private val listener = parent.fooListener + + override def render(request: HttpServletRequest): Seq[Node] = { + val results = listener.jobResultMap.toSeq.sortBy { case (k, _) => k } + val content = +
    +
    + Foo Jobs: +
      + {results.map { case (k, v) =>
    • Job {k}: {v}
    • }} +
    +
    +
    + UIUtils.headerSparkPage(content, basePath, appName, "Foo", parent.headerTabs, parent) + } +} + +/** A listener that maintains a mapping between job IDs and job results. */ +private[spark] class FooListener extends SparkListener { + val jobResultMap = mutable.Map[Int, String]() + + override def onJobEnd(end: SparkListenerJobEnd) { + jobResultMap(end.jobId) = end.jobResult.toString + } +} + + +/** + * Start a SparkContext and a SparkUI with a FooTab attached. + */ +private[spark] object FooTab { + def main(args: Array[String]) { + val sc = new SparkContext("local", "Foo Tab", new SparkConf) + val fooTab = new FooTab(sc.ui) + sc.ui.attachTab(fooTab) + + // Run a few jobs + sc.parallelize(1 to 1000).count() + sc.parallelize(1 to 2000).persist().count() + sc.parallelize(1 to 3000).map(i => (i/2, i)).groupByKey().count() + sc.parallelize(1 to 4000).map(i => (i/2, i)).groupByKey().persist().count() + sc.parallelize(1 to 5000).map(i => (i/2, i)).groupByKey().persist().count() + sc.parallelize(1 to 6000).map(i => (i/2, i)).groupByKey().persist().count() + sc.parallelize(1 to 7000).map(i => (i/2, i)).groupByKey().persist().count() + + readLine("\n> Started SparkUI with a Foo tab...") + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index c333dd3784bb7..ac22189f9f04f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -52,8 +52,9 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener + listenerBus.addListener(storageStatusListener) - /** Initialize all components of the server. Must be called before bind(). */ + /** Initialize all components of the server. */ def start() { attachTab(new JobProgressTab(this)) attachTab(new BlockManagerTab(this)) @@ -64,14 +65,10 @@ private[spark] class SparkUI( if (live) { sc.env.metricsSystem.getServletHandlers.foreach(attachHandler) } - // Storage status listener must receive events first, as other listeners depend on its state - listenerBus.addListener(storageStatusListener) - getListeners.foreach(listenerBus.addListener) } /** Bind to the HTTP server behind this web interface. */ def bind() { - assert(!handlers.isEmpty, "SparkUI has not started yet!") try { serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) logInfo("Started Spark web UI at http://%s:%d".format(publicHost, boundPort)) @@ -82,6 +79,12 @@ private[spark] class SparkUI( } } + /** Attach a tab to this UI, along with its corresponding listener if it exists. */ + override def attachTab(tab: UITab) { + super.attachTab(tab) + tab.listener.foreach(listenerBus.addListener) + } + /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 0b847a9a471f0..4392814fd1b39 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -35,7 +35,6 @@ import org.apache.spark.util.Utils * * Each WebUI represents a collection of tabs, each of which in turn represents a collection of * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. - * All tabs and pages must be attached before bind()'ing the server. */ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: String = "") { protected val tabs = ArrayBuffer[UITab]() @@ -46,14 +45,14 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: def getHandlers: Seq[ServletContextHandler] = handlers.toSeq def getListeners: Seq[SparkListener] = tabs.flatMap(_.listener) - /** Attach a tab to this UI, along with all of its attached pages. Only valid before bind(). */ + /** Attach a tab to this UI, along with all of its attached pages. */ def attachTab(tab: UITab) { tab.start() tab.pages.foreach(attachPage) tabs += tab } - /** Attach a page to this UI. Only valid before bind(). */ + /** Attach a page to this UI. */ def attachPage(page: UIPage) { val pagePath = "/" + page.prefix attachHandler(createServletHandler(pagePath, @@ -64,9 +63,26 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: } } - /** Attach a handler to this UI. Only valid before bind(). */ + /** Attach a handler to this UI. */ def attachHandler(handler: ServletContextHandler) { handlers += handler + serverInfo.foreach { info => + info.rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a handler from this UI. */ + def detachHandler(handler: ServletContextHandler) { + handlers -= handler + serverInfo.foreach { info => + info.rootHandler.removeHandler(handler) + if (handler.isStarted) { + handler.stop() + } + } } /** Initialize all components of the server. Must be called before bind(). */ @@ -89,6 +105,7 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: } } + /** * A tab that represents a collection of pages and a unit of listening for Spark events. * Associating each tab with a listener is arbitrary and need not be the case. @@ -108,6 +125,7 @@ private[spark] abstract class UITab(val prefix: String) { def start() } + /** * A page that represents the leaf node in the UI hierarchy. * From 8f7323b76ffb64a30c6efc5b471f9898f94c0979 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 3 Apr 2014 16:33:23 -0700 Subject: [PATCH 11/26] End of file new lines, indentation, and imports (minor) --- core/src/main/scala/org/apache/spark/ui/FooTab.scala | 2 +- core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala | 2 +- .../src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala | 1 - 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/FooTab.scala b/core/src/main/scala/org/apache/spark/ui/FooTab.scala index 1e30fa75a263d..620fe8001a85c 100644 --- a/core/src/main/scala/org/apache/spark/ui/FooTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/FooTab.scala @@ -102,4 +102,4 @@ private[spark] object FooTab { readLine("\n> Started SparkUI with a Foo tab...") } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala index 1956b6c63929e..bc6a822b080c3 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala @@ -11,7 +11,7 @@ * 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 + * See the License for the specific language governing permissions and * limitations under the License. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala index 51b11a29cbd98..4063ce3d7ca44 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala @@ -18,7 +18,6 @@ package org.apache.spark.streaming.ui import scala.xml.Node -import org.apache.spark.ui.Page private[spark] object UIUtils { From c78c92d2240d563219c9d7036d480619dc789d76 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Apr 2014 16:26:00 -0700 Subject: [PATCH 12/26] Remove outdated comment --- core/src/main/scala/org/apache/spark/ui/WebUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 4392814fd1b39..126a7ff2f6080 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -85,7 +85,7 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: } } - /** Initialize all components of the server. Must be called before bind(). */ + /** Initialize all components of the server. */ def start() /** From 1af239b29cf41619ff4df684741ef1e4816ded87 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 9 Apr 2014 22:32:48 -0700 Subject: [PATCH 13/26] Changed streaming UI to attach itself as a tab with the Spark UI. --- .../scala/org/apache/spark/ui/FooTab.scala | 105 ------- .../scala/org/apache/spark/ui/UIUtils.scala | 15 +- .../spark/streaming/StreamingContext.scala | 26 +- .../dstream/NetworkInputDStream.scala | 6 +- .../scheduler/NetworkInputTracker.scala | 4 +- .../ui/StreamingProgressListener.scala | 131 +++++++++ .../spark/streaming/ui/StreamingUI.scala | 264 ++++++------------ .../apache/spark/streaming/ui/UIUtils.scala | 95 ------- 8 files changed, 241 insertions(+), 405 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/ui/FooTab.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala diff --git a/core/src/main/scala/org/apache/spark/ui/FooTab.scala b/core/src/main/scala/org/apache/spark/ui/FooTab.scala deleted file mode 100644 index 620fe8001a85c..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/FooTab.scala +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ui - -import javax.servlet.http.HttpServletRequest - -import scala.collection.mutable -import scala.xml.Node - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ -import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} - -/* - * This is an example of how to extend the SparkUI by adding new tabs to it. It is intended - * only as a demonstration and should be removed before merging into master! - * - * bin/spark-class org.apache.spark.ui.FooTab - */ - -/** A tab that displays basic information about jobs seen so far. */ -private[spark] class FooTab(parent: SparkUI) extends UITab("foo") { - val appName = parent.appName - val basePath = parent.basePath - - def start() { - listener = Some(new FooListener) - attachPage(new IndexPage(this)) - } - - def fooListener: FooListener = { - assert(listener.isDefined, "ExecutorsTab has not started yet!") - listener.get.asInstanceOf[FooListener] - } - - def headerTabs: Seq[UITab] = parent.getTabs -} - -/** A foo page. Enough said. */ -private[spark] class IndexPage(parent: FooTab) extends UIPage("") { - private val appName = parent.appName - private val basePath = parent.basePath - private val listener = parent.fooListener - - override def render(request: HttpServletRequest): Seq[Node] = { - val results = listener.jobResultMap.toSeq.sortBy { case (k, _) => k } - val content = -
    -
    - Foo Jobs: -
      - {results.map { case (k, v) =>
    • Job {k}: {v}
    • }} -
    -
    -
    - UIUtils.headerSparkPage(content, basePath, appName, "Foo", parent.headerTabs, parent) - } -} - -/** A listener that maintains a mapping between job IDs and job results. */ -private[spark] class FooListener extends SparkListener { - val jobResultMap = mutable.Map[Int, String]() - - override def onJobEnd(end: SparkListenerJobEnd) { - jobResultMap(end.jobId) = end.jobResult.toString - } -} - - -/** - * Start a SparkContext and a SparkUI with a FooTab attached. - */ -private[spark] object FooTab { - def main(args: Array[String]) { - val sc = new SparkContext("local", "Foo Tab", new SparkConf) - val fooTab = new FooTab(sc.ui) - sc.ui.attachTab(fooTab) - - // Run a few jobs - sc.parallelize(1 to 1000).count() - sc.parallelize(1 to 2000).persist().count() - sc.parallelize(1 to 3000).map(i => (i/2, i)).groupByKey().count() - sc.parallelize(1 to 4000).map(i => (i/2, i)).groupByKey().persist().count() - sc.parallelize(1 to 5000).map(i => (i/2, i)).groupByKey().persist().count() - sc.parallelize(1 to 6000).map(i => (i/2, i)).groupByKey().persist().count() - sc.parallelize(1 to 7000).map(i => (i/2, i)).groupByKey().persist().count() - - readLine("\n> Started SparkUI with a Foo tab...") - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 7cf16b5ed29b1..fcda341ae5941 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -61,7 +61,9 @@ private[spark] object UIUtils { appName: String, title: String, tabs: Seq[UITab], - activeTab: UITab) : Seq[Node] = { + activeTab: UITab, + refreshInterval: Option[Int] = None + ) : Seq[Node] = { val header = tabs.map { tab =>
  • @@ -78,8 +80,17 @@ private[spark] object UIUtils { type="text/css" /> {appName} - {title} + - +
  • Overview
  • - } - - - - - - - - {appName} - {title} - - - - - -
    -
    -
    -

    - {title} -

    -
    -
    - {content} -
    - - - } - - def listingTable[T]( - headers: Seq[String], - makeRow: T => Seq[Node], - rows: Seq[T], - fixedWidth: Boolean = false): Seq[Node] = { - org.apache.spark.ui.UIUtils.listingTable(headers, makeRow, rows, fixedWidth) - } - - def listingTable[T]( - headers: Seq[String], - rows: Seq[Seq[String]], - fixedWidth: Boolean = false - ): Seq[Node] = { - def makeRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} - org.apache.spark.ui.UIUtils.listingTable(headers, makeRow, rows, fixedWidth) - } -} From 1c0bcefd773d091b014542becdf906e4ed05293d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 9 Apr 2014 22:46:15 -0700 Subject: [PATCH 14/26] Refactored streaming UI into two files. --- ...{StreamingUI.scala => StreamingPage.scala} | 11 +------ .../ui/StreamingProgressListener.scala | 17 ++++++++++ .../spark/streaming/ui/StreamingTab.scala | 32 +++++++++++++++++++ 3 files changed, 50 insertions(+), 10 deletions(-) rename streaming/src/main/scala/org/apache/spark/streaming/ui/{StreamingUI.scala => StreamingPage.scala} (97%) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala similarity index 97% rename from streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala rename to streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 81f883dadc798..8ef5ebb4cd96b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingUI.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -23,10 +23,10 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.Logging -import org.apache.spark.streaming.StreamingContext import org.apache.spark.ui._ import org.apache.spark.util.Distribution +/** Page for Spark Web UI that shows statistics of a streaming job */ private[ui] class StreamingPage(parent: StreamingTab) extends UIPage("") with Logging { @@ -287,12 +287,3 @@ private[ui] class StreamingPage(parent: StreamingTab) } } -private[spark] class StreamingTab(val ssc: StreamingContext) - extends UITab("streaming") with Logging { - - val streamingPage = new StreamingPage(this) - ssc.sc.ui.attachTab(this) - - /** Initialize listener and attach pages. */ - def start() { } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala index 6db1af5245ce2..32a4644e2a3e9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.streaming.ui import org.apache.spark.streaming.{Time, StreamingContext} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala new file mode 100644 index 0000000000000..1aaf7764b5ceb --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.ui.UITab +import org.apache.spark.Logging + +/** Spark Web UI tab that shows statistics of a streaming job */ +private[spark] class StreamingTab(val ssc: StreamingContext) + extends UITab("streaming") with Logging { + + val streamingPage = new StreamingPage(this) + ssc.sc.ui.attachTab(this) + + def start() { } +} From fa760fe0e9c30a6f65035bc341cf110ab5020138 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 9 Apr 2014 22:53:04 -0700 Subject: [PATCH 15/26] Fixed long line. --- .../spark/streaming/scheduler/NetworkInputTracker.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index 4fe4a10163c31..a1e6f5176825a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -118,7 +118,13 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } /** Register a receiver */ - def registerReceiver(streamId: Int, typ: String, host: String, receiverActor: ActorRef, sender: ActorRef) { + def registerReceiver( + streamId: Int, + typ: String, + host: String, + receiverActor: ActorRef, + sender: ActorRef + ) { if (!networkInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } From ee6543fc2c0a3fa4bb42a233197170f1e0b20a73 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 10 Apr 2014 14:28:46 -0700 Subject: [PATCH 16/26] Minor changes based on Andrew's comments. --- .../apache/spark/streaming/ui/StreamingPage.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 8ef5ebb4cd96b..5cd900c2f88f0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -85,7 +85,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val headerRow = Seq( "Receiver", "Location", - s"Records in last batch", + "Records in last batch", "Minimum rate\n[records/sec]", "25th percentile rate\n[records/sec]", "Median rate\n[records/sec]", @@ -102,8 +102,7 @@ private[ui] class StreamingPage(parent: StreamingTab) }.getOrElse { Seq(emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest) } - Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ - receivedRecordStats + Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats } Some(listingTable(headerRow, dataRows, fixedWidth = true)) } else { @@ -225,15 +224,18 @@ private[ui] class StreamingPage(parent: StreamingTab) day -> s"$hourString $minuteString $secondString", week -> s"$dayString $hourString $minuteString", year -> s"$weekString $dayString $hourString" - ).foreach { - case (durationLimit, durationString) if (ms < durationLimit) => + ).foreach { case (durationLimit, durationString) => + if (ms < durationLimit) { + // if time is less than the limit (upto year) return durationString - case e: Any => // matcherror is thrown without this + } } + // if time is more than a year return s"$yearString $weekString $dayString" } catch { case e: Exception => logError("Error converting time to string", e) + // if there is some error, return blank string return "" } } From 548c98c1f1e80f1f51a1b7b08356c13fd8ea25ec Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 10 Apr 2014 19:17:02 -0700 Subject: [PATCH 17/26] Wide refactoring of WebUI, UITab, and UIPage (see commit message) The biggest changes include (1) Decoupling the SparkListener from any member of the hierarchy. This was previously arbitrarily tied to the UITab. (2) Decoupling initializing a UITab from attaching it to a WebUI. This involves having each UITab initializing itself instead. (3) Add an abstract parent for each UITab. This allows us to move the access of header tabs of the UI into the UITab abstract class itself. (4) Abstract bind() logic into WebUI. (5) Renaming UITab -> WebUITab, and UIPage -> WebUIPage. --- .../scala/org/apache/spark/SparkContext.scala | 1 - .../spark/deploy/history/HistoryServer.scala | 32 +++------ .../spark/deploy/history/IndexPage.scala | 4 +- .../apache/spark/deploy/master/Master.scala | 2 - .../deploy/master/ui/ApplicationPage.scala | 4 +- .../spark/deploy/master/ui/IndexPage.scala | 4 +- .../spark/deploy/master/ui/MasterWebUI.scala | 22 ++---- .../apache/spark/deploy/worker/Worker.scala | 1 - .../spark/deploy/worker/ui/IndexPage.scala | 4 +- .../spark/deploy/worker/ui/LogPage.scala | 4 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 35 ++++------ .../scala/org/apache/spark/ui/SparkUI.scala | 45 +++++------- .../scala/org/apache/spark/ui/UIUtils.scala | 7 +- .../scala/org/apache/spark/ui/WebUI.scala | 69 +++++++++++-------- .../apache/spark/ui/env/EnvironmentTab.scala | 16 ++--- .../org/apache/spark/ui/env/IndexPage.scala | 6 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 18 ++--- .../org/apache/spark/ui/exec/IndexPage.scala | 6 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 6 +- .../spark/ui/jobs/JobProgressListener.scala | 10 ++- .../apache/spark/ui/jobs/JobProgressTab.scala | 22 +++--- .../org/apache/spark/ui/jobs/PoolPage.scala | 6 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 6 +- .../org/apache/spark/ui/jobs/StageTable.scala | 2 +- .../spark/ui/storage/BlockManagerTab.scala | 16 ++--- .../apache/spark/ui/storage/IndexPage.scala | 6 +- .../org/apache/spark/ui/storage/RDDPage.scala | 6 +- .../spark/streaming/StreamingContext.scala | 1 - .../spark/streaming/ui/StreamingPage.scala | 2 +- .../spark/streaming/ui/StreamingTab.scala | 12 ++-- 32 files changed, 164 insertions(+), 215 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c9af0778bdb29..28923a1d8c340 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -212,7 +212,6 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the Spark UI, registering all associated listeners private[spark] val ui = new SparkUI(this) - ui.start() ui.bind() // Optionally log Spark events diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 16abfe920da72..df3c394bacfa9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -17,12 +17,9 @@ package org.apache.spark.deploy.history -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} -import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler._ @@ -45,15 +42,15 @@ import org.apache.spark.util.Utils */ class HistoryServer( val baseLogDir: String, + securityManager: SecurityManager, conf: SparkConf) - extends WebUI(new SecurityManager(conf)) with Logging { + extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging { import HistoryServer._ private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = WEB_UI_PORT // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTime = -1L @@ -90,30 +87,20 @@ class HistoryServer( // A mapping of application ID to its history information, which includes the rendered UI val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + initialize() + /** - * Start the history server. + * Initialize the history server. * * This starts a background thread that periodically synchronizes information displayed on * this UI with the event logs in the provided base directory. */ - def start() { + def initialize() { attachPage(new IndexPage(this)) attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) logCheckingThread.start() } - /** Bind to the HTTP server behind this web interface. */ - def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) - logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to bind HistoryServer", e) - System.exit(1) - } - } - /** * Check for any updates to event logs in the base directory. This is only effective once * the server has been bound. @@ -179,12 +166,11 @@ class HistoryServer( val path = logDir.getPath val appId = path.getName val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) - val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) val appListener = new ApplicationEventListener replayBus.addListener(appListener) + val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) // Do not call ui.bind() to avoid creating a new server for each application - ui.start() replayBus.replay() if (appListener.applicationStarted) { attachUI(ui) @@ -267,9 +253,9 @@ object HistoryServer { def main(argStrings: Array[String]) { val args = new HistoryServerArguments(argStrings) - val server = new HistoryServer(args.logDir, conf) + val securityManager = new SecurityManager(conf) + val server = new HistoryServer(args.logDir, securityManager, conf) server.bind() - server.start() // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index eeb22ab000558..69a6baa4aaeab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -21,9 +21,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} -private[spark] class IndexPage(parent: HistoryServer) extends UIPage("") { +private[spark] class IndexPage(parent: HistoryServer) extends WebUIPage("") { override def render(request: HttpServletRequest): Seq[Node] = { val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9c90c4b4d11ef..076bb92bf2a10 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -118,7 +118,6 @@ private[spark] class Master( logInfo("Starting Spark master at " + masterUrl) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi.start() webUi.bind() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) @@ -670,7 +669,6 @@ private[spark] class Master( val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) val ui = new SparkUI( new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) - ui.start() replayBus.replay() app.desc.appUiUrl = ui.basePath appIdToUI(app.id) = ui diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 24282048b842e..d8c3321ea51ec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -28,11 +28,11 @@ import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils private[spark] class ApplicationPage(parent: MasterWebUI) - extends UIPage("app", includeJson = true) { + extends WebUIPage("app", includeJson = true) { private val master = parent.masterActorRef private val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index f011c830a02da..3d2ad04110b77 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -28,10 +28,10 @@ import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: MasterWebUI) extends UIPage("", includeJson = true) { +private[spark] class IndexPage(parent: MasterWebUI) extends WebUIPage("", includeJson = true) { private val master = parent.masterActorRef private val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index d0f1a9bc9ffd1..965f7a0fac9e2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy.master.ui -import javax.servlet.http.HttpServletRequest - import org.apache.spark.Logging import org.apache.spark.deploy.master.Master import org.apache.spark.ui.{SparkUI, WebUI} @@ -30,15 +28,15 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) - extends WebUI(master.securityMgr) with Logging { + extends WebUI(master.securityMgr, requestedPort, master.conf) with Logging { - private val host = Utils.localHostName() - private val port = requestedPort val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) + initialize() + /** Initialize all components of the server. */ - def start() { + def initialize() { attachPage(new ApplicationPage(this)) attachPage(new IndexPage(this)) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) @@ -46,18 +44,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler) } - /** Bind to the HTTP server behind this web interface. */ - def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Master web UI", e) - System.exit(1) - } - } - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ def attachUI(ui: SparkUI) { assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 08ad87957c3d4..52c164ca3c574 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -130,7 +130,6 @@ private[spark] class Worker( createWorkDir() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) - webUi.start() webUi.bind() registerWithMaster() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index bf7d552101484..42ef8ed703779 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -28,10 +28,10 @@ import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: WorkerWebUI) extends UIPage("", includeJson = true) { +private[spark] class IndexPage(parent: WorkerWebUI) extends WebUIPage("", includeJson = true) { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index f57900c99ce3d..8f6b36faf85ee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -22,10 +22,10 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class LogPage(parent: WorkerWebUI) extends UIPage("logPage") { +private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { private val worker = parent.worker private val workDir = parent.workDir diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index c1fdc5cea173c..34b5acd2f9b64 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -20,26 +20,29 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends WebUI(worker.securityMgr) with Logging { +class WorkerWebUI( + val worker: Worker, + val workDir: File, + port: Option[Int] = None) + extends WebUI(worker.securityMgr, WorkerWebUI.getUIPort(port, worker.conf), worker.conf) + with Logging { - private val host = Utils.localHostName() - private val port = requestedPort.getOrElse( - worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) val timeout = AkkaUtils.askTimeout(worker.conf) + initialize() + /** Initialize all components of the server. */ - def start() { + def initialize() { val logPage = new LogPage(this) attachPage(logPage) attachPage(new IndexPage(this)) @@ -48,21 +51,13 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr)) worker.metricsSystem.getServletHandlers.foreach(attachHandler) } - - /** Bind to the HTTP server behind this web interface. */ - def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, worker.conf)) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Worker web UI", e) - System.exit(1) - } - } } private[spark] object WorkerWebUI { val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR + + def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { + requestedPort.getOrElse(conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index f2633dfa8abd7..2eda1aff5ac73 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -25,19 +25,19 @@ import org.apache.spark.ui.env.EnvironmentTab import org.apache.spark.ui.exec.ExecutorsTab import org.apache.spark.ui.jobs.JobProgressTab import org.apache.spark.ui.storage.BlockManagerTab -import org.apache.spark.util.Utils /** * Top level user interface for Spark. */ private[spark] class SparkUI( val sc: SparkContext, - conf: SparkConf, + val conf: SparkConf, val securityManager: SecurityManager, val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") - extends WebUI(securityManager, basePath) with Logging { + extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath) + with Logging { def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = @@ -46,21 +46,14 @@ private[spark] class SparkUI( // If SparkContext is not provided, assume the associated application is not live val live = sc != null - private val bindHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) - private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) - // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - listenerBus.addListener(storageStatusListener) - /** Set the app name for this UI. */ - def setAppName(name: String) { - appName = name - } + initialize() /** Initialize all components of the server. */ - def start() { + def initialize() { + listenerBus.addListener(storageStatusListener) attachTab(new JobProgressTab(this)) attachTab(new BlockManagerTab(this)) attachTab(new EnvironmentTab(this)) @@ -72,22 +65,14 @@ private[spark] class SparkUI( } } - /** Bind to the HTTP server behind this web interface. */ - def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark web UI", e) - System.exit(1) - } + /** Set the app name for this UI. */ + def setAppName(name: String) { + appName = name } - /** Attach a tab to this UI, along with its corresponding listener if it exists. */ - override def attachTab(tab: UITab) { - super.attachTab(tab) - tab.listener.foreach(listenerBus.addListener) + /** Register the given listener with the listener bus. */ + def registerListener(listener: SparkListener) { + listenerBus.addListener(listener) } /** Stop the server behind this web interface. Only valid after bind(). */ @@ -96,10 +81,14 @@ private[spark] class SparkUI( logInfo("Stopped Spark web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort + private[spark] def appUIAddress = "http://" + publicHostName + ":" + boundPort } private[spark] object SparkUI { val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + + def getUIPort(conf: SparkConf): Int = { + conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index fcda341ae5941..e7b756b2bd276 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -60,10 +60,9 @@ private[spark] object UIUtils { basePath: String, appName: String, title: String, - tabs: Seq[UITab], - activeTab: UITab, - refreshInterval: Option[Int] = None - ) : Seq[Node] = { + tabs: Seq[WebUITab], + activeTab: WebUITab, + refreshInterval: Option[Int] = None): Seq[Node] = { val header = tabs.map { tab =>
  • diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 126a7ff2f6080..655239089015c 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -25,8 +25,7 @@ import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler import org.json4s.JsonAST.{JNothing, JValue} -import org.apache.spark.SecurityManager -import org.apache.spark.scheduler.SparkListener +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils @@ -36,24 +35,31 @@ import org.apache.spark.util.Utils * Each WebUI represents a collection of tabs, each of which in turn represents a collection of * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. */ -private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: String = "") { - protected val tabs = ArrayBuffer[UITab]() +private[spark] abstract class WebUI( + securityManager: SecurityManager, + port: Int, + conf: SparkConf, + basePath: String = "") + extends Logging { + + protected val tabs = ArrayBuffer[WebUITab]() protected val handlers = ArrayBuffer[ServletContextHandler]() protected var serverInfo: Option[ServerInfo] = None + protected val localHostName = Utils.localHostName() + protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) + private val className = Utils.getFormattedClassName(this) - def getTabs: Seq[UITab] = tabs.toSeq + def getTabs: Seq[WebUITab] = tabs.toSeq def getHandlers: Seq[ServletContextHandler] = handlers.toSeq - def getListeners: Seq[SparkListener] = tabs.flatMap(_.listener) /** Attach a tab to this UI, along with all of its attached pages. */ - def attachTab(tab: UITab) { - tab.start() + def attachTab(tab: WebUITab) { tab.pages.foreach(attachPage) tabs += tab } /** Attach a page to this UI. */ - def attachPage(page: UIPage) { + def attachPage(page: WebUIPage) { val pagePath = "/" + page.prefix attachHandler(createServletHandler(pagePath, (request: HttpServletRequest) => page.render(request), securityManager, basePath)) @@ -86,13 +92,20 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: } /** Initialize all components of the server. */ - def start() - - /** - * Bind to the HTTP server behind this web interface. - * Overridden implementation should set serverInfo. - */ - def bind() + def initialize() + + /** Bind to the HTTP server behind this web interface. */ + def bind() { + assert(!serverInfo.isDefined, "Attempted to bind %s more than once!".format(className)) + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started %s at http://%s:%d".format(className, publicHostName, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind %s".format(className), e) + System.exit(1) + } + } /** Return the actual port to which this server is bound. Only valid after bind(). */ def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) @@ -100,39 +113,41 @@ private[spark] abstract class WebUI(securityManager: SecurityManager, basePath: /** Stop the server behind this web interface. Only valid after bind(). */ def stop() { assert(serverInfo.isDefined, - "Attempted to stop %s before binding to a server!".format(Utils.getFormattedClassName(this))) + "Attempted to stop %s before binding to a server!".format(className)) serverInfo.get.server.stop() } } /** - * A tab that represents a collection of pages and a unit of listening for Spark events. - * Associating each tab with a listener is arbitrary and need not be the case. + * A tab that represents a collection of pages. */ -private[spark] abstract class UITab(val prefix: String) { - val pages = ArrayBuffer[UIPage]() - var listener: Option[SparkListener] = None - var name = prefix.capitalize +private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { + val pages = ArrayBuffer[WebUIPage]() + val name = prefix.capitalize /** Attach a page to this tab. This prepends the page's prefix with the tab's own prefix. */ - def attachPage(page: UIPage) { + def attachPage(page: WebUIPage) { page.prefix = (prefix + "/" + page.prefix).stripSuffix("/") pages += page } - /** Initialize listener and attach pages. */ - def start() + /** Initialize this tab and attach all relevant pages. */ + def initialize() + + /** Get a list of header tabs from the parent UI. */ + def headerTabs: Seq[WebUITab] = parent.getTabs } /** * A page that represents the leaf node in the UI hierarchy. * + * The direct parent of a WebUIPage is not specified as it can be either a WebUI or a WebUITab. * If includeJson is true, the parent WebUI (direct or indirect) creates handlers for both the * HTML and the JSON content, rather than just the former. */ -private[spark] abstract class UIPage(var prefix: String, val includeJson: Boolean = false) { +private[spark] abstract class WebUIPage(var prefix: String, val includeJson: Boolean = false) { def render(request: HttpServletRequest): Seq[Node] = Seq[Node]() def renderJson(request: HttpServletRequest): JValue = JNothing } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index 6a2304f1ad42f..0f1ea7fa8d44d 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -20,21 +20,17 @@ package org.apache.spark.ui.env import org.apache.spark.scheduler._ import org.apache.spark.ui._ -private[ui] class EnvironmentTab(parent: SparkUI) extends UITab("environment") { +private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "environment") { val appName = parent.appName val basePath = parent.basePath + val listener = new EnvironmentListener - def start() { - listener = Some(new EnvironmentListener) - attachPage(new IndexPage(this)) - } + initialize() - def environmentListener: EnvironmentListener = { - assert(listener.isDefined, "EnvironmentTab has not started yet!") - listener.get.asInstanceOf[EnvironmentListener] + def initialize() { + attachPage(new IndexPage(this)) + parent.registerListener(listener) } - - def headerTabs: Seq[UITab] = parent.getTabs } /** diff --git a/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala index bde672909bbcc..55a19774ed02d 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala @@ -21,12 +21,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIUtils, UIPage} +import org.apache.spark.ui.{UIUtils, WebUIPage} -private[ui] class IndexPage(parent: EnvironmentTab) extends UIPage("") { +private[ui] class IndexPage(parent: EnvironmentTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath - private val listener = parent.environmentListener + private val listener = parent.listener override def render(request: HttpServletRequest): Seq[Node] = { val runtimeInformationTable = UIUtils.listingTable( diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index c1f5ca856ffe1..843db7c8d956d 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -22,23 +22,19 @@ import scala.collection.mutable.HashMap import org.apache.spark.ExceptionFailure import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.{SparkUI, UITab} +import org.apache.spark.ui.{SparkUI, WebUITab} -private[ui] class ExecutorsTab(parent: SparkUI) extends UITab("executors") { +private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "executors") { val appName = parent.appName val basePath = parent.basePath + val listener = new ExecutorsListener(parent.storageStatusListener) - def start() { - listener = Some(new ExecutorsListener(parent.storageStatusListener)) - attachPage(new IndexPage(this)) - } + initialize() - def executorsListener: ExecutorsListener = { - assert(listener.isDefined, "ExecutorsTab has not started yet!") - listener.get.asInstanceOf[ExecutorsListener] + def initialize() { + attachPage(new IndexPage(this)) + parent.registerListener(listener) } - - def headerTabs: Seq[UITab] = parent.getTabs } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala index bc6a822b080c3..83c89c2fbca3e 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala @@ -21,13 +21,13 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[ui] class IndexPage(parent: ExecutorsTab) extends UIPage("") { +private[ui] class IndexPage(parent: ExecutorsTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath - private val listener = parent.executorsListener + private val listener = parent.listener override def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 31173e48d7a1e..c83e196c9c156 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.util.Utils /** Page showing executor summary */ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { - private lazy val listener = parent.jobProgressListener + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 2b54603af104e..f217965ea2053 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -22,15 +22,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class IndexPage(parent: JobProgressTab) extends UIPage("") { +private[ui] class IndexPage(parent: JobProgressTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.jobProgressListener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler override def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 5167e20ea3d7d..18559f732d2a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -222,12 +222,10 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { - val schedulingModeName = - environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") - schedulingMode = schedulingModeName match { - case Some(name) => Some(SchedulingMode.withName(name)) - case None => None - } + environmentUpdate + .environmentDetails("Spark Properties").toMap + .get("spark.scheduler.mode") + .map(SchedulingMode.withName) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 93d26f7dd3632..7fe06b39346f5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -19,29 +19,25 @@ package org.apache.spark.ui.jobs import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, UITab} +import org.apache.spark.ui.{SparkUI, WebUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressTab(parent: SparkUI) extends UITab("stages") { +private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stages") { val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc + val conf = if (live) sc.conf else new SparkConf + val listener = new JobProgressListener(conf) - def start() { - val conf = if (live) sc.conf else new SparkConf - listener = Some(new JobProgressListener(conf)) + initialize() + + def initialize() { attachPage(new IndexPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) + parent.registerListener(listener) } - def jobProgressListener: JobProgressListener = { - assert(listener.isDefined, "JobProgressTab has not started yet!") - listener.get.asInstanceOf[JobProgressListener] - } - - def isFairScheduler = jobProgressListener.schedulingMode.exists(_ == SchedulingMode.FAIR) - - def headerTabs: Seq[UITab] = parent.getTabs + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 7fffe2affb0f2..228bfb2881c53 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -22,15 +22,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressTab) extends UIPage("pool") { +private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.jobProgressListener + private val listener = parent.listener override def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index bb7a9c14f7761..f4b68f241966d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -26,7 +26,7 @@ import org.apache.spark.ui.UIUtils /** Table showing list of pools */ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { private val basePath = parent.basePath - private lazy val listener = parent.jobProgressListener + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 372210919cd91..71eda45d253e1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,14 +22,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressTab) extends UIPage("stage") { +private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.jobProgressListener + private val listener = parent.listener override def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index d918feafd97d0..5cc1fcd10a08d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressTab) { private val basePath = parent.basePath - private lazy val listener = parent.jobProgressListener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala index ac83f71ed31de..492c223625e6b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala @@ -24,22 +24,18 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class BlockManagerTab(parent: SparkUI) extends UITab("storage") { +private[ui] class BlockManagerTab(parent: SparkUI) extends WebUITab(parent, "storage") { val appName = parent.appName val basePath = parent.basePath + val listener = new BlockManagerListener(parent.storageStatusListener) - def start() { - listener = Some(new BlockManagerListener(parent.storageStatusListener)) + initialize() + + def initialize() { attachPage(new IndexPage(this)) attachPage(new RddPage(this)) + parent.registerListener(listener) } - - def blockManagerListener: BlockManagerListener = { - assert(listener.isDefined, "BlockManagerTab has not started yet!") - listener.get.asInstanceOf[BlockManagerListener] - } - - def headerTabs: Seq[UITab] = parent.getTabs } /** diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index cb1b0dc7574f8..054369bc4730c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -22,14 +22,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class IndexPage(parent: BlockManagerTab) extends UIPage("") { +private[ui] class IndexPage(parent: BlockManagerTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.blockManagerListener + private val listener = parent.listener override def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index a65ba0a020bcd..5eaf41c985ecf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -22,14 +22,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.{UIPage, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RddPage(parent: BlockManagerTab) extends UIPage("rdd") { +private[ui] class RddPage(parent: BlockManagerTab) extends WebUIPage("rdd") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.blockManagerListener + private val listener = parent.listener override def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 906d4067a14eb..cf4aca2ed8b62 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -158,7 +158,6 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter private[streaming] val ui = new StreamingTab(this) - ui.start() /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 5cd900c2f88f0..80bd364af4e10 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Distribution /** Page for Spark Web UI that shows statistics of a streaming job */ private[ui] class StreamingPage(parent: StreamingTab) - extends UIPage("") with Logging { + extends WebUIPage("") with Logging { private val ssc = parent.ssc private val sc = ssc.sparkContext diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 1aaf7764b5ceb..44f230976427a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -18,15 +18,17 @@ package org.apache.spark.streaming.ui import org.apache.spark.streaming.StreamingContext -import org.apache.spark.ui.UITab +import org.apache.spark.ui.WebUITab import org.apache.spark.Logging /** Spark Web UI tab that shows statistics of a streaming job */ private[spark] class StreamingTab(val ssc: StreamingContext) - extends UITab("streaming") with Logging { + extends WebUITab(ssc.sc.ui, "streaming") with Logging { - val streamingPage = new StreamingPage(this) - ssc.sc.ui.attachTab(this) + initialize() - def start() { } + def initialize() { + attachPage(new StreamingPage(this)) + ssc.sc.ui.attachTab(this) + } } From 914b8ffd575b30670cc9829d25d6f277e7746841 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 10 Apr 2014 19:30:19 -0700 Subject: [PATCH 18/26] Moved utils functions to UIUtils. --- .../scala/org/apache/spark/ui/UIUtils.scala | 108 +++++++++++-- .../spark/streaming/StreamingContext.scala | 3 +- ...ala => StreamingJobProgressListener.scala} | 2 +- .../spark/streaming/ui/StreamingPage.scala | 153 +++--------------- .../spark/streaming/ui/StreamingTab.scala | 31 +++- .../org/apache/spark/streaming/UISuite.scala | 55 ++++++- 6 files changed, 197 insertions(+), 155 deletions(-) rename streaming/src/main/scala/org/apache/spark/streaming/ui/{StreamingProgressListener.scala => StreamingJobProgressListener.scala} (98%) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index fcda341ae5941..b210c8d852898 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -18,12 +18,13 @@ package org.apache.spark.ui import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Locale, Date} import scala.xml.Node +import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ -private[spark] object UIUtils { +private[spark] object UIUtils extends Logging { // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -49,6 +50,80 @@ private[spark] object UIUtils { "%.1f h".format(hours) } + /** Generate a verbose human-readable string representing a duration such as "5 second 35 ms" */ + def formatDurationVerbose(ms: Long): String = { + try { + val second = 1000L + val minute = 60 * second + val hour = 60 * minute + val day = 24 * hour + val week = 7 * day + val year = 365 * day + + def toString(num: Long, unit: String): String = { + if (num == 0) { + "" + } else if (num == 1) { + s"$num $unit" + } else { + s"$num ${unit}s" + } + } + + val millisecondsString = if (ms >= second && ms % second == 0) "" else s"${ms % second} ms" + val secondString = toString((ms % minute) / second, "second") + val minuteString = toString((ms % hour) / minute, "minute") + val hourString = toString((ms % day) / hour, "hour") + val dayString = toString((ms % week) / day, "day") + val weekString = toString((ms % year) / week, "week") + val yearString = toString(ms / year, "year") + + Seq( + second -> millisecondsString, + minute -> s"$secondString $millisecondsString", + hour -> s"$minuteString $secondString", + day -> s"$hourString $minuteString $secondString", + week -> s"$dayString $hourString $minuteString", + year -> s"$weekString $dayString $hourString" + ).foreach { case (durationLimit, durationString) => + if (ms < durationLimit) { + // if time is less than the limit (upto year) + return durationString + } + } + // if time is more than a year + return s"$yearString $weekString $dayString" + } catch { + case e: Exception => + logError("Error converting time to string", e) + // if there is some error, return blank string + return "" + } + } + + /** Generate a human-readable string representing a number (e.g. 100 K) */ + def formatNumber(records: Double): String = { + val trillion = 1e12 + val billion = 1e9 + val million = 1e6 + val thousand = 1e3 + + val (value, unit) = { + if (records >= 2*trillion) { + (records / trillion, " T") + } else if (records >= 2*billion) { + (records / billion, " B") + } else if (records >= 2*million) { + (records / million, " M") + } else if (records >= 2*thousand) { + (records / thousand, " K") + } else { + (records, "") + } + } + "%.1f%s".formatLocal(Locale.US, value, unit) + } + // Yarn has to go through a proxy so the base uri is provided and has to be on all links val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("") @@ -146,21 +221,36 @@ private[spark] object UIUtils { /** Returns an HTML table constructed by generating a row for each object in a sequence. */ def listingTable[T]( headers: Seq[String], - makeRow: T => Seq[Node], - rows: Seq[T], + generateDataRow: T => Seq[Node], + data: Seq[T], fixedWidth: Boolean = false): Seq[Node] = { - val colWidth = 100.toDouble / headers.size - val colWidthAttr = if (fixedWidth) colWidth + "%" else "" var tableClass = "table table-bordered table-striped table-condensed sortable" if (fixedWidth) { tableClass += " table-fixed" } - + val colWidth = 100.toDouble / headers.size + val colWidthAttr =if (fixedWidth) colWidth + "%" else "" + val headerRow: Seq[Node] = { + // if none of the headers have "\n" in them + if (headers.forall(!_.contains("\n"))) { + // represent header as simple text + headers.map(h => {h}) + } else { + // represent header text as list while respecting "\n" + headers.map { case h => + +
      + { h.split("\n").map { case t =>
    • {t}
    • } } +
    + + } + } + } - {headers.map(h => )} + {headerRow} - {rows.map(r => makeRow(r))} + {data.map(r => generateDataRow(r))}
    {h}
    } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 906d4067a14eb..ff5d0aaa3d0bd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -157,8 +157,7 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter - private[streaming] val ui = new StreamingTab(this) - ui.start() + private[streaming] val uiTab = new StreamingTab(this) /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala similarity index 98% rename from streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala rename to streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 32a4644e2a3e9..8921b99f53a23 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -28,7 +28,7 @@ import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution -private[ui] class StreamingProgressListener(ssc: StreamingContext) extends StreamingListener { +private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends StreamingListener { private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 5cd900c2f88f0..58960812e1205 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -17,30 +17,24 @@ package org.apache.spark.streaming.ui -import java.util.{Calendar, Locale} +import java.util.Calendar import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.Logging import org.apache.spark.ui._ +import org.apache.spark.ui.UIUtils._ import org.apache.spark.util.Distribution /** Page for Spark Web UI that shows statistics of a streaming job */ private[ui] class StreamingPage(parent: StreamingTab) extends UIPage("") with Logging { - private val ssc = parent.ssc - private val sc = ssc.sparkContext - private val sparkUI = sc.ui - private val listener = new StreamingProgressListener(ssc) - private val calendar = Calendar.getInstance() - private val startTime = calendar.getTime() + private val listener = parent.streamingListener + private val startTime = Calendar.getInstance().getTime() private val emptyCellTest = "-" - ssc.addStreamingListener(listener) - parent.attachPage(this) - /** Render the page */ override def render(request: HttpServletRequest): Seq[Node] = { val content = @@ -49,7 +43,7 @@ private[ui] class StreamingPage(parent: StreamingTab) generateNetworkStatsTable() ++ generateBatchStatsTable() UIUtils.headerSparkPage( - content, sparkUI.basePath, sc.appName, "Streaming", sparkUI.getTabs, parent, Some(5000)) + content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) } /** Generate basic stats of the streaming program */ @@ -60,13 +54,13 @@ private[ui] class StreamingPage(parent: StreamingTab) Started at: {startTime.toString}
  • - Time since start: {msDurationToString(timeSinceStart)} + Time since start: {formatDurationVerbose(timeSinceStart)}
  • Network receivers: {listener.numNetworkReceivers}
  • - Batch interval: {msDurationToString(listener.batchDuration)} + Batch interval: {formatDurationVerbose(listener.batchDuration)}
  • Processed batches: {listener.numTotalCompletedBatches} @@ -85,7 +79,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val headerRow = Seq( "Receiver", "Location", - "Records in last batch", + "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime())+ "]", "Minimum rate\n[records/sec]", "25th percentile rate\n[records/sec]", "Median rate\n[records/sec]", @@ -96,15 +90,15 @@ private[ui] class StreamingPage(parent: StreamingTab) val receiverInfo = listener.receiverInfo(receiverId) val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCellTest) - val receiverLastBatchRecords = numberToString(lastBatchReceivedRecord(receiverId)) + val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => - d.getQuantiles().map(r => numberToString(r.toLong)) + d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) }.getOrElse { Seq(emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest) } Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats } - Some(listingTable(headerRow, dataRows, fixedWidth = true)) + Some(listingTable(headerRow, dataRows)) } else { None } @@ -124,19 +118,19 @@ private[ui] class StreamingPage(parent: StreamingTab) val processingDelayQuantilesRow = { Seq( "Processing Time", - msDurationToString(lastCompletedBatch.flatMap(_.processingDelay)) + formatDurationOption(lastCompletedBatch.flatMap(_.processingDelay)) ) ++ getQuantiles(listener.processingDelayDistribution) } val schedulingDelayQuantilesRow = { Seq( "Scheduling Delay", - msDurationToString(lastCompletedBatch.flatMap(_.schedulingDelay)) + formatDurationOption(lastCompletedBatch.flatMap(_.schedulingDelay)) ) ++ getQuantiles(listener.schedulingDelayDistribution) } val totalDelayQuantilesRow = { Seq( "Total Delay", - msDurationToString(lastCompletedBatch.flatMap(_.totalDelay)) + formatDurationOption(lastCompletedBatch.flatMap(_.totalDelay)) ) ++ getQuantiles(listener.totalDelayDistribution) } val headerRow = Seq("Metric", "Last batch", "Minimum", "25th percentile", @@ -146,7 +140,7 @@ private[ui] class StreamingPage(parent: StreamingTab) schedulingDelayQuantilesRow, totalDelayQuantilesRow ) - Some(listingTable(headerRow, dataRows, fixedWidth = true)) + Some(listingTable(headerRow, dataRows)) } else { None } @@ -162,130 +156,25 @@ private[ui] class StreamingPage(parent: StreamingTab) content } - /** - * Returns a human-readable string representing a number - */ - private def numberToString(records: Double): String = { - val trillion = 1e12 - val billion = 1e9 - val million = 1e6 - val thousand = 1e3 - - val (value, unit) = { - if (records >= 2*trillion) { - (records / trillion, " T") - } else if (records >= 2*billion) { - (records / billion, " B") - } else if (records >= 2*million) { - (records / million, " M") - } else if (records >= 2*thousand) { - (records / thousand, " K") - } else { - (records, "") - } - } - "%.1f%s".formatLocal(Locale.US, value, unit) - } /** * Returns a human-readable string representing a duration such as "5 second 35 ms" */ - private def msDurationToString(ms: Long): String = { - try { - val second = 1000L - val minute = 60 * second - val hour = 60 * minute - val day = 24 * hour - val week = 7 * day - val year = 365 * day - - def toString(num: Long, unit: String): String = { - if (num == 0) { - "" - } else if (num == 1) { - s"$num $unit" - } else { - s"$num ${unit}s" - } - } - - val millisecondsString = if (ms >= second && ms % second == 0) "" else s"${ms % second} ms" - val secondString = toString((ms % minute) / second, "second") - val minuteString = toString((ms % hour) / minute, "minute") - val hourString = toString((ms % day) / hour, "hour") - val dayString = toString((ms % week) / day, "day") - val weekString = toString((ms % year) / week, "week") - val yearString = toString(ms / year, "year") - - Seq( - second -> millisecondsString, - minute -> s"$secondString $millisecondsString", - hour -> s"$minuteString $secondString", - day -> s"$hourString $minuteString $secondString", - week -> s"$dayString $hourString $minuteString", - year -> s"$weekString $dayString $hourString" - ).foreach { case (durationLimit, durationString) => - if (ms < durationLimit) { - // if time is less than the limit (upto year) - return durationString - } - } - // if time is more than a year - return s"$yearString $weekString $dayString" - } catch { - case e: Exception => - logError("Error converting time to string", e) - // if there is some error, return blank string - return "" - } - } - - /** - * Returns a human-readable string representing a duration such as "5 second 35 ms" - */ - private def msDurationToString(msOption: Option[Long]): String = { - msOption.map(msDurationToString).getOrElse(emptyCellTest) + private def formatDurationOption(msOption: Option[Long]): String = { + msOption.map(formatDurationVerbose).getOrElse(emptyCellTest) } /** Get quantiles for any time distribution */ private def getQuantiles(timeDistributionOption: Option[Distribution]) = { - timeDistributionOption.get.getQuantiles().map { ms => msDurationToString(ms.toLong) } + timeDistributionOption.get.getQuantiles().map { ms => formatDurationVerbose(ms.toLong) } } - /** Generate an HTML table constructed by generating a row for each object in a sequence. */ - def listingTable[T]( - headerRow: Seq[String], - dataRows: Seq[Seq[String]], - fixedWidth: Boolean = false - ): Seq[Node] = { - - val colWidth = 100.toDouble / headerRow.size - val colWidthAttr = if (fixedWidth) colWidth + "%" else "" - var tableClass = "table table-bordered table-striped table-condensed sortable" - if (fixedWidth) { - tableClass += " table-fixed" - } - - def generateHeaderRow(header: Seq[String]): Seq[Node] = { - headerRow.map { case h => - -
      - { h.split("\n").map { case t =>
    • {t}
    • } } -
    - - } - } - + /** Generate HTML table from string data */ + private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { def generateDataRow(data: Seq[String]): Seq[Node] = { {data.map(d => {d})} } - - - {generateHeaderRow(headerRow)} - - {dataRows.map(r => generateDataRow(r))} - -
    + UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 1aaf7764b5ceb..5a817b067e4fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -18,15 +18,34 @@ package org.apache.spark.streaming.ui import org.apache.spark.streaming.StreamingContext -import org.apache.spark.ui.UITab +import org.apache.spark.ui.{SparkUI, UITab} import org.apache.spark.Logging +import java.util.concurrent.atomic.AtomicInteger -/** Spark Web UI tab that shows statistics of a streaming job */ -private[spark] class StreamingTab(val ssc: StreamingContext) - extends UITab("streaming") with Logging { +/** Streaming tab in the Spark web UI */ +private[spark] class StreamingTab(ssc: StreamingContext) + extends UITab(StreamingTab.streamingTabName) with Logging { - val streamingPage = new StreamingPage(this) - ssc.sc.ui.attachTab(this) + val parent = ssc.sc.ui + val streamingListener = new StreamingJobProgressListener(ssc) + val basePath = parent.basePath + val appName = parent.appName + + ssc.addStreamingListener(streamingListener) + attachPage(new StreamingPage(this)) + parent.attachTab(this) + + def headerTabs = parent.getTabs def start() { } } + +object StreamingTab { + private val atomicInteger = new AtomicInteger(0) + + /** Generate the name of the streaming tab. For the first streaming tab it will be */ + def streamingTabName: String = { + val count = atomicInteger.getAndIncrement + if (count == 0) "streaming" else s"streaming-$count" + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 8f6e3ea9dce40..5bba5d9a39dd7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -17,15 +17,60 @@ package org.apache.spark.streaming -import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite} -import org.apache.spark.streaming.dstream.InputDStream import scala.reflect.ClassTag -import org.apache.spark.rdd.RDD import scala.util.Random +import scala.io.Source + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ +import org.scalatest.matchers.ShouldMatchers + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream.InputDStream + +class UISuite extends FunSuite with ShouldMatchers with BeforeAndAfterAll with BeforeAndAfter { + var sc: SparkContext = null + var ssc: StreamingContext = null + + override def beforeAll() { + val conf = new SparkConf().setMaster("local").setAppName(this.getClass.getSimpleName) + conf.set("spark.cleaner.ttl", "1800") + sc = new SparkContext(conf) + } + + override def afterAll() { + if (sc != null) sc.stop() + } + + before { + ssc = new StreamingContext(sc, Seconds(1)) + } + + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + } -class UISuite extends FunSuite with BeforeAndAfterAll { + test("streaming tab in spark UI") { + val ssc = new StreamingContext(sc, Seconds(1)) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val uiData = Source.fromURL( + ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString + assert(uiData.contains("streaming")) + } + } + + test("multiple streaming tabs") { + val ssc1 = new StreamingContext(sc, Seconds(1)) + val ssc2 = new StreamingContext(sc, Seconds(2)) + ssc1.uiTab.prefix should not be ssc2.uiTab.prefix + } - test("Testing") { + ignore("Testing") { runStreaming(1000000) } From f8e105390b634e2221f68081639fd658ba9a6f54 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 10 Apr 2014 20:22:27 -0700 Subject: [PATCH 19/26] Added Spark and Streaming UI unit tests. --- .../scala/org/apache/spark/ui/WebUI.scala | 3 - .../apache/spark/ui/env/EnvironmentTab.scala | 8 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 8 +- .../apache/spark/ui/jobs/JobProgressTab.scala | 12 +-- .../spark/ui/storage/BlockManagerTab.scala | 10 +-- .../scala/org/apache/spark/ui/UISuite.scala | 67 ++++++++++++++- .../spark/streaming/ui/StreamingTab.scala | 12 +-- .../org/apache/spark/streaming/UISuite.scala | 84 ++++--------------- 8 files changed, 94 insertions(+), 110 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 655239089015c..592a440219d65 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -132,9 +132,6 @@ private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { pages += page } - /** Initialize this tab and attach all relevant pages. */ - def initialize() - /** Get a list of header tabs from the parent UI. */ def headerTabs: Seq[WebUITab] = parent.getTabs } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index 0f1ea7fa8d44d..7797057fa1aa9 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -25,12 +25,8 @@ private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "envi val basePath = parent.basePath val listener = new EnvironmentListener - initialize() - - def initialize() { - attachPage(new IndexPage(this)) - parent.registerListener(listener) - } + attachPage(new IndexPage(this)) + parent.registerListener(listener) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 843db7c8d956d..e9ec18a3e74af 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -29,12 +29,8 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "execut val basePath = parent.basePath val listener = new ExecutorsListener(parent.storageStatusListener) - initialize() - - def initialize() { - attachPage(new IndexPage(this)) - parent.registerListener(listener) - } + attachPage(new IndexPage(this)) + parent.registerListener(listener) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 7fe06b39346f5..da9de035f89f1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -30,14 +30,10 @@ private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stag val conf = if (live) sc.conf else new SparkConf val listener = new JobProgressListener(conf) - initialize() - - def initialize() { - attachPage(new IndexPage(this)) - attachPage(new StagePage(this)) - attachPage(new PoolPage(this)) - parent.registerListener(listener) - } + attachPage(new IndexPage(this)) + attachPage(new StagePage(this)) + attachPage(new PoolPage(this)) + parent.registerListener(listener) def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala index 492c223625e6b..05b6ef2cf0f4e 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala @@ -29,13 +29,9 @@ private[ui] class BlockManagerTab(parent: SparkUI) extends WebUITab(parent, "sto val basePath = parent.basePath val listener = new BlockManagerListener(parent.storageStatusListener) - initialize() - - def initialize() { - attachPage(new IndexPage(this)) - attachPage(new RddPage(this)) - parent.registerListener(listener) - } + attachPage(new IndexPage(this)) + attachPage(new RddPage(this)) + parent.registerListener(listener) } /** diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 45c322427930d..0ca5c0cd237cc 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,16 +18,81 @@ package org.apache.spark.ui import java.net.ServerSocket +import javax.servlet.http.HttpServletRequest +import scala.io.Source import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.LocalSparkContext._ +import scala.xml.Node class UISuite extends FunSuite { + + test("basic ui visibility") { + withSpark(new SparkContext("local", "test")) { sc => + // test if the ui is visible, and all the expected tabs are visible + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + } + } + + test("visibility at localhost:4040") { + withSpark(new SparkContext("local", "test")) { sc => + // test if visible from http://localhost:4040 + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL("http://localhost:4040").mkString + assert(html.toLowerCase.contains("stages")) + } + } + } + + test("attaching a new tab") { + withSpark(new SparkContext("local", "test")) { sc => + val sparkUI = sc.ui + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + override def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + + // check whether new page exists + assert(html.toLowerCase.contains("foo")) + + // check whether other pages still exist + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString + // check whether new page exists + assert(html.contains("magic")) + } + } + } + test("jetty port increases under contention") { val startPort = 4040 val server = new Server(startPort) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index be8e652899ebe..51448d15c6516 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -17,8 +17,6 @@ package org.apache.spark.streaming.ui -import java.util.concurrent.atomic.AtomicInteger - import org.apache.spark.Logging import org.apache.spark.streaming.StreamingContext import org.apache.spark.ui.WebUITab @@ -32,11 +30,7 @@ private[spark] class StreamingTab(ssc: StreamingContext) val basePath = parent.basePath val listener = new StreamingJobProgressListener(ssc) - initialize() - - def initialize() { - ssc.addStreamingListener(listener) - attachPage(new StreamingPage(this)) - parent.attachTab(this) - } + ssc.addStreamingListener(listener) + attachPage(new StreamingPage(this)) + parent.attachTab(this) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 502896d76c494..35538ec188f67 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -17,86 +17,30 @@ package org.apache.spark.streaming -import scala.reflect.ClassTag -import scala.util.Random import scala.io.Source -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} +import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.dstream.InputDStream - -class UISuite extends FunSuite with ShouldMatchers with BeforeAndAfterAll with BeforeAndAfter { - var sc: SparkContext = null - var ssc: StreamingContext = null - - override def beforeAll() { - val conf = new SparkConf().setMaster("local").setAppName(this.getClass.getSimpleName) - conf.set("spark.cleaner.ttl", "1800") - sc = new SparkContext(conf) - } - - override def afterAll() { - if (sc != null) sc.stop() - } - - before { - ssc = new StreamingContext(sc, Seconds(1)) - } - - after { - if (ssc != null) { - ssc.stop() - ssc = null - } - } +class UISuite extends FunSuite { test("streaming tab in spark UI") { - val ssc = new StreamingContext(sc, Seconds(1)) + val ssc = new StreamingContext("local", "test", Seconds(1)) eventually(timeout(10 seconds), interval(50 milliseconds)) { - val uiData = Source.fromURL( - ssc.sparkContext.ui.appUIAddress).mkString - assert(!uiData.contains("random data that should not be present")) - assert(uiData.contains("streaming")) + val html = Source.fromURL(ssc.sparkContext.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + // test if streaming tab exist + assert(html.toLowerCase.contains("streaming")) + // test if other Spark tabs still exist + assert(html.toLowerCase.contains("stages")) } - } - ignore("Testing") { - runStreaming(1000000) - } - - def runStreaming(duration: Long) { - val ssc1 = new StreamingContext(sc, Seconds(1)) - val servers1 = (1 to 3).map { i => new TestServer(10000 + i) } - - val inputStream1 = ssc1.union(servers1.map(server => ssc1.socketTextStream("localhost", server.port))) - inputStream1.count.print - - ssc1.start() - servers1.foreach(_.start()) - - val startTime = System.currentTimeMillis() - while (System.currentTimeMillis() - startTime < duration) { - servers1.map(_.send(Random.nextString(10) + "\n")) - //Thread.sleep(1) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL( + ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString + assert(html.toLowerCase.contains("batch")) + assert(html.toLowerCase.contains("network")) } - ssc1.stop() - servers1.foreach(_.stop()) } } - -class FunctionBasedInputDStream[T: ClassTag]( - ssc_ : StreamingContext, - function: (StreamingContext, Time) => Option[RDD[T]] - ) extends InputDStream[T](ssc_) { - - def start(): Unit = {} - - def stop(): Unit = {} - - def compute(validTime: Time): Option[RDD[T]] = function(ssc, validTime) -} \ No newline at end of file From aa396d408e441933fb84500f2e53875c43d7a6eb Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 10 Apr 2014 20:33:45 -0700 Subject: [PATCH 20/26] Rename tabs and pages (No more IndexPage.scala) Previously there were 7 different IndexPage.scala's in different packages. --- .../deploy/history/{IndexPage.scala => HistoryPage.scala} | 0 .../deploy/master/ui/{IndexPage.scala => MasterPage.scala} | 0 .../deploy/worker/ui/{IndexPage.scala => WorkerPage.scala} | 0 core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 4 ++-- .../spark/ui/env/{IndexPage.scala => EnvironmentPage.scala} | 0 .../spark/ui/exec/{IndexPage.scala => ExecutorsPage.scala} | 0 .../ui/jobs/{IndexPage.scala => JobProgressPage.scala} | 0 .../main/scala/org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../spark/ui/storage/{IndexPage.scala => StoragePage.scala} | 2 +- .../ui/storage/{BlockManagerTab.scala => StorageTab.scala} | 6 +++--- 10 files changed, 7 insertions(+), 7 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/history/{IndexPage.scala => HistoryPage.scala} (100%) rename core/src/main/scala/org/apache/spark/deploy/master/ui/{IndexPage.scala => MasterPage.scala} (100%) rename core/src/main/scala/org/apache/spark/deploy/worker/ui/{IndexPage.scala => WorkerPage.scala} (100%) rename core/src/main/scala/org/apache/spark/ui/env/{IndexPage.scala => EnvironmentPage.scala} (100%) rename core/src/main/scala/org/apache/spark/ui/exec/{IndexPage.scala => ExecutorsPage.scala} (100%) rename core/src/main/scala/org/apache/spark/ui/jobs/{IndexPage.scala => JobProgressPage.scala} (100%) rename core/src/main/scala/org/apache/spark/ui/storage/{IndexPage.scala => StoragePage.scala} (96%) rename core/src/main/scala/org/apache/spark/ui/storage/{BlockManagerTab.scala => StorageTab.scala} (92%) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 2eda1aff5ac73..38ef4dfe501b4 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -24,7 +24,7 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentTab import org.apache.spark.ui.exec.ExecutorsTab import org.apache.spark.ui.jobs.JobProgressTab -import org.apache.spark.ui.storage.BlockManagerTab +import org.apache.spark.ui.storage.StorageTab /** * Top level user interface for Spark. @@ -55,7 +55,7 @@ private[spark] class SparkUI( def initialize() { listenerBus.addListener(storageStatusListener) attachTab(new JobProgressTab(this)) - attachTab(new BlockManagerTab(this)) + attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) diff --git a/core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/ui/env/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala diff --git a/core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/ui/exec/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 5eaf41c985ecf..e04253aa0336c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -26,7 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RddPage(parent: BlockManagerTab) extends WebUIPage("rdd") { +private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { private val appName = parent.appName private val basePath = parent.basePath private val listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 054369bc4730c..f9e6738a1d7e4 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -26,7 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class IndexPage(parent: BlockManagerTab) extends WebUIPage("") { +private[ui] class IndexPage(parent: StorageTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath private val listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala similarity index 92% rename from core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 05b6ef2cf0f4e..0886950c3f8e6 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -24,10 +24,10 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class BlockManagerTab(parent: SparkUI) extends WebUITab(parent, "storage") { +private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") { val appName = parent.appName val basePath = parent.basePath - val listener = new BlockManagerListener(parent.storageStatusListener) + val listener = new StorageListener(parent.storageStatusListener) attachPage(new IndexPage(this)) attachPage(new RddPage(this)) @@ -37,7 +37,7 @@ private[ui] class BlockManagerTab(parent: SparkUI) extends WebUITab(parent, "sto /** * A SparkListener that prepares information to be displayed on the BlockManagerUI */ -private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) +private[ui] class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() From 2fc09c86f052daedf8b3de5e024e55ee81c6f64c Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 10 Apr 2014 20:41:59 -0700 Subject: [PATCH 21/26] Added binary check exclusions --- project/MimaBuild.scala | 8 +++++++- .../org/apache/spark/streaming/ui/StreamingPage.scala | 2 +- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 5ea4817bfde18..4bfa9bf95ad78 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -60,6 +60,7 @@ object MimaBuild { Seq( excludePackage("org.apache.spark.api.java"), excludePackage("org.apache.spark.streaming.api.java"), + excludePackage("org.apache.spark.streaming.scheduler"), excludePackage("org.apache.spark.mllib") ) ++ excludeSparkClass("rdd.ClassTags") ++ @@ -70,7 +71,12 @@ object MimaBuild { excludeSparkClass("mllib.regression.LassoWithSGD") ++ excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ excludeSparkClass("streaming.dstream.NetworkReceiver") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") + excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ + excludeSparkClass("streaming.dstream.ReportError") ++ + excludeSparkClass("org.apache.spark.streaming.dstream.ReportBlock") ++ + excludeSparkClass("org.apache.spark.streaming.dstream.DStream") case _ => Seq() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 4089285d7c287..6fdfd8d05dcbb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -79,7 +79,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val headerRow = Seq( "Receiver", "Location", - "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime())+ "]", + "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", "Minimum rate\n[records/sec]", "25th percentile rate\n[records/sec]", "Median rate\n[records/sec]", From 90feb8d111c05c5734388b85f403bf45713b6634 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 10 Apr 2014 23:56:16 -0700 Subject: [PATCH 22/26] Address Patrick's comments --- .../spark/deploy/history/HistoryPage.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 8 ++++---- .../org/apache/spark/deploy/master/Master.scala | 4 ++-- .../spark/deploy/master/ui/ApplicationPage.scala | 5 ++--- .../spark/deploy/master/ui/MasterPage.scala | 4 ++-- .../spark/deploy/master/ui/MasterWebUI.scala | 4 ++-- .../apache/spark/deploy/worker/ui/LogPage.scala | 2 +- .../spark/deploy/worker/ui/WorkerPage.scala | 4 ++-- .../main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../main/scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../main/scala/org/apache/spark/ui/WebUI.scala | 16 ++++++++-------- .../apache/spark/ui/env/EnvironmentPage.scala | 2 +- .../org/apache/spark/ui/exec/ExecutorsPage.scala | 2 +- .../apache/spark/ui/jobs/JobProgressPage.scala | 2 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../apache/spark/ui/storage/StoragePage.scala | 2 +- .../test/scala/org/apache/spark/ui/UISuite.scala | 2 +- .../spark/streaming/ui/StreamingPage.scala | 2 +- 20 files changed, 35 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 69a6baa4aaeab..7f7372746f92e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} private[spark] class IndexPage(parent: HistoryServer) extends WebUIPage("") { - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } val appTable = UIUtils.listingTable(appHeader, appRow, appRows) val content = diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index df3c394bacfa9..f495edcf1c6af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -131,7 +131,7 @@ class HistoryServer( // Remove any applications that should no longer be retained appIdToInfo.foreach { case (appId, info) => if (!retainedAppIds.contains(appId)) { - detachUI(info.ui) + detachSparkUI(info.ui) appIdToInfo.remove(appId) } } @@ -173,7 +173,7 @@ class HistoryServer( // Do not call ui.bind() to avoid creating a new server for each application replayBus.replay() if (appListener.applicationStarted) { - attachUI(ui) + attachSparkUI(ui) val appName = appListener.appName val sparkUser = appListener.sparkUser val startTime = appListener.startTime @@ -193,13 +193,13 @@ class HistoryServer( } /** Attach a reconstructed UI to this server. Only valid after bind(). */ - private def attachUI(ui: SparkUI) { + private def attachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") ui.getHandlers.foreach(attachHandler) } /** Detach a reconstructed UI from this server. Only valid after bind(). */ - private def detachUI(ui: SparkUI) { + private def detachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") ui.getHandlers.foreach(detachHandler) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 076bb92bf2a10..6c58e741df001 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -625,7 +625,7 @@ private[spark] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { - appIdToUI.remove(a.id).foreach { ui => webUi.detachUI(ui) } + appIdToUI.remove(a.id).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) @@ -672,7 +672,7 @@ private[spark] class Master( replayBus.replay() app.desc.appUiUrl = ui.basePath appIdToUI(app.id) = ui - webUi.attachUI(ui) + webUi.attachSparkUI(ui) return true } catch { case t: Throwable => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index d8c3321ea51ec..b5cd4d2ea963f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -31,8 +31,7 @@ import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) - extends WebUIPage("app", includeJson = true) { +private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { private val master = parent.masterActorRef private val timeout = parent.timeout @@ -49,7 +48,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) } /** Executor details for a particular application */ - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 3d2ad04110b77..30c2e4b1563d8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: MasterWebUI) extends WebUIPage("", includeJson = true) { +private[spark] class IndexPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterActorRef private val timeout = parent.timeout @@ -42,7 +42,7 @@ private[spark] class IndexPage(parent: MasterWebUI) extends WebUIPage("", includ } /** Index view listing applications and executors */ - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 965f7a0fac9e2..939cf2ea9a678 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -45,13 +45,13 @@ class MasterWebUI(val master: Master, requestedPort: Int) } /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachUI(ui: SparkUI) { + def attachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") ui.getHandlers.foreach(attachHandler) } /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachUI(ui: SparkUI) { + def detachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") ui.getHandlers.foreach(detachHandler) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 8f6b36faf85ee..fec1207948628 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -56,7 +56,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { pre + Utils.offsetBytes(path, startByte, endByte) } - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val defaultBytes = 100 * 1024 val appId = Option(request.getParameter("appId")) val executorId = Option(request.getParameter("executorId")) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 42ef8ed703779..15b79872bc556 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: WorkerWebUI) extends WebUIPage("", includeJson = true) { +private[spark] class IndexPage(parent: WorkerWebUI) extends WebUIPage("") { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout @@ -42,7 +42,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) extends WebUIPage("", includ JsonProtocol.writeWorkerState(workerState) } - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, timeout) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 38ef4dfe501b4..bca4c3c42d27f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -27,7 +27,7 @@ import org.apache.spark.ui.jobs.JobProgressTab import org.apache.spark.ui.storage.StorageTab /** - * Top level user interface for Spark. + * Top level user interface for a Spark application. */ private[spark] class SparkUI( val sc: SparkContext, diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index d26109d06c186..ecdb9c6385657 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -229,7 +229,7 @@ private[spark] object UIUtils extends Logging { tableClass += " table-fixed" } val colWidth = 100.toDouble / headers.size - val colWidthAttr =if (fixedWidth) colWidth + "%" else "" + val colWidthAttr = if (fixedWidth) colWidth + "%" else "" val headerRow: Seq[Node] = { // if none of the headers have "\n" in them if (headers.forall(!_.contains("\n"))) { diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 592a440219d65..b08f308fda1dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -63,10 +63,8 @@ private[spark] abstract class WebUI( val pagePath = "/" + page.prefix attachHandler(createServletHandler(pagePath, (request: HttpServletRequest) => page.render(request), securityManager, basePath)) - if (page.includeJson) { - attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", - (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) - } + attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) } /** Attach a handler to this UI. */ @@ -121,6 +119,7 @@ private[spark] abstract class WebUI( /** * A tab that represents a collection of pages. + * The prefix is appended to the parent address to form a full path, and must not contain slashes. */ private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { val pages = ArrayBuffer[WebUIPage]() @@ -141,10 +140,11 @@ private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { * A page that represents the leaf node in the UI hierarchy. * * The direct parent of a WebUIPage is not specified as it can be either a WebUI or a WebUITab. - * If includeJson is true, the parent WebUI (direct or indirect) creates handlers for both the - * HTML and the JSON content, rather than just the former. + * If the parent is a WebUI, the prefix is appended to the parent's address to form a full path. + * Else, if the parent is a WebUITab, the prefix is appended to the super prefix of the parent + * to form a relative path. The prefix must not contain slashes. */ -private[spark] abstract class WebUIPage(var prefix: String, val includeJson: Boolean = false) { - def render(request: HttpServletRequest): Seq[Node] = Seq[Node]() +private[spark] abstract class WebUIPage(var prefix: String) { + def render(request: HttpServletRequest): Seq[Node] def renderJson(request: HttpServletRequest): JValue = JNothing } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 55a19774ed02d..70578c3eb87c8 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -28,7 +28,7 @@ private[ui] class IndexPage(parent: EnvironmentTab) extends WebUIPage("") { private val basePath = parent.basePath private val listener = parent.listener - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val runtimeInformationTable = UIUtils.listingTable( propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable( diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 83c89c2fbca3e..56c3887923758 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -29,7 +29,7 @@ private[ui] class IndexPage(parent: ExecutorsTab) extends WebUIPage("") { private val basePath = parent.basePath private val listener = parent.listener - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index f217965ea2053..12c82796349c9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -33,7 +33,7 @@ private[ui] class IndexPage(parent: JobProgressTab) extends WebUIPage("") { private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 228bfb2881c53..fd83d37583967 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -32,7 +32,7 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { private val sc = parent.sc private val listener = parent.listener - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 71eda45d253e1..4bce472036f7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -31,7 +31,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { private val basePath = parent.basePath private val listener = parent.listener - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index e04253aa0336c..d07f1c9b20fcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -31,7 +31,7 @@ private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { private val basePath = parent.basePath private val listener = parent.listener - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index f9e6738a1d7e4..c5cfee777aab5 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -31,7 +31,7 @@ private[ui] class IndexPage(parent: StorageTab) extends WebUIPage("") { private val basePath = parent.basePath private val listener = parent.listener - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 0ca5c0cd237cc..0332a2a0539ee 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -65,7 +65,7 @@ class UISuite extends FunSuite { val newTab = new WebUITab(sparkUI, "foo") { attachPage(new WebUIPage("") { - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { "html magic" } }) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 6fdfd8d05dcbb..1f3fcacf66695 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -36,7 +36,7 @@ private[ui] class StreamingPage(parent: StreamingTab) private val emptyCellTest = "-" /** Render the page */ - override def render(request: HttpServletRequest): Seq[Node] = { + def render(request: HttpServletRequest): Seq[Node] = { val content = generateBasicStats() ++

    Statistics over last {listener.completedBatches.size} processed batches

    ++ From 125a05485c918938a4d3830da974e9c639df17f9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 11 Apr 2014 10:09:08 -0700 Subject: [PATCH 23/26] Disable serving static resources with gzip --- core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index e1a1f209c9282..3609073cecf7c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -119,6 +119,7 @@ private[spark] object JettyUtils extends Logging { /** Create a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { val contextHandler = new ServletContextHandler + contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) Option(getClass.getClassLoader.getResource(resourceBase)) match { From e038b4b076aba23e56915b6184908b00476d4f0f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 11 Apr 2014 12:03:43 -0700 Subject: [PATCH 24/26] Addressed Patrick's comments. --- project/MimaBuild.scala | 4 ++-- .../streaming/ui/StreamingJobProgressListener.scala | 10 +++++----- .../apache/spark/streaming/ui/StreamingPage.scala | 12 ++++++------ 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 4bfa9bf95ad78..9cb31d70444ff 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -75,8 +75,8 @@ object MimaBuild { excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ excludeSparkClass("streaming.dstream.ReportError") ++ - excludeSparkClass("org.apache.spark.streaming.dstream.ReportBlock") ++ - excludeSparkClass("org.apache.spark.streaming.dstream.DStream") + excludeSparkClass("streaming.dstream.ReportBlock") ++ + excludeSparkClass("streaming.dstream.DStream") case _ => Seq() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 8921b99f53a23..8b025b09ed34d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -33,7 +33,7 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] private val completedaBatchInfos = new Queue[BatchInfo] - private val batchInfoLimit = ssc.conf.getInt("spark.steaming.ui.maxBatches", 100) + private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) private var totalCompletedBatches = 0L private val receiverInfos = new HashMap[Int, ReceiverInfo] @@ -82,7 +82,7 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St runningBatchInfos.values.toSeq } - def completedBatches: Seq[BatchInfo] = synchronized { + def retainedCompletedBatches: Seq[BatchInfo] = synchronized { completedaBatchInfos.toSeq } @@ -99,7 +99,7 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St } def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { - val latestBatchInfos = allBatches.reverse.take(batchInfoLimit) + val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) (0 until numNetworkReceivers).map { receiverId => val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => @@ -134,10 +134,10 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St } def lastReceivedBatch: Option[BatchInfo] = { - allBatches.lastOption + retainedBatches.lastOption } - private def allBatches: Seq[BatchInfo] = synchronized { + private def retainedBatches: Seq[BatchInfo] = synchronized { (waitingBatchInfos.values.toSeq ++ runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time.ordering) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 6fdfd8d05dcbb..290efaef51481 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -33,13 +33,13 @@ private[ui] class StreamingPage(parent: StreamingTab) private val listener = parent.listener private val startTime = Calendar.getInstance().getTime() - private val emptyCellTest = "-" + private val emptyCell = "-" /** Render the page */ override def render(request: HttpServletRequest): Seq[Node] = { val content = generateBasicStats() ++ -

    Statistics over last {listener.completedBatches.size} processed batches

    ++ +

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ generateNetworkStatsTable() ++ generateBatchStatsTable() UIUtils.headerSparkPage( @@ -89,12 +89,12 @@ private[ui] class StreamingPage(parent: StreamingTab) val dataRows = (0 until listener.numNetworkReceivers).map { receiverId => val receiverInfo = listener.receiverInfo(receiverId) val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") - val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCellTest) + val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) }.getOrElse { - Seq(emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest) + Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) } Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats } @@ -112,7 +112,7 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Generate stats of batch jobs of the streaming program */ private def generateBatchStatsTable(): Seq[Node] = { - val numBatches = listener.completedBatches.size + val numBatches = listener.retainedCompletedBatches.size val lastCompletedBatch = listener.lastCompletedBatch val table = if (numBatches > 0) { val processingDelayQuantilesRow = { @@ -161,7 +161,7 @@ private[ui] class StreamingPage(parent: StreamingTab) * Returns a human-readable string representing a duration such as "5 second 35 ms" */ private def formatDurationOption(msOption: Option[Long]): String = { - msOption.map(formatDurationVerbose).getOrElse(emptyCellTest) + msOption.map(formatDurationVerbose).getOrElse(emptyCell) } /** Get quantiles for any time distribution */ From f4f4cbec116678e6c13311b4ebe208999a52738a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 11 Apr 2014 12:19:21 -0700 Subject: [PATCH 25/26] More minor fixes. --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../scala/org/apache/spark/streaming/ui/StreamingPage.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index ecdb9c6385657..e9b83922a4b1a 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -158,7 +158,7 @@ private[spark] object UIUtils extends Logging { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 393680b66e46d..6607437db560a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -38,8 +38,8 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { val content = - generateBasicStats() ++ -

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ + generateBasicStats() ++

    ++ +

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ generateNetworkStatsTable() ++ generateBatchStatsTable() UIUtils.headerSparkPage( From 642dd8806e565a0802b89ac6dba79e8116401b3d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 11 Apr 2014 22:41:48 -0700 Subject: [PATCH 26/26] Merge SparkUISuite.scala into UISuite.scala --- .../scala/org/apache/spark/SparkUISuite.scala | 35 ------------------- .../scala/org/apache/spark/ui/UISuite.scala | 14 ++++++++ 2 files changed, 14 insertions(+), 35 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/SparkUISuite.scala diff --git a/core/src/test/scala/org/apache/spark/SparkUISuite.scala b/core/src/test/scala/org/apache/spark/SparkUISuite.scala deleted file mode 100644 index d0d119c15081d..0000000000000 --- a/core/src/test/scala/org/apache/spark/SparkUISuite.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import java.net.URI - -import org.scalatest.FunSuite - -class SparkUISuite extends FunSuite with SharedSparkContext { - - test("verify appUIAddress contains the scheme") { - val uiAddress = sc.ui.appUIAddress - assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) - } - - test("verify appUIAddress contains the port") { - val splitUIAddress = sc.ui.appUIAddress.split(':') - assert(splitUIAddress(2).toInt == sc.ui.boundPort) - } -} diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 52081227186e4..b85c483ca2a08 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -125,4 +125,18 @@ class UISuite extends FunSuite { case Failure(e) => } } + + test("verify appUIAddress contains the scheme") { + withSpark(new SparkContext("local", "test")) { sc => + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + } + + test("verify appUIAddress contains the port") { + withSpark(new SparkContext("local", "test")) { sc => + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } + } }