Skip to content

Commit 8673fe1

Browse files
author
Kostas Sakellis
committed
CR feedback. Hide the log column if there are no logs available
1 parent 5bf6952 commit 8673fe1

File tree

3 files changed

+23
-19
lines changed

3 files changed

+23
-19
lines changed

core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend(
5353
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
5454
}
5555

56-
def extractLogUrls : Map[String, String] = {
56+
def extractLogUrls: Map[String, String] = {
5757
val prefix = "SPARK_LOG_URL_"
5858
sys.env.filterKeys(_.startsWith(prefix))
5959
.map(e => (e._1.substring(prefix.length).toLowerCase, e._2))

core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala

Lines changed: 15 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,7 @@ private[ui] class ExecutorsPage(
5656
val diskUsed = storageStatusList.map(_.diskUsed).sum
5757
val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId)
5858
val execInfoSorted = execInfo.sortBy(_.id)
59+
val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty
5960

6061
val execTable =
6162
<table class={UIUtils.TABLE_CLASS_STRIPED}>
@@ -80,11 +81,11 @@ private[ui] class ExecutorsPage(
8081
Shuffle Write
8182
</span>
8283
</th>
83-
<th class="sorttable_nosort">Logs</th>
84+
{if (logsExist) <th class="sorttable_nosort">Logs</th> else Seq.empty}
8485
{if (threadDumpEnabled) <th class="sorttable_nosort">Thread Dump</th> else Seq.empty}
8586
</thead>
8687
<tbody>
87-
{execInfoSorted.map(execRow)}
88+
{execInfoSorted.map(execRow(_, logsExist))}
8889
</tbody>
8990
</table>
9091

@@ -109,7 +110,7 @@ private[ui] class ExecutorsPage(
109110
}
110111

111112
/** Render an HTML row representing an executor */
112-
private def execRow(info: ExecutorSummaryInfo): Seq[Node] = {
113+
private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = {
113114
val maximumMemory = info.maxMemory
114115
val memoryUsed = info.memoryUsed
115116
val diskUsed = info.diskUsed
@@ -140,13 +141,19 @@ private[ui] class ExecutorsPage(
140141
<td sorttable_customkey={info.totalShuffleWrite.toString}>
141142
{Utils.bytesToString(info.totalShuffleWrite)}
142143
</td>
143-
<td>
144144
{
145-
info.executorLogs.map(entry => {
146-
<div><a href={s"${entry._2}"}>{entry._1}</a></div>
147-
})
145+
if (logsExist) {
146+
<td>
147+
{info.executorLogs.map { entry => {
148+
<div>
149+
<a href={s"${entry._2}"}>
150+
{entry._1}
151+
</a>
152+
</div>}
153+
}}
154+
</td>
155+
}
148156
}
149-
</td>
150157
{
151158
if (threadDumpEnabled) {
152159
val encodedId = URLEncoder.encode(info.id, "UTF-8")

yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala

Lines changed: 7 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -20,22 +20,19 @@ package org.apache.spark.deploy.yarn
2020
import java.io.File
2121
import java.util.concurrent.TimeUnit
2222

23-
import org.apache.spark.scheduler.cluster.ExecutorInfo
24-
import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded}
25-
2623
import scala.collection.JavaConversions._
24+
import scala.collection.mutable
2725

2826
import com.google.common.base.Charsets
2927
import com.google.common.io.Files
30-
import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
31-
3228
import org.apache.hadoop.yarn.conf.YarnConfiguration
3329
import org.apache.hadoop.yarn.server.MiniYARNCluster
30+
import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
3431

35-
import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException}
32+
import org.apache.spark.scheduler.cluster.ExecutorInfo
33+
import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded}
3634
import org.apache.spark.util.Utils
37-
38-
import scala.collection.mutable
35+
import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException}
3936

4037
class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers with Logging {
4138

@@ -167,9 +164,9 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit
167164
checkResult(result)
168165

169166
// verify log urls are present.
170-
YarnClusterDriver.listener.addedExecutorInfos.foreach(e => {
167+
YarnClusterDriver.listener.addedExecutorInfos.foreach { e => {
171168
assert(e._2.logUrlMap.nonEmpty)
172-
})
169+
}}
173170
}
174171

175172
test("run Spark in yarn-cluster mode unsuccessfully") {

0 commit comments

Comments
 (0)