Skip to content

Commit d2ab91d

Browse files
committed
Addressed comments, and added base URL prepending to links in description
1 parent b5cb02e commit d2ab91d

File tree

5 files changed

+50
-45
lines changed

5 files changed

+50
-45
lines changed

core/src/main/scala/org/apache/spark/ui/UIUtils.scala

Lines changed: 42 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,9 @@ package org.apache.spark.ui
2020
import java.text.SimpleDateFormat
2121
import java.util.{Locale, Date}
2222

23-
import scala.xml.{Node, Text, Unparsed}
23+
import scala.util.control.NonFatal
24+
import scala.xml._
25+
import scala.xml.transform.{RewriteRule, RuleTransformer}
2426

2527
import org.apache.spark.Logging
2628
import org.apache.spark.ui.scope.RDDOperationGraph
@@ -395,4 +397,43 @@ private[spark] object UIUtils extends Logging {
395397
</script>
396398
}
397399

400+
/**
401+
* Convert a description string to HTML. It will try to parse the string as HTML and sanitize
402+
* any links. If that fails, then whole string will treated as a simple text.
403+
*/
404+
def makeDescription(desc: String, basePathUri: String): NodeSeq = {
405+
import scala.language.postfixOps
406+
407+
// If the description can be parsed as HTML and has only relative links, then render
408+
// as HTML, otherwise render as escaped string
409+
try {
410+
// Try to load the description as unescaped HTML
411+
val xml = XML.loadString("<span class=\"description-input\" " +
412+
s"title=${"\"" + Utility.escape(desc) + "\""}>$desc</span>")
413+
val allLinks = xml \\ "_" flatMap { _.attributes } filter { _.key == "href" }
414+
val areAllLinksRelative = allLinks.forall { _.value.toString.startsWith ("/") }
415+
416+
// If all the links are relative then, transform the links to absolute links
417+
// with basePathUri
418+
if (areAllLinksRelative) {
419+
val rule = new RewriteRule() {
420+
override def transform(n: Node): Seq[Node] = {
421+
n match {
422+
case e: Elem if e \ "@href" nonEmpty =>
423+
val relativePath = e.attribute("href").get.toString
424+
val fullUri = s"${basePathUri.stripSuffix("/")}/${relativePath.stripPrefix("/")}"
425+
e % Attribute(null, "href", fullUri, Null)
426+
case _ => n
427+
}
428+
}
429+
}
430+
new RuleTransformer(rule).transform(xml)
431+
} else {
432+
<span class="description-input" title={desc}> {desc} </span>
433+
}
434+
} catch {
435+
case NonFatal(e) =>
436+
<span class="description-input" title={desc}> {desc} </span>
437+
}
438+
}
398439
}

core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala

Lines changed: 1 addition & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@ import java.util.Date
2121
import javax.servlet.http.HttpServletRequest
2222

2323
import scala.collection.mutable.{HashMap, ListBuffer}
24-
import scala.util.control.NonFatal
2524
import scala.xml._
2625

2726
import org.apache.spark.JobExecutionStatus
@@ -225,27 +224,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
225224
}
226225
val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown")
227226
val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown")
228-
val jobDescription = {
229-
val d = lastStageDescription
230-
// If the description can be parsed as HTML and has only relative links, then render
231-
// as HTML, otherwise render as escaped string
232-
try {
233-
// Try to load the description as unescaped HTML
234-
val xml = XML.loadString("<span class=\"description-input\" " +
235-
s"title=${"\"" + Utility.escape(d) + "\""}>$d</span>")
236-
val allLinks = xml \\ "_" flatMap { _.attributes } filter { _.key == "href" }
237-
val areAllLinksRelative = allLinks.forall { _.value.toString.startsWith ("/") }
238-
if (areAllLinksRelative) {
239-
xml
240-
} else {
241-
<span class="description-input" title={d}> {d} </span>
242-
}
243-
} catch {
244-
case NonFatal(e) =>
245-
<span class="description-input" title={d}> {d} </span>
246-
}
247-
}
248-
227+
val jobDescription = UIUtils.makeDescription(lastStageDescription, parent.basePath)
249228

250229
val detailUrl =
251230
"%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId)

core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala

Lines changed: 2 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,7 @@ package org.apache.spark.ui.jobs
1919

2020
import java.util.Date
2121

22-
import scala.util.control.NonFatal
23-
import scala.xml.{Utility, Node, Text, XML}
22+
import scala.xml.{Node, Text}
2423

2524
import org.apache.commons.lang3.StringEscapeUtils
2625

@@ -116,23 +115,7 @@ private[ui] class StageTableBase(
116115
stageData <- listener.stageIdToData.get((s.stageId, s.attemptId))
117116
desc <- stageData.description
118117
} yield {
119-
// If the description can be parsed as HTML and has only relative links, then render
120-
// as HTML, otherwise render as escaped string
121-
try {
122-
// Try to load the description as unescaped HTML
123-
val xml = XML.loadString("<span class=\"description-input\" " +
124-
s"title=${"\"" + Utility.escape(desc) + "\""}>$desc</span>")
125-
val allLinks = xml \\ "_" flatMap { _.attributes } filter { _.key == "href" }
126-
val areAllLinksRelative = allLinks.forall { _.value.toString.startsWith ("/") }
127-
if (areAllLinksRelative) {
128-
xml
129-
} else {
130-
<span class="description-input" title={desc}> {desc} </span>
131-
}
132-
} catch {
133-
case NonFatal(e) =>
134-
<span class="description-input" title={desc}> {desc} </span>
135-
}
118+
UIUtils.makeDescription(desc, basePathUri)
136119
}
137120
<div>{stageDesc.getOrElse("")} {killLink} {nameLink} {details}</div>
138121
}

streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -198,7 +198,9 @@ class StreamingContext private[streaming] (
198198

199199
private var state: StreamingContextState = INITIALIZED
200200

201-
private[streaming] val startSite = new AtomicReference[CallSite](null)
201+
private val startSite = new AtomicReference[CallSite](null)
202+
203+
private[streaming] def getStartSite(): CallSite = startSite.get()
202204

203205
private var shutdownHookRef: AnyRef = _
204206

@@ -735,7 +737,7 @@ object StreamingContext extends Logging {
735737
throw new IllegalStateException(
736738
"Only one StreamingContext may be started in this JVM. " +
737739
"Currently running StreamingContext was started at" +
738-
activeContext.get.startSite.get.longForm)
740+
activeContext.get.getStartSite().longForm)
739741
}
740742
}
741743
}

streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -555,7 +555,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
555555
}
556556
receiverRDD.setName(s"Receiver $receiverId")
557557
ssc.sparkContext.setJobDescription(s"Streaming job running receiver $receiverId")
558-
ssc.sparkContext.setCallSite(ssc.startSite.get)
558+
ssc.sparkContext.setCallSite(ssc.getStartSite())
559559

560560
val future = ssc.sparkContext.submitJob[Receiver[_], Unit, Unit](
561561
receiverRDD, startReceiverFunc, Seq(0), (_, _) => Unit, ())

0 commit comments

Comments
 (0)