forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 1
/
ApplicationMaster.scala
464 lines (400 loc) · 17.1 KB
/
ApplicationMaster.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
/*
* 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.deploy.yarn
import java.io.IOException
import java.util.concurrent.CopyOnWriteArrayList
import java.util.concurrent.atomic.AtomicReference
import scala.collection.JavaConversions._
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.util.ShutdownHookManager
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.protocolrecords._
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.client.api.AMRMClient
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.util.ConverterUtils
import org.apache.hadoop.yarn.webapp.util.WebAppUtils
import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.util.Utils
/**
* An application master that runs the user's driver program and allocates executors.
*/
class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
sparkConf: SparkConf) extends Logging {
def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
this(args, new Configuration(), sparkConf)
def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
private var appAttemptId: ApplicationAttemptId = _
private var userThread: Thread = _
private val fs = FileSystem.get(yarnConf)
private var yarnAllocator: YarnAllocationHandler = _
private var isFinished: Boolean = false
private var uiAddress: String = _
private val maxAppAttempts: Int = conf.getInt(
YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
private var isLastAMRetry: Boolean = true
private var amClient: AMRMClient[ContainerRequest] = _
// Default to numExecutors * 2, with minimum of 3
private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures",
sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3)))
private var registered = false
def run() {
// Setup the directories so things go to YARN approved directories rather
// than user specified and /tmp.
System.setProperty("spark.local.dir", getLocalDirs())
// Set the web ui port to be ephemeral for yarn so we don't conflict with
// other spark processes running on the same box
System.setProperty("spark.ui.port", "0")
// When running the AM, the Spark master is always "yarn-cluster"
System.setProperty("spark.master", "yarn-cluster")
// Use priority 30 as it's higher than HDFS. It's the same priority MapReduce is using.
ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
appAttemptId = ApplicationMaster.getApplicationAttemptId()
logInfo("ApplicationAttemptId: " + appAttemptId)
isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
amClient = AMRMClient.createAMRMClient()
amClient.init(yarnConf)
amClient.start()
// setup AmIpFilter for the SparkUI - do this before we start the UI
addAmIpFilter()
ApplicationMaster.register(this)
// Call this to force generation of secret so it gets populated into the
// Hadoop UGI. This has to happen before the startUserClass which does a
// doAs in order for the credentials to be passed on to the executor containers.
val securityMgr = new SecurityManager(sparkConf)
// Start the user's JAR
userThread = startUserClass()
// This a bit hacky, but we need to wait until the spark.driver.port property has
// been set by the Thread executing the user class.
waitForSparkContextInitialized()
// Do this after Spark master is up and SparkContext is created so that we can register UI Url.
synchronized {
if (!isFinished) {
registerApplicationMaster()
registered = true
}
}
// Allocate all containers
allocateExecutors()
// Launch thread that will heartbeat to the RM so it won't think the app has died.
launchReporterThread()
// Wait for the user class to finish
userThread.join()
System.exit(0)
}
// add the yarn amIpFilter that Yarn requires for properly securing the UI
private def addAmIpFilter() {
val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter"
System.setProperty("spark.ui.filters", amFilter)
val proxy = WebAppUtils.getProxyHostAndPort(conf)
val parts : Array[String] = proxy.split(":")
val uriBase = "http://" + proxy +
System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV)
val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase
System.setProperty(
"spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", params)
}
// Get the Yarn approved local directories.
private def getLocalDirs(): String = {
// Hadoop 0.23 and 2.x have different Environment variable names for the
// local dirs, so lets check both. We assume one of the 2 is set.
// LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
.orElse(Option(System.getenv("LOCAL_DIRS")))
localDirs match {
case None => throw new Exception("Yarn local dirs can't be empty")
case Some(l) => l
}
}
private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
logInfo("Registering the ApplicationMaster")
amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
}
private def startUserClass(): Thread = {
logInfo("Starting the user JAR in a separate Thread")
System.setProperty("spark.executor.instances",args.numExecutors.toString)
val mainMethod = Class.forName(
args.userClass,
false,
Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
val t = new Thread {
override def run() {
var succeeded = false
try {
// Copy
val mainArgs = new Array[String](args.userArgs.size)
args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size)
mainMethod.invoke(null, mainArgs)
// Some apps have "System.exit(0)" at the end. The user thread will stop here unless
// it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED.
succeeded = true
} finally {
logDebug("Finishing main")
isLastAMRetry = true
if (succeeded) {
ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
} else {
ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED)
}
}
}
}
t.setName("Driver")
t.start()
t
}
// This needs to happen before allocateExecutors()
private def waitForSparkContextInitialized() {
logInfo("Waiting for Spark context initialization")
try {
var sparkContext: SparkContext = null
ApplicationMaster.sparkContextRef.synchronized {
var numTries = 0
val waitTime = 10000L
val maxNumTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10)
while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries
&& !isFinished) {
logInfo("Waiting for Spark context initialization ... " + numTries)
numTries = numTries + 1
ApplicationMaster.sparkContextRef.wait(waitTime)
}
sparkContext = ApplicationMaster.sparkContextRef.get()
assert(sparkContext != null || numTries >= maxNumTries)
if (sparkContext != null) {
uiAddress = sparkContext.ui.appUIHostPort
this.yarnAllocator = YarnAllocationHandler.newAllocator(
yarnConf,
amClient,
appAttemptId,
args,
sparkContext.preferredNodeLocationData,
sparkContext.getConf)
} else {
logWarning("Unable to retrieve SparkContext in spite of waiting for %d, maxNumTries = %d".
format(numTries * waitTime, maxNumTries))
this.yarnAllocator = YarnAllocationHandler.newAllocator(
yarnConf,
amClient,
appAttemptId,
args,
sparkContext.getConf)
}
}
} finally {
// In case of exceptions, etc - ensure that the loop in
// ApplicationMaster#sparkContextInitialized() breaks.
ApplicationMaster.doneWithInitialAllocations()
}
}
private def allocateExecutors() {
try {
logInfo("Requesting" + args.numExecutors + " executors.")
// Wait until all containers have launched
yarnAllocator.addResourceRequests(args.numExecutors)
yarnAllocator.allocateResources()
// Exits the loop if the user thread exits.
var iters = 0
while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) {
checkNumExecutorsFailed()
allocateMissingExecutor()
yarnAllocator.allocateResources()
if (iters == ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) {
ApplicationMaster.doneWithInitialAllocations()
}
Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL)
iters += 1
}
} finally {
// In case of exceptions, etc - ensure that the loop in
// ApplicationMaster#sparkContextInitialized() breaks.
ApplicationMaster.doneWithInitialAllocations()
}
logInfo("All executors have launched.")
}
private def allocateMissingExecutor() {
val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning -
yarnAllocator.getNumPendingAllocate
if (missingExecutorCount > 0) {
logInfo("Allocating %d containers to make up for (potentially) lost containers".
format(missingExecutorCount))
yarnAllocator.addResourceRequests(missingExecutorCount)
}
}
private def checkNumExecutorsFailed() {
if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) {
finishApplicationMaster(FinalApplicationStatus.FAILED,
"max number of executor failures reached")
}
}
private def launchReporterThread(): Thread = {
// Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
// we want to be reasonably responsive without causing too many requests to RM.
val schedulerInterval =
sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000)
// must be <= timeoutInterval / 2.
val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval))
val t = new Thread {
override def run() {
while (userThread.isAlive) {
checkNumExecutorsFailed()
allocateMissingExecutor()
logDebug("Sending progress")
yarnAllocator.allocateResources()
Thread.sleep(interval)
}
}
}
// Setting to daemon status, though this is usually not a good idea.
t.setDaemon(true)
t.start()
logInfo("Started progress reporter thread - heartbeat interval : " + interval)
t
}
def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
synchronized {
if (isFinished) {
return
}
isFinished = true
logInfo("Unregistering ApplicationMaster with " + status)
if (registered) {
val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "")
amClient.unregisterApplicationMaster(status, diagnostics, trackingUrl)
}
}
}
/**
* Clean up the staging directory.
*/
private def cleanupStagingDir() {
var stagingDirPath: Path = null
try {
val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean
if (!preserveFiles) {
stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
if (stagingDirPath == null) {
logError("Staging directory is null")
return
}
logInfo("Deleting staging directory " + stagingDirPath)
fs.delete(stagingDirPath, true)
}
} catch {
case ioe: IOException =>
logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
}
}
// The shutdown hook that runs when a signal is received AND during normal close of the JVM.
class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
def run() {
logInfo("AppMaster received a signal.")
// We need to clean up staging dir before HDFS is shut down
// make sure we don't delete it until this is the last AM
if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
}
}
}
object ApplicationMaster {
// Number of times to wait for the allocator loop to complete.
// Each loop iteration waits for 100ms, so maximum of 3 seconds.
// This is to ensure that we have reasonable number of containers before we start
// TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
// optimal as more containers are available. Might need to handle this better.
private val ALLOCATOR_LOOP_WAIT_COUNT = 30
private val ALLOCATE_HEARTBEAT_INTERVAL = 100
private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]()
val sparkContextRef: AtomicReference[SparkContext] =
new AtomicReference[SparkContext](null)
// Variable used to notify the YarnClusterScheduler that it should stop waiting
// for the initial set of executors to be started and get on with its business.
val doneWithInitialAllocationsMonitor = new Object()
@volatile var isDoneWithInitialAllocations = false
def doneWithInitialAllocations() {
isDoneWithInitialAllocations = true
doneWithInitialAllocationsMonitor.synchronized {
// to wake threads off wait ...
doneWithInitialAllocationsMonitor.notifyAll()
}
}
def register(master: ApplicationMaster) {
applicationMasters.add(master)
}
/**
* Called from YarnClusterScheduler to notify the AM code that a SparkContext has been
* initialized in the user code.
*/
def sparkContextInitialized(sc: SparkContext): Boolean = {
var modified = false
sparkContextRef.synchronized {
modified = sparkContextRef.compareAndSet(null, sc)
sparkContextRef.notifyAll()
}
// Add a shutdown hook - as a best effort in case users do not call sc.stop or do
// System.exit.
// Should not really have to do this, but it helps YARN to evict resources earlier.
// Not to mention, prevent the Client from declaring failure even though we exited properly.
// Note that this will unfortunately not properly clean up the staging files because it gets
// called too late, after the filesystem is already shutdown.
if (modified) {
Runtime.getRuntime().addShutdownHook(new Thread with Logging {
// This is not only logs, but also ensures that log system is initialized for this instance
// when we are actually 'run'-ing.
logInfo("Adding shutdown hook for context " + sc)
override def run() {
logInfo("Invoking sc stop from shutdown hook")
sc.stop()
// Best case ...
for (master <- applicationMasters) {
master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
}
}
})
}
// Wait for initialization to complete and at least 'some' nodes to get allocated.
modified
}
/**
* Returns when we've either
* 1) received all the requested executors,
* 2) waited ALLOCATOR_LOOP_WAIT_COUNT * ALLOCATE_HEARTBEAT_INTERVAL ms,
* 3) hit an error that causes us to terminate trying to get containers.
*/
def waitForInitialAllocations() {
doneWithInitialAllocationsMonitor.synchronized {
while (!isDoneWithInitialAllocations) {
doneWithInitialAllocationsMonitor.wait(1000L)
}
}
}
def getApplicationAttemptId(): ApplicationAttemptId = {
val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name())
val containerId = ConverterUtils.toContainerId(containerIdString)
val appAttemptId = containerId.getApplicationAttemptId()
appAttemptId
}
def main(argStrings: Array[String]) {
val args = new ApplicationMasterArguments(argStrings)
SparkHadoopUtil.get.runAsSparkUser { () =>
new ApplicationMaster(args).run()
}
}
}