spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [spark] sarutak commented on a change in pull request #26201: [SPARK-29543][SS][UI] Init structured streaming ui
Date Mon, 25 Nov 2019 12:22:11 GMT
sarutak commented on a change in pull request #26201: [SPARK-29543][SS][UI] Init structured
streaming ui
URL: https://github.com/apache/spark/pull/26201#discussion_r350150555
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
 ##########
 @@ -0,0 +1,282 @@
+/*
+ * 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.sql.streaming.ui
+
+import java.{util => ju}
+import java.lang.{Long => JLong}
+import java.text.SimpleDateFormat
+import java.util.UUID
+import javax.servlet.http.HttpServletRequest
+
+import scala.collection.JavaConverters._
+import scala.xml.{Node, Unparsed}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone
+import org.apache.spark.sql.execution.streaming.{QuerySummary, StreamQueryStore}
+import org.apache.spark.sql.streaming.StreamingQuery
+import org.apache.spark.sql.streaming.ui.UIUtils._
+import org.apache.spark.ui.{GraphUIData, JsCollector, UIUtils => SparkUIUtils, WebUIPage}
+
+class StreamingQueryStatisticsPage(
+    parent: StreamingQueryTab,
+    store: StreamQueryStore)
+  extends WebUIPage("statistics") with Logging {
+  val df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'")
+  df.setTimeZone(getTimeZone("UTC"))
+
+  def generateLoadResources(request: HttpServletRequest): Seq[Node] = {
+    // scalastyle:off
+    <script src={SparkUIUtils.prependBaseUri(request, "/static/d3.min.js")}></script>
+        <link rel="stylesheet" href={SparkUIUtils.prependBaseUri(request, "/static/streaming-page.css")}
type="text/css"/>
+      <script src={SparkUIUtils.prependBaseUri(request, "/static/streaming-page.js")}></script>
+      <script src={SparkUIUtils.prependBaseUri(request, "/static/structured-streaming-page.js")}></script>
+    // scalastyle:on
+  }
+
+  override def render(request: HttpServletRequest): Seq[Node] = {
+    val parameterId = request.getParameter("id")
+    require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
+
+    val (query, timeSinceStart) = store.allStreamQueries.find { case (q, _) =>
+      q.runId.equals(UUID.fromString(parameterId))
+    }.getOrElse(throw new IllegalArgumentException(s"Failed to find streaming query $parameterId"))
+
+    val resources = generateLoadResources(request)
+    val basicInfo = generateBasicInfo(query, timeSinceStart)
+    val content =
+      resources ++
+        basicInfo ++
+        generateStatTable(query)
+    SparkUIUtils.headerSparkPage(request, "Streaming Query Statistics", content, parent)
+  }
+
+  def generateTimeMap(times: Seq[Long]): Seq[Node] = {
+    val js = "var timeFormat = {};\n" + times.map { time =>
+      val formattedTime = SparkUIUtils.formatBatchTime(time, 1, showYYYYMMSS = false)
+      s"timeFormat[$time] = '$formattedTime';"
+    }.mkString("\n")
+
+    <script>{Unparsed(js)}</script>
+  }
+
+  def generateVar(values: Array[(Long, ju.Map[String, JLong])]): Seq[Node] = {
+    val js = "var timeToValues = {};\n" + values.map { case (x, y) =>
+      val s =
+        y.asScala.toSeq.sortBy(_._1).map(e => s""""${e._2.toDouble}"""").mkString("[",
",", "]")
+      s"""timeToValues["${SparkUIUtils.formatBatchTime(x, 1, showYYYYMMSS = false)}"] = $s;"""
+    }.mkString("\n")
+
+    <script>{Unparsed(js)}</script>
+  }
+
+  def generateBasicInfo(query: StreamingQuery, timeSinceStart: Long): Seq[Node] = {
+    val duration = if (query.isActive) {
+      SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - timeSinceStart)
+    } else {
+      withNoProgress(query, {
+        val end = query.lastProgress.timestamp
+        val start = query.recentProgress.head.timestamp
+        SparkUIUtils.formatDurationVerbose(
+          df.parse(end).getTime - df.parse(start).getTime)
+      }, "-")
+    }
+
+    val name = UIUtils.getQueryName(query)
+    val numBatches = withNoProgress(query, { query.lastProgress.batchId + 1L }, 0)
+    val totalRecords = query.getQuerySummary.getMetric(QuerySummary.TOTAL_INPUT_RECORDS,
0L)
+    <div>Running batches for
+      <strong>
+        {duration}
+      </strong>
+      since
+      <strong>
+        {SparkUIUtils.formatDate(timeSinceStart)}
+      </strong>
+      (<strong>{numBatches}</strong> completed batches, <strong>{totalRecords}</strong>
records)
+    </div>
+    <br />
+    <div><strong>Name: </strong>{name}</div>
+    <div><strong>Id: </strong>{query.id}</div>
+    <div><strong>RunId: </strong>{query.runId}</div>
+    <br />
+  }
+
+  def generateStatTable(query: StreamingQuery): Seq[Node] = {
+    val batchTimes = withNoProgress(query,
+      query.recentProgress.map(p => df.parse(p.timestamp).getTime), Array.empty[Long])
+    val minBatchTime =
+      withNoProgress(query, df.parse(query.recentProgress.head.timestamp).getTime, 0L)
+    val maxBatchTime =
+      withNoProgress(query, df.parse(query.lastProgress.timestamp).getTime, 0L)
+    val maxRecordRate =
+      withNoProgress(query, query.recentProgress.map(_.inputRowsPerSecond).max, 0L)
+    val minRecordRate = 0L
+    val maxProcessRate =
+      withNoProgress(query, query.recentProgress.map(_.processedRowsPerSecond).max, 0L)
+
+    val minProcessRate = 0L
+    val maxRows = withNoProgress(query, query.recentProgress.map(_.numInputRows).max, 0L)
+    val minRows = 0L
+    val maxBatchDuration = withNoProgress(query, query.recentProgress.map(_.batchDuration).max,
0L)
+    val minBatchDuration = 0L
+
+    val inputRateData = withNoProgress(query,
+      query.recentProgress.map(p => (df.parse(p.timestamp).getTime,
+        withNumberInvalid { p.inputRowsPerSecond })), Array.empty[(Long, Double)])
+    val processRateData = withNoProgress(query,
+      query.recentProgress.map(p => (df.parse(p.timestamp).getTime,
+        withNumberInvalid { p.processedRowsPerSecond })), Array.empty[(Long, Double)])
+    val inputRowsData = withNoProgress(query,
+      query.recentProgress.map(p => (df.parse(p.timestamp).getTime,
+        withNumberInvalid { p.numInputRows })), Array.empty[(Long, Double)])
+    val batchDurations = withNoProgress(query,
+      query.recentProgress.map(p => (df.parse(p.timestamp).getTime,
+        withNumberInvalid { p.batchDuration })), Array.empty[(Long, Double)])
+    val operationDurationData = withNoProgress(query, query.recentProgress.map { p =>
+      val durationMs = p.durationMs
+      // remove "triggerExecution" as it count the other operation duration.
+      durationMs.remove("triggerExecution")
+      (df.parse(p.timestamp).getTime, durationMs)}, Array.empty[(Long, ju.Map[String, JLong])])
+    val operationLabels = withNoProgress(query, {
+      val durationKeys = query.lastProgress.durationMs.keySet()
 
 Review comment:
   `operationLabels` should not be determined based on only the **last** progress.
   If we use some v2-writers, one micro-batch contains >1 jobs.
   One is an [actual data processing job](https://github.com/apache/spark/blob/54c5087a3ae306ad766df81a5a6279f219b2ea47/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala#L376).
   And another job is a [dummy](https://github.com/apache/spark/blob/54c5087a3ae306ad766df81a5a6279f219b2ea47/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala#L412).
   So, in such case, the `last` progress can be the dummy job and `durationMs` should not
be set.
   
   As the result, if we see like a following graph,
   <img width="1069" alt="スクリーンショット 2019-11-25 20 34 03" src="https://user-images.githubusercontent.com/4736016/69539856-b1dabe00-0fc8-11ea-9b66-5767358dfa75.png">
   
   and then reload, the graph disappears because labels are not set for the dummy job.
   <img width="1062" alt="スクリーンショット 2019-11-25 20 34 19" src="https://user-images.githubusercontent.com/4736016/69540060-244b9e00-0fc9-11ea-9c59-e2b60ee0ec87.png">
   
   You can reproduce this issue easily by setting `writeStream.format("console")`.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


Mime
View raw message