You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by td...@apache.org on 2015/04/15 01:58:02 UTC

spark git commit: [SPARK-6796][Streaming][WebUI] Add "Active Batches" and "Completed Batches" lists to StreamingPage

Repository: spark
Updated Branches:
  refs/heads/master a76b921a9 -> 6de282e2d


[SPARK-6796][Streaming][WebUI] Add "Active Batches" and "Completed Batches" lists to StreamingPage

This PR adds two lists, `Active Batches` and `Completed Batches`. Here is the screenshot:

![batch_list](https://cloud.githubusercontent.com/assets/1000778/7060458/d8898572-deb3-11e4-938b-6f8602c71a9f.png)

Due to [SPARK-6766](https://issues.apache.org/jira/browse/SPARK-6766), I need to merge #5414 in my local machine to get the above screenshot.

Author: zsxwing <zs...@gmail.com>

Closes #5434 from zsxwing/SPARK-6796 and squashes the following commits:

be50fc6 [zsxwing] Fix the code style
51b792e [zsxwing] Fix the unit test
6f3078e [zsxwing] Make 'startTime' readable
f40e0a9 [zsxwing] Merge branch 'master' into SPARK-6796
2525336 [zsxwing] Rename 'Processed batches' and 'Waiting batches' and also add links
a69c091 [zsxwing] Show the number of total completed batches too
a12ad7b [zsxwing] Change 'records' to 'events' in the UI
86b5e7f [zsxwing] Make BatchTableBase abstract
b248787 [zsxwing] Add tests to verify the new tables
d18ab7d [zsxwing] Fix the code style
6ceffb3 [zsxwing] Add "Active Batches" and "Completed Batches" lists to StreamingPage


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6de282e2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6de282e2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6de282e2

Branch: refs/heads/master
Commit: 6de282e2de3cb69f9b746d03fde581429248824a
Parents: a76b921
Author: zsxwing <zs...@gmail.com>
Authored: Tue Apr 14 16:51:36 2015 -0700
Committer: Tathagata Das <ta...@gmail.com>
Committed: Tue Apr 14 16:51:36 2015 -0700

----------------------------------------------------------------------
 .../spark/streaming/ui/AllBatchesTable.scala    | 114 +++++++++++++++++++
 .../spark/streaming/ui/StreamingPage.scala      |  44 +++++--
 .../spark/streaming/UISeleniumSuite.scala       |  11 ++
 3 files changed, 159 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/6de282e2/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala
new file mode 100644
index 0000000..df1c0a1
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala
@@ -0,0 +1,114 @@
+/*
+ * 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.xml.Node
+
+import org.apache.spark.streaming.scheduler.BatchInfo
+import org.apache.spark.ui.UIUtils
+
+private[ui] abstract class BatchTableBase(tableId: String) {
+
+  protected def columns: Seq[Node] = {
+    <th>Batch Time</th>
+      <th>Input Size</th>
+      <th>Scheduling Delay</th>
+      <th>Processing Time</th>
+  }
+
+  protected def baseRow(batch: BatchInfo): Seq[Node] = {
+    val batchTime = batch.batchTime.milliseconds
+    val formattedBatchTime = UIUtils.formatDate(batch.batchTime.milliseconds)
+    val eventCount = batch.receivedBlockInfo.values.map {
+      receivers => receivers.map(_.numRecords).sum
+    }.sum
+    val schedulingDelay = batch.schedulingDelay
+    val formattedSchedulingDelay = schedulingDelay.map(UIUtils.formatDuration).getOrElse("-")
+    val processingTime = batch.processingDelay
+    val formattedProcessingTime = processingTime.map(UIUtils.formatDuration).getOrElse("-")
+
+    <td sorttable_customkey={batchTime.toString}>{formattedBatchTime}</td>
+      <td sorttable_customkey={eventCount.toString}>{eventCount.toString} events</td>
+      <td sorttable_customkey={schedulingDelay.getOrElse(Long.MaxValue).toString}>
+        {formattedSchedulingDelay}
+      </td>
+      <td sorttable_customkey={processingTime.getOrElse(Long.MaxValue).toString}>
+        {formattedProcessingTime}
+      </td>
+  }
+
+  private def batchTable: Seq[Node] = {
+    <table id={tableId} class="table table-bordered table-striped table-condensed sortable">
+      <thead>
+        {columns}
+      </thead>
+      <tbody>
+        {renderRows}
+      </tbody>
+    </table>
+  }
+
+  def toNodeSeq: Seq[Node] = {
+    batchTable
+  }
+
+  /**
+   * Return HTML for all rows of this table.
+   */
+  protected def renderRows: Seq[Node]
+}
+
+private[ui] class ActiveBatchTable(runningBatches: Seq[BatchInfo], waitingBatches: Seq[BatchInfo])
+  extends BatchTableBase("active-batches-table") {
+
+  override protected def columns: Seq[Node] = super.columns ++ <th>Status</th>
+
+  override protected def renderRows: Seq[Node] = {
+    // The "batchTime"s of "waitingBatches" must be greater than "runningBatches"'s, so display
+    // waiting batches before running batches
+    waitingBatches.flatMap(batch => <tr>{waitingBatchRow(batch)}</tr>) ++
+      runningBatches.flatMap(batch => <tr>{runningBatchRow(batch)}</tr>)
+  }
+
+  private def runningBatchRow(batch: BatchInfo): Seq[Node] = {
+    baseRow(batch) ++ <td>processing</td>
+  }
+
+  private def waitingBatchRow(batch: BatchInfo): Seq[Node] = {
+    baseRow(batch) ++ <td>queued</td>
+  }
+}
+
+private[ui] class CompletedBatchTable(batches: Seq[BatchInfo])
+  extends BatchTableBase("completed-batches-table") {
+
+  override protected def columns: Seq[Node] = super.columns ++ <th>Total Delay</th>
+
+  override protected def renderRows: Seq[Node] = {
+    batches.flatMap(batch => <tr>{completedBatchRow(batch)}</tr>)
+  }
+
+  private def completedBatchRow(batch: BatchInfo): Seq[Node] = {
+    val totalDelay = batch.totalDelay
+    val formattedTotalDelay = totalDelay.map(UIUtils.formatDuration).getOrElse("-")
+    baseRow(batch) ++
+      <td sorttable_customkey={totalDelay.getOrElse(Long.MaxValue).toString}>
+        {formattedTotalDelay}
+      </td>
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/6de282e2/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
----------------------------------------------------------------------
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 b6dcb62..07fa285 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
@@ -41,7 +41,8 @@ private[ui] class StreamingPage(parent: StreamingTab)
       generateBasicStats() ++ <br></br> ++
       <h4>Statistics over last {listener.retainedCompletedBatches.size} processed batches</h4> ++
       generateReceiverStats() ++
-      generateBatchStatsTable()
+      generateBatchStatsTable() ++
+      generateBatchListTables()
     }
     UIUtils.headerSparkPage("Streaming", content, parent, Some(5000))
   }
@@ -49,9 +50,10 @@ private[ui] class StreamingPage(parent: StreamingTab)
   /** Generate basic stats of the streaming program */
   private def generateBasicStats(): Seq[Node] = {
     val timeSinceStart = System.currentTimeMillis() - startTime
+    // scalastyle:off
     <ul class ="unstyled">
       <li>
-        <strong>Started at: </strong> {startTime.toString}
+        <strong>Started at: </strong> {UIUtils.formatDate(startTime)}
       </li>
       <li>
         <strong>Time since start: </strong>{formatDurationVerbose(timeSinceStart)}
@@ -63,18 +65,19 @@ private[ui] class StreamingPage(parent: StreamingTab)
         <strong>Batch interval: </strong>{formatDurationVerbose(listener.batchDuration)}
       </li>
       <li>
-        <strong>Processed batches: </strong>{listener.numTotalCompletedBatches}
+        <a href="#completed"><strong>Completed batches: </strong></a>{listener.numTotalCompletedBatches}
       </li>
       <li>
-        <strong>Waiting batches: </strong>{listener.numUnprocessedBatches}
+        <a href="#active"><strong>Active batches: </strong></a>{listener.numUnprocessedBatches}
       </li>
       <li>
-        <strong>Received records: </strong>{listener.numTotalReceivedRecords}
+        <strong>Received events: </strong>{listener.numTotalReceivedRecords}
       </li>
       <li>
-        <strong>Processed records: </strong>{listener.numTotalProcessedRecords}
+        <strong>Processed events: </strong>{listener.numTotalProcessedRecords}
       </li>
     </ul>
+    // scalastyle:on
   }
 
   /** Generate stats of data received by the receivers in the streaming program */
@@ -86,10 +89,10 @@ private[ui] class StreamingPage(parent: StreamingTab)
         "Receiver",
         "Status",
         "Location",
-        "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]",
-        "Minimum rate\n[records/sec]",
-        "Median rate\n[records/sec]",
-        "Maximum rate\n[records/sec]",
+        "Events in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]",
+        "Minimum rate\n[events/sec]",
+        "Median rate\n[events/sec]",
+        "Maximum rate\n[events/sec]",
         "Last Error"
       )
       val dataRows = (0 until listener.numReceivers).map { receiverId =>
@@ -190,5 +193,26 @@ private[ui] class StreamingPage(parent: StreamingTab)
     }
     UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true)
   }
+
+  private def generateBatchListTables(): Seq[Node] = {
+    val runningBatches = listener.runningBatches.sortBy(_.batchTime.milliseconds).reverse
+    val waitingBatches = listener.waitingBatches.sortBy(_.batchTime.milliseconds).reverse
+    val completedBatches = listener.retainedCompletedBatches.
+      sortBy(_.batchTime.milliseconds).reverse
+
+    val activeBatchesContent = {
+      <h4 id="active">Active Batches ({runningBatches.size + waitingBatches.size})</h4> ++
+        new ActiveBatchTable(runningBatches, waitingBatches).toNodeSeq
+    }
+
+    val completedBatchesContent = {
+      <h4 id="completed">
+        Completed Batches (last {completedBatches.size} out of {listener.numTotalCompletedBatches})
+      </h4> ++
+        new CompletedBatchTable(completedBatches).toNodeSeq
+    }
+
+    activeBatchesContent ++ completedBatchesContent
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/6de282e2/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
index 998426e..205ddf6 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
@@ -75,6 +75,17 @@ class UISeleniumSuite
         val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq
         statisticText should contain("Network receivers:")
         statisticText should contain("Batch interval:")
+
+        val h4Text = findAll(cssSelector("h4")).map(_.text).toSeq
+        h4Text should contain("Active Batches (0)")
+        h4Text should contain("Completed Batches (last 0 out of 0)")
+
+        findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be {
+          List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Status")
+        }
+        findAll(cssSelector("""#completed-batches-table th""")).map(_.text).toSeq should be {
+          List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Total Delay")
+        }
       }
 
       ssc.stop(false)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org