|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.streaming.ui |
| 19 | + |
| 20 | +import scala.collection.mutable.SynchronizedQueue |
| 21 | +import scala.xml.Node |
| 22 | + |
| 23 | +import javax.servlet.http.HttpServletRequest |
| 24 | +import org.eclipse.jetty.servlet.ServletContextHandler |
| 25 | + |
| 26 | +import org.apache.spark.Logging |
| 27 | +import org.apache.spark.streaming.StreamingContext |
| 28 | +import org.apache.spark.streaming.scheduler.{BatchInfo, StreamingListener, StreamingListenerBatchCompleted} |
| 29 | +import org.apache.spark.ui.{ServerInfo, SparkUI} |
| 30 | +import org.apache.spark.ui.JettyUtils._ |
| 31 | +import org.apache.spark.util.{Distribution, Utils} |
| 32 | + |
| 33 | +private[spark] class StreamingUIListener() extends StreamingListener { |
| 34 | + |
| 35 | + private val batchInfos = new SynchronizedQueue[BatchInfo] |
| 36 | + private val maxBatchInfos = 100 |
| 37 | + |
| 38 | + override def onBatchCompleted(batchStarted: StreamingListenerBatchCompleted) { |
| 39 | + batchInfos.enqueue(batchStarted.batchInfo) |
| 40 | + if (batchInfos.size > maxBatchInfos) batchInfos.dequeue() |
| 41 | + } |
| 42 | + |
| 43 | + def processingDelayDistribution = extractDistribution(_.processingDelay) |
| 44 | + |
| 45 | + def schedulingDelayDistribution = extractDistribution(_.schedulingDelay) |
| 46 | + |
| 47 | + def totalDelay = extractDistribution(_.totalDelay) |
| 48 | + |
| 49 | + def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { |
| 50 | + Distribution(batchInfos.flatMap(getMetric(_)).map(_.toDouble)) |
| 51 | + } |
| 52 | + |
| 53 | + def numBatchInfos = batchInfos.size |
| 54 | +} |
| 55 | + |
| 56 | +private[spark] class StreamingUI(ssc: StreamingContext) extends Logging { |
| 57 | + |
| 58 | + private val sc = ssc.sparkContext |
| 59 | + private val conf = sc.conf |
| 60 | + private val appName = sc.appName |
| 61 | + private val bindHost = Utils.localHostName() |
| 62 | + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) |
| 63 | + private val port = conf.getInt("spark.streaming.ui.port", StreamingUI.DEFAULT_PORT) |
| 64 | + private val securityManager = sc.env.securityManager |
| 65 | + private val listener = new StreamingUIListener() |
| 66 | + private val handlers: Seq[ServletContextHandler] = { |
| 67 | + Seq( |
| 68 | + createServletHandler("/", |
| 69 | + (request: HttpServletRequest) => render(request), securityManager), |
| 70 | + createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static") |
| 71 | + ) |
| 72 | + } |
| 73 | + |
| 74 | + private var serverInfo: Option[ServerInfo] = None |
| 75 | + |
| 76 | + ssc.addStreamingListener(listener) |
| 77 | + |
| 78 | + def bind() { |
| 79 | + try { |
| 80 | + serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) |
| 81 | + logInfo("Started Spark Streaming Web UI at http://%s:%d".format(publicHost, boundPort)) |
| 82 | + } catch { |
| 83 | + case e: Exception => |
| 84 | + logError("Failed to create Spark JettyUtils", e) |
| 85 | + System.exit(1) |
| 86 | + } |
| 87 | + } |
| 88 | + |
| 89 | + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) |
| 90 | + |
| 91 | + private def render(request: HttpServletRequest): Seq[Node] = { |
| 92 | + val batchStatsTable = generateBatchStatsTable() |
| 93 | + val content = batchStatsTable |
| 94 | + UIUtils.headerStreamingPage(content, "", appName, "Spark Streaming Overview") |
| 95 | + } |
| 96 | + |
| 97 | + private def generateBatchStatsTable(): Seq[Node] = { |
| 98 | + def getQuantiles(timeDistributionOption: Option[Distribution]) = { |
| 99 | + timeDistributionOption.get.getQuantiles().map { ms => Utils.msDurationToString(ms.toLong) } |
| 100 | + } |
| 101 | + val numBatches = listener.numBatchInfos |
| 102 | + val table = if (numBatches > 0) { |
| 103 | + val processingDelayQuantilesRow = |
| 104 | + "Processing Times" +: getQuantiles(listener.processingDelayDistribution) |
| 105 | + val schedulingDelayQuantilesRow = |
| 106 | + "Scheduling Delay:" +: getQuantiles(listener.processingDelayDistribution) |
| 107 | + val totalDelayQuantilesRow = |
| 108 | + "End-to-end Delay:" +: getQuantiles(listener.totalDelay) |
| 109 | + |
| 110 | + val headerRow = Seq("Metric", "Min", "25th percentile", |
| 111 | + "Median", "75th percentile", "Max") |
| 112 | + val dataRows: Seq[Seq[String]] = Seq( |
| 113 | + processingDelayQuantilesRow, |
| 114 | + schedulingDelayQuantilesRow, |
| 115 | + totalDelayQuantilesRow |
| 116 | + ) |
| 117 | + Some(UIUtils.listingTable(headerRow, dataRows, fixedWidth = true)) |
| 118 | + } else { |
| 119 | + None |
| 120 | + } |
| 121 | + |
| 122 | + val content = |
| 123 | + <h4>Batch Processing Statistics</h4> ++ |
| 124 | + <div>{table.getOrElse("No statistics have been generated yet.")}</div> |
| 125 | + content |
| 126 | + } |
| 127 | +} |
| 128 | + |
| 129 | +object StreamingUI { |
| 130 | + val DEFAULT_PORT = 6060 |
| 131 | +} |
0 commit comments