Skip to content

Commit 27d3b0a

Browse files
rvessemccheah
authored andcommitted
[SPARK-25222][K8S] Improve container status logging
## What changes were proposed in this pull request? Currently when running Spark on Kubernetes a logger is run by the client that watches the K8S API for events related to the Driver pod and logs them. However for the container status aspect of the logging this simply dumps the raw object which is not human readable e.g. ![screen shot 2018-08-24 at 10 37 46](https://user-images.githubusercontent.com/2104864/44577799-e0486880-a789-11e8-9ae9-fdeddacbbea8.png) ![screen shot 2018-08-24 at 10 38 14](https://user-images.githubusercontent.com/2104864/44577800-e0e0ff00-a789-11e8-81f5-3bb315dbbdb1.png) This is despite the fact that the logging class in question actually has methods to pretty print this information but only invokes these at the end of a job. This PR improves the logging to always use the pretty printing methods, additionally modifying them to include further useful information provided by the K8S API. A similar issue also exists when tasks are lost that will be addressed by further commits to this PR - [x] Improved `LoggingPodStatusWatcher` - [x] Improved container status on task failure ## How was this patch tested? Built and launched jobs with the updated Spark client and observed the new human readable output: ![screen shot 2018-08-24 at 11 09 32](https://user-images.githubusercontent.com/2104864/44579429-5353de00-a78e-11e8-9228-c750af8e6311.png) ![screen shot 2018-08-24 at 11 09 42](https://user-images.githubusercontent.com/2104864/44579430-5353de00-a78e-11e8-8fce-d5bb2a3ae65f.png) ![screen shot 2018-08-24 at 11 10 13](https://user-images.githubusercontent.com/2104864/44579431-53ec7480-a78e-11e8-9fa2-aeabc5b28ec4.png) ![screen shot 2018-08-24 at 17 47 44](https://user-images.githubusercontent.com/2104864/44596922-db090f00-a7c5-11e8-910c-bc2339f5a196.png) Suggested reviewers: liyinan926 mccheah Author: Rob Vesse <rvesse@dotnetrdf.org> Closes #22215 from rvesse/SPARK-25222.
1 parent c84bc40 commit 27d3b0a

File tree

4 files changed

+95
-79
lines changed

4 files changed

+95
-79
lines changed

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala

Lines changed: 82 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,11 @@
1616
*/
1717
package org.apache.spark.deploy.k8s
1818

19-
import org.apache.spark.SparkConf
19+
import scala.collection.JavaConverters._
20+
21+
import io.fabric8.kubernetes.api.model.{ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, Time}
22+
23+
import org.apache.spark.{SparkConf, SparkException}
2024
import org.apache.spark.util.Utils
2125

2226
private[spark] object KubernetesUtils {
@@ -60,4 +64,81 @@ private[spark] object KubernetesUtils {
6064
}
6165

6266
def parseMasterUrl(url: String): String = url.substring("k8s://".length)
67+
68+
def formatPairsBundle(pairs: Seq[(String, String)], indent: Int = 1) : String = {
69+
// Use more loggable format if value is null or empty
70+
val indentStr = "\t" * indent
71+
pairs.map {
72+
case (k, v) => s"\n$indentStr $k: ${Option(v).filter(_.nonEmpty).getOrElse("N/A")}"
73+
}.mkString("")
74+
}
75+
76+
/**
77+
* Given a pod, output a human readable representation of its state
78+
*
79+
* @param pod Pod
80+
* @return Human readable pod state
81+
*/
82+
def formatPodState(pod: Pod): String = {
83+
val details = Seq[(String, String)](
84+
// pod metadata
85+
("pod name", pod.getMetadata.getName),
86+
("namespace", pod.getMetadata.getNamespace),
87+
("labels", pod.getMetadata.getLabels.asScala.mkString(", ")),
88+
("pod uid", pod.getMetadata.getUid),
89+
("creation time", formatTime(pod.getMetadata.getCreationTimestamp)),
90+
91+
// spec details
92+
("service account name", pod.getSpec.getServiceAccountName),
93+
("volumes", pod.getSpec.getVolumes.asScala.map(_.getName).mkString(", ")),
94+
("node name", pod.getSpec.getNodeName),
95+
96+
// status
97+
("start time", formatTime(pod.getStatus.getStartTime)),
98+
("phase", pod.getStatus.getPhase),
99+
("container status", containersDescription(pod, 2))
100+
)
101+
102+
formatPairsBundle(details)
103+
}
104+
105+
def containersDescription(p: Pod, indent: Int = 1): String = {
106+
p.getStatus.getContainerStatuses.asScala.map { status =>
107+
Seq(
108+
("container name", status.getName),
109+
("container image", status.getImage)) ++
110+
containerStatusDescription(status)
111+
}.map(p => formatPairsBundle(p, indent)).mkString("\n\n")
112+
}
113+
114+
def containerStatusDescription(containerStatus: ContainerStatus)
115+
: Seq[(String, String)] = {
116+
val state = containerStatus.getState
117+
Option(state.getRunning)
118+
.orElse(Option(state.getTerminated))
119+
.orElse(Option(state.getWaiting))
120+
.map {
121+
case running: ContainerStateRunning =>
122+
Seq(
123+
("container state", "running"),
124+
("container started at", formatTime(running.getStartedAt)))
125+
case waiting: ContainerStateWaiting =>
126+
Seq(
127+
("container state", "waiting"),
128+
("pending reason", waiting.getReason))
129+
case terminated: ContainerStateTerminated =>
130+
Seq(
131+
("container state", "terminated"),
132+
("container started at", formatTime(terminated.getStartedAt)),
133+
("container finished at", formatTime(terminated.getFinishedAt)),
134+
("exit code", terminated.getExitCode.toString),
135+
("termination reason", terminated.getReason))
136+
case unknown =>
137+
throw new SparkException(s"Unexpected container status type ${unknown.getClass}.")
138+
}.getOrElse(Seq(("container state", "N/A")))
139+
}
140+
141+
def formatTime(time: Time): String = {
142+
if (time != null) time.getTime else "N/A"
143+
}
63144
}

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala

Lines changed: 1 addition & 72 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher}
2525
import io.fabric8.kubernetes.client.Watcher.Action
2626

2727
import org.apache.spark.SparkException
28+
import org.apache.spark.deploy.k8s.KubernetesUtils._
2829
import org.apache.spark.internal.Logging
2930
import org.apache.spark.util.ThreadUtils
3031

@@ -99,82 +100,10 @@ private[k8s] class LoggingPodStatusWatcherImpl(
99100
scheduler.shutdown()
100101
}
101102

102-
private def formatPodState(pod: Pod): String = {
103-
val details = Seq[(String, String)](
104-
// pod metadata
105-
("pod name", pod.getMetadata.getName),
106-
("namespace", pod.getMetadata.getNamespace),
107-
("labels", pod.getMetadata.getLabels.asScala.mkString(", ")),
108-
("pod uid", pod.getMetadata.getUid),
109-
("creation time", formatTime(pod.getMetadata.getCreationTimestamp)),
110-
111-
// spec details
112-
("service account name", pod.getSpec.getServiceAccountName),
113-
("volumes", pod.getSpec.getVolumes.asScala.map(_.getName).mkString(", ")),
114-
("node name", pod.getSpec.getNodeName),
115-
116-
// status
117-
("start time", formatTime(pod.getStatus.getStartTime)),
118-
("container images",
119-
pod.getStatus.getContainerStatuses
120-
.asScala
121-
.map(_.getImage)
122-
.mkString(", ")),
123-
("phase", pod.getStatus.getPhase),
124-
("status", pod.getStatus.getContainerStatuses.toString)
125-
)
126-
127-
formatPairsBundle(details)
128-
}
129-
130-
private def formatPairsBundle(pairs: Seq[(String, String)]) = {
131-
// Use more loggable format if value is null or empty
132-
pairs.map {
133-
case (k, v) => s"\n\t $k: ${Option(v).filter(_.nonEmpty).getOrElse("N/A")}"
134-
}.mkString("")
135-
}
136-
137103
override def awaitCompletion(): Unit = {
138104
podCompletedFuture.await()
139105
logInfo(pod.map { p =>
140106
s"Container final statuses:\n\n${containersDescription(p)}"
141107
}.getOrElse("No containers were found in the driver pod."))
142108
}
143-
144-
private def containersDescription(p: Pod): String = {
145-
p.getStatus.getContainerStatuses.asScala.map { status =>
146-
Seq(
147-
("Container name", status.getName),
148-
("Container image", status.getImage)) ++
149-
containerStatusDescription(status)
150-
}.map(formatPairsBundle).mkString("\n\n")
151-
}
152-
153-
private def containerStatusDescription(
154-
containerStatus: ContainerStatus): Seq[(String, String)] = {
155-
val state = containerStatus.getState
156-
Option(state.getRunning)
157-
.orElse(Option(state.getTerminated))
158-
.orElse(Option(state.getWaiting))
159-
.map {
160-
case running: ContainerStateRunning =>
161-
Seq(
162-
("Container state", "Running"),
163-
("Container started at", formatTime(running.getStartedAt)))
164-
case waiting: ContainerStateWaiting =>
165-
Seq(
166-
("Container state", "Waiting"),
167-
("Pending reason", waiting.getReason))
168-
case terminated: ContainerStateTerminated =>
169-
Seq(
170-
("Container state", "Terminated"),
171-
("Exit code", terminated.getExitCode.toString))
172-
case unknown =>
173-
throw new SparkException(s"Unexpected container status type ${unknown.getClass}.")
174-
}.getOrElse(Seq(("Container state", "N/A")))
175-
}
176-
177-
private def formatTime(time: Time): String = {
178-
if (time != null) time.getTime else "N/A"
179-
}
180109
}

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import scala.collection.mutable
2424

2525
import org.apache.spark.SparkConf
2626
import org.apache.spark.deploy.k8s.Config._
27+
import org.apache.spark.deploy.k8s.KubernetesUtils._
2728
import org.apache.spark.internal.Logging
2829
import org.apache.spark.scheduler.ExecutorExited
2930
import org.apache.spark.util.Utils
@@ -151,13 +152,15 @@ private[spark] class ExecutorPodsLifecycleManager(
151152

152153
private def exitReasonMessage(podState: FinalPodState, execId: Long, exitCode: Int) = {
153154
val pod = podState.pod
155+
val reason = Option(pod.getStatus.getReason)
156+
val message = Option(pod.getStatus.getMessage)
154157
s"""
155158
|The executor with id $execId exited with exit code $exitCode.
156-
|The API gave the following brief reason: ${pod.getStatus.getReason}
157-
|The API gave the following message: ${pod.getStatus.getMessage}
159+
|The API gave the following brief reason: ${reason.getOrElse("N/A")}
160+
|The API gave the following message: ${message.getOrElse("N/A")}
158161
|The API gave the following container statuses:
159162
|
160-
|${pod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")}
163+
|${containersDescription(pod)}
161164
""".stripMargin
162165
}
163166

resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ import scala.collection.mutable
3131

3232
import org.apache.spark.{SparkConf, SparkFunSuite}
3333
import org.apache.spark.deploy.k8s.Fabric8Aliases._
34+
import org.apache.spark.deploy.k8s.KubernetesUtils._
3435
import org.apache.spark.scheduler.ExecutorExited
3536
import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils._
3637

@@ -104,13 +105,15 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte
104105
}
105106

106107
private def exitReasonMessage(failedExecutorId: Int, failedPod: Pod): String = {
108+
val reason = Option(failedPod.getStatus.getReason)
109+
val message = Option(failedPod.getStatus.getMessage)
107110
s"""
108111
|The executor with id $failedExecutorId exited with exit code 1.
109-
|The API gave the following brief reason: ${failedPod.getStatus.getReason}
110-
|The API gave the following message: ${failedPod.getStatus.getMessage}
112+
|The API gave the following brief reason: ${reason.getOrElse("N/A")}
113+
|The API gave the following message: ${message.getOrElse("N/A")}
111114
|The API gave the following container statuses:
112115
|
113-
|${failedPod.getStatus.getContainerStatuses.asScala.map(_.toString).mkString("\n===\n")}
116+
|${containersDescription(failedPod)}
114117
""".stripMargin
115118
}
116119

0 commit comments

Comments
 (0)