@@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s
18
18
19
19
import java .util .concurrent .atomic .{AtomicInteger , AtomicLong }
20
20
21
+ import com .palantir .logsafe .SafeArg
21
22
import io .fabric8 .kubernetes .api .model .PodBuilder
22
23
import io .fabric8 .kubernetes .client .KubernetesClient
23
24
import scala .collection .mutable
@@ -26,15 +27,15 @@ import org.apache.spark.{SparkConf, SparkException}
26
27
import org .apache .spark .deploy .k8s .Config ._
27
28
import org .apache .spark .deploy .k8s .Constants ._
28
29
import org .apache .spark .deploy .k8s .KubernetesConf
29
- import org .apache .spark .internal .Logging
30
+ import org .apache .spark .internal .SafeLogging
30
31
import org .apache .spark .util .{Clock , Utils }
31
32
32
33
private [spark] class ExecutorPodsAllocator (
33
34
conf : SparkConf ,
34
35
executorBuilder : KubernetesExecutorBuilder ,
35
36
kubernetesClient : KubernetesClient ,
36
37
snapshotsStore : ExecutorPodsSnapshotsStore ,
37
- clock : Clock ) extends Logging {
38
+ clock : Clock ) extends SafeLogging {
38
39
39
40
private val EXECUTOR_ID_COUNTER = new AtomicLong (0L )
40
41
@@ -82,10 +83,12 @@ private[spark] class ExecutorPodsAllocator(
82
83
newlyCreatedExecutors.foreach { case (execId, timeCreated) =>
83
84
val currentTime = clock.getTimeMillis()
84
85
if (currentTime - timeCreated > podCreationTimeout) {
85
- logWarning( s " Executor with id $execId was not detected in the Kubernetes " +
86
- s " cluster after $podCreationTimeout milliseconds despite the fact that a" +
86
+ safeLogWarning( " Executor was not detected in the Kubernetes" +
87
+ " cluster after timeout despite the fact that a" +
87
88
" previous allocation attempt tried to create it. The executor may have been" +
88
- " deleted but the application missed the deletion event." )
89
+ " deleted but the application missed the deletion event." ,
90
+ SafeArg .of(" executorId" , execId),
91
+ SafeArg .of(" podCreationTimeoutMs" , podCreationTimeout))
89
92
Utils .tryLogNonFatalError {
90
93
kubernetesClient
91
94
.pods()
@@ -94,8 +97,10 @@ private[spark] class ExecutorPodsAllocator(
94
97
}
95
98
newlyCreatedExecutors -= execId
96
99
} else {
97
- logDebug(s " Executor with id $execId was not found in the Kubernetes cluster since it " +
98
- s " was created ${currentTime - timeCreated} milliseconds ago. " )
100
+ safeLogDebug(" Executor was not found in the Kubernetes cluster since it" +
101
+ " was created some time ago." ,
102
+ SafeArg .of(" executorId" , execId),
103
+ SafeArg .of(" timeSinceCreationMs" , currentTime - timeCreated))
99
104
}
100
105
}
101
106
@@ -112,15 +117,19 @@ private[spark] class ExecutorPodsAllocator(
112
117
case _ => false
113
118
}
114
119
val currentTotalExpectedExecutors = totalExpectedExecutors.get
115
- logDebug(s " Currently have $currentRunningExecutors running executors and " +
116
- s " $currentPendingExecutors pending executors. $newlyCreatedExecutors executors " +
117
- s " have been requested but are pending appearance in the cluster. " )
120
+ safeLogDebug(" Currently have running executors and" +
121
+ " pending executors. Newly created executors" +
122
+ " have been requested but are pending appearance in the cluster." ,
123
+ SafeArg .of(" numCurrentRunningExecutors" , currentRunningExecutors),
124
+ SafeArg .of(" numCurrentPendingExecutors" , currentPendingExecutors),
125
+ SafeArg .of(" newlyCreatedExecutors" , newlyCreatedExecutors))
118
126
if (newlyCreatedExecutors.isEmpty
119
127
&& currentPendingExecutors == 0
120
128
&& currentRunningExecutors < currentTotalExpectedExecutors) {
121
129
val numExecutorsToAllocate = math.min(
122
130
currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize)
123
- logInfo(s " Going to request $numExecutorsToAllocate executors from Kubernetes. " )
131
+ safeLogInfo(" Going to request executors from Kubernetes." ,
132
+ SafeArg .of(" numExecutorsToAllocate" , numExecutorsToAllocate))
124
133
for ( _ <- 0 until numExecutorsToAllocate) {
125
134
val newExecutorId = EXECUTOR_ID_COUNTER .incrementAndGet()
126
135
val executorConf = KubernetesConf .createExecutorConf(
@@ -136,18 +145,22 @@ private[spark] class ExecutorPodsAllocator(
136
145
.build()
137
146
kubernetesClient.pods().create(podWithAttachedContainer)
138
147
newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis()
139
- logDebug(s " Requested executor with id $newExecutorId from Kubernetes. " )
148
+ safeLogDebug(" Requested executor from Kubernetes." ,
149
+ SafeArg .of(" newExecutorId" , newExecutorId))
140
150
}
141
151
} else if (currentRunningExecutors >= currentTotalExpectedExecutors) {
142
152
// TODO handle edge cases if we end up with more running executors than expected.
143
- logDebug (" Current number of running executors is equal to the number of requested" +
153
+ safeLogDebug (" Current number of running executors is equal to the number of requested" +
144
154
" executors. Not scaling up further." )
145
155
} else if (newlyCreatedExecutors.nonEmpty || currentPendingExecutors != 0 ) {
146
- logDebug(s " Still waiting for ${newlyCreatedExecutors.size + currentPendingExecutors}" +
147
- s " executors to begin running before requesting for more executors. # of executors in " +
148
- s " pending status in the cluster: $currentPendingExecutors. # of executors that we have " +
149
- s " created but we have not observed as being present in the cluster yet: " +
150
- s " ${newlyCreatedExecutors.size}. " )
156
+ safeLogDebug(" Still waiting for" +
157
+ " executors to begin running before requesting for more executors, including executors" +
158
+ " in pending status in the cluster, and executors that we have" +
159
+ " created but we have not observed as being present in the cluster yet." ,
160
+ SafeArg .of(" numTotalCurrentWaitingExecutors" ,
161
+ newlyCreatedExecutors.size + currentPendingExecutors),
162
+ SafeArg .of(" numCurrentPendingExecutors" , currentPendingExecutors),
163
+ SafeArg .of(" numNewlyCreatedExecutors" , newlyCreatedExecutors.size))
151
164
}
152
165
}
153
166
}
0 commit comments