-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-10652][SPARK-10742][Streaming] Set meaningful job descriptions for all streaming jobs #8791
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
e8e7894
130e82e
7206f2e
b15012f
fd0d435
b5cb02e
d2ab91d
03bef12
21c90ea
4a45c82
4a0aba8
f9524c6
2fbfd1f
606b284
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,66 @@ | ||
| /* | ||
| * 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 scala.xml.Elem | ||
|
|
||
| import org.apache.spark.SparkFunSuite | ||
|
|
||
| class UIUtilsSuite extends SparkFunSuite { | ||
| import UIUtils._ | ||
|
|
||
| test("makeDescription") { | ||
| verify( | ||
| """test <a href="/link"> text </a>""", | ||
| <span class="description-input">test <a href="/link"> text </a></span>, | ||
| "Correctly formatted text with only anchors and relative links should generate HTML" | ||
| ) | ||
|
|
||
| verify( | ||
| """test <a href="/link" text </a>""", | ||
| <span class="description-input">{"""test <a href="/link" text </a>"""}</span>, | ||
| "Badly formatted text should make the description be treated as a streaming instead of HTML" | ||
| ) | ||
|
|
||
| verify( | ||
| """test <a href="link"> text </a>""", | ||
| <span class="description-input">{"""test <a href="link"> text </a>"""}</span>, | ||
| "Non-relative links should make the description be treated as a string instead of HTML" | ||
| ) | ||
|
|
||
| verify( | ||
| """test<a><img></img></a>""", | ||
| <span class="description-input">{"""test<a><img></img></a>"""}</span>, | ||
| "Non-anchor elements should make the description be treated as a string instead of HTML" | ||
| ) | ||
|
|
||
| verify( | ||
| """test <a href="/link"> text </a>""", | ||
| <span class="description-input">test <a href="base/link"> text </a></span>, | ||
| baseUrl = "base", | ||
| errorMsg = "Base URL should be prepended to html links" | ||
| ) | ||
| } | ||
|
|
||
| private def verify( | ||
| desc: String, expected: Elem, errorMsg: String = "", baseUrl: String = ""): Unit = { | ||
| val generated = makeDescription(desc, baseUrl) | ||
| assert(generated.sameElements(expected), | ||
| s"\n$errorMsg\n\nExpected:\n$expected\nGenerated:\n$generated") | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,6 +25,7 @@ import scala.util.{Failure, Success} | |
| import org.apache.spark.Logging | ||
| import org.apache.spark.rdd.PairRDDFunctions | ||
| import org.apache.spark.streaming._ | ||
| import org.apache.spark.streaming.ui.UIUtils | ||
| import org.apache.spark.util.{EventLoop, ThreadUtils} | ||
|
|
||
|
|
||
|
|
@@ -190,10 +191,20 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { | |
| } | ||
|
|
||
| private class JobHandler(job: Job) extends Runnable with Logging { | ||
| import JobScheduler._ | ||
|
|
||
| def run() { | ||
| ssc.sc.setLocalProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, job.time.milliseconds.toString) | ||
| ssc.sc.setLocalProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, job.outputOpId.toString) | ||
| try { | ||
| val formattedTime = UIUtils.formatBatchTime( | ||
| job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) | ||
| val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" | ||
| val batchLinkText = s"[output operation ${job.outputOpId}, batch time ${formattedTime}]" | ||
|
|
||
| ssc.sc.setJobDescription( | ||
| s"""Streaming job from <a href="$batchUrl">$batchLinkText</a>""") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is assuming the UI is the only consumer of this? I suppose that's OK since no one uses
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Whoever uses it will get the data as text. Only the UI would actually process it as a link. Nothing breaks.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It looks like this caused a cosmetic issue in the tooltips though: #11845 |
||
| ssc.sc.setLocalProperty(BATCH_TIME_PROPERTY_KEY, job.time.milliseconds.toString) | ||
| ssc.sc.setLocalProperty(OUTPUT_OP_ID_PROPERTY_KEY, job.outputOpId.toString) | ||
|
|
||
| // We need to assign `eventLoop` to a temp variable. Otherwise, because | ||
| // `JobScheduler.stop(false)` may set `eventLoop` to null when this method is running, then | ||
| // it's possible that when `post` is called, `eventLoop` happens to null. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,14 +19,14 @@ package org.apache.spark.streaming.ui | |
|
|
||
| import javax.servlet.http.HttpServletRequest | ||
|
|
||
| import scala.xml.{NodeSeq, Node, Text, Unparsed} | ||
| import scala.xml._ | ||
|
|
||
| import org.apache.commons.lang3.StringEscapeUtils | ||
|
|
||
| import org.apache.spark.streaming.Time | ||
| import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} | ||
| import org.apache.spark.streaming.ui.StreamingJobProgressListener.{SparkJobId, OutputOpId} | ||
| import org.apache.spark.streaming.ui.StreamingJobProgressListener.{OutputOpId, SparkJobId} | ||
| import org.apache.spark.ui.jobs.UIData.JobUIData | ||
| import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} | ||
|
|
||
| private[ui] case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) | ||
|
|
||
|
|
@@ -207,16 +207,25 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { | |
| sparkListener.stageIdToInfo.get(sparkJob.stageIds.max) | ||
| } | ||
| } | ||
| val lastStageData = lastStageInfo.flatMap { s => | ||
| sparkListener.stageIdToData.get((s.stageId, s.attemptId)) | ||
| } | ||
|
|
||
| val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") | ||
| val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") | ||
| lastStageInfo match { | ||
| case Some(stageInfo) => | ||
| val details = if (stageInfo.details.nonEmpty) { | ||
| <span | ||
| onclick="this.parentNode.querySelector('.stage-details').classList.toggle('collapsed')" | ||
| class="expand-details"> | ||
| +details | ||
| </span> ++ | ||
| <div class="stage-details collapsed"> | ||
| <pre>{stageInfo.details}</pre> | ||
| </div> | ||
| } else { | ||
| NodeSeq.Empty | ||
| } | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
|
|
||
| <span class="description-input" title={lastStageDescription}> | ||
| {lastStageDescription} | ||
| </span> ++ Text(lastStageName) | ||
| <div> {stageInfo.name} {details} </div> | ||
| case None => | ||
| Text("(Unknown)") | ||
| } | ||
| } | ||
|
|
||
| private def failureReasonCell(failureReason: String): Seq[Node] = { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can you add a paragraph on the security implications here? (What happens if I put in a
<script>tag or something, or if I reference a malicious link)