/
Client.scala
1553 lines (1389 loc) · 63.9 KB
/
Client.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
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.{File, FileOutputStream, IOException, OutputStreamWriter}
import java.net.{InetAddress, UnknownHostException, URI}
import java.nio.ByteBuffer
import java.nio.charset.StandardCharsets
import java.util.{Properties, UUID}
import java.util.zip.{ZipEntry, ZipOutputStream}
import scala.collection.JavaConverters._
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map}
import scala.util.{Failure, Success, Try}
import scala.util.control.NonFatal
import com.google.common.base.Objects
import com.google.common.io.Files
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
import org.apache.hadoop.fs.permission.FsPermission
import org.apache.hadoop.io.DataOutputBuffer
import org.apache.hadoop.mapreduce.MRJobConfig
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
import org.apache.hadoop.util.StringUtils
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.hadoop.yarn.api.protocolrecords._
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication}
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException
import org.apache.hadoop.yarn.util.Records
import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.deploy.yarn.config._
import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
import org.apache.spark.util.{CallerContext, Utils}
private[spark] class Client(
val args: ClientArguments,
val hadoopConf: Configuration,
val sparkConf: SparkConf)
extends Logging {
import Client._
import YarnSparkHadoopUtil._
def this(clientArgs: ClientArguments, spConf: SparkConf) =
this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf)
private val yarnClient = YarnClient.createYarnClient
private val yarnConf = new YarnConfiguration(hadoopConf)
private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster"
// AM related configurations
private val amMemory = if (isClusterMode) {
sparkConf.get(DRIVER_MEMORY).toInt
} else {
sparkConf.get(AM_MEMORY).toInt
}
private val amMemoryOverhead = {
val amMemoryOverheadEntry = if (isClusterMode) DRIVER_MEMORY_OVERHEAD else AM_MEMORY_OVERHEAD
sparkConf.get(amMemoryOverheadEntry).getOrElse(
math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
}
private val amCores = if (isClusterMode) {
sparkConf.get(DRIVER_CORES)
} else {
sparkConf.get(AM_CORES)
}
// Executor related configurations
private val executorMemory = sparkConf.get(EXECUTOR_MEMORY)
private val executorMemoryOverhead = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse(
math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
private val distCacheMgr = new ClientDistributedCacheManager()
private var loginFromKeytab = false
private var principal: String = null
private var keytab: String = null
private var credentials: Credentials = null
private var amKeytabFileName: String = null
private val launcherBackend = new LauncherBackend() {
override def onStopRequest(): Unit = {
if (isClusterMode && appId != null) {
yarnClient.killApplication(appId)
} else {
setState(SparkAppHandle.State.KILLED)
stop()
}
}
}
private val fireAndForget = isClusterMode && !sparkConf.get(WAIT_FOR_APP_COMPLETION)
private var appId: ApplicationId = null
// The app staging dir based on the STAGING_DIR configuration if configured
// otherwise based on the users home directory.
private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) }
.getOrElse(FileSystem.get(hadoopConf).getHomeDirectory())
private val credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
def reportLauncherState(state: SparkAppHandle.State): Unit = {
launcherBackend.setState(state)
}
def stop(): Unit = {
launcherBackend.close()
yarnClient.stop()
// Unset YARN mode system env variable, to allow switching between cluster types.
System.clearProperty("SPARK_YARN_MODE")
}
/**
* Submit an application running our ApplicationMaster to the ResourceManager.
*
* The stable Yarn API provides a convenience method (YarnClient#createApplication) for
* creating applications and setting up the application submission context. This was not
* available in the alpha API.
*/
def submitApplication(): ApplicationId = {
var appId: ApplicationId = null
try {
launcherBackend.connect()
// Setup the credentials before doing anything else,
// so we have don't have issues at any point.
setupCredentials()
yarnClient.init(yarnConf)
yarnClient.start()
logInfo("Requesting a new application from cluster with %d NodeManagers"
.format(yarnClient.getYarnClusterMetrics.getNumNodeManagers))
// Get a new application from our RM
val newApp = yarnClient.createApplication()
val newAppResponse = newApp.getNewApplicationResponse()
appId = newAppResponse.getApplicationId()
reportLauncherState(SparkAppHandle.State.SUBMITTED)
launcherBackend.setAppId(appId.toString)
new CallerContext("CLIENT", Option(appId.toString)).setCurrentContext()
// Verify whether the cluster has enough resources for our AM
verifyClusterResources(newAppResponse)
// Set up the appropriate contexts to launch our AM
val containerContext = createContainerLaunchContext(newAppResponse)
val appContext = createApplicationSubmissionContext(newApp, containerContext)
// Finally, submit and monitor the application
logInfo(s"Submitting application $appId to ResourceManager")
yarnClient.submitApplication(appContext)
appId
} catch {
case e: Throwable =>
if (appId != null) {
cleanupStagingDir(appId)
}
throw e
}
}
/**
* Cleanup application staging directory.
*/
private def cleanupStagingDir(appId: ApplicationId): Unit = {
val stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
try {
val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES)
val fs = stagingDirPath.getFileSystem(hadoopConf)
if (!preserveFiles && fs.delete(stagingDirPath, true)) {
logInfo(s"Deleted staging directory $stagingDirPath")
}
} catch {
case ioe: IOException =>
logWarning("Failed to cleanup staging dir " + stagingDirPath, ioe)
}
}
/**
* Set up the context for submitting our ApplicationMaster.
* This uses the YarnClientApplication not available in the Yarn alpha API.
*/
def createApplicationSubmissionContext(
newApp: YarnClientApplication,
containerContext: ContainerLaunchContext): ApplicationSubmissionContext = {
val appContext = newApp.getApplicationSubmissionContext
appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark"))
appContext.setQueue(sparkConf.get(QUEUE_NAME))
appContext.setAMContainerSpec(containerContext)
appContext.setApplicationType("SPARK")
sparkConf.get(APPLICATION_TAGS).foreach { tags =>
try {
// The setApplicationTags method was only introduced in Hadoop 2.4+, so we need to use
// reflection to set it, printing a warning if a tag was specified but the YARN version
// doesn't support it.
val method = appContext.getClass().getMethod(
"setApplicationTags", classOf[java.util.Set[String]])
method.invoke(appContext, new java.util.HashSet[String](tags.asJava))
} catch {
case e: NoSuchMethodException =>
logWarning(s"Ignoring ${APPLICATION_TAGS.key} because this version of " +
"YARN does not support it")
}
}
sparkConf.get(MAX_APP_ATTEMPTS) match {
case Some(v) => appContext.setMaxAppAttempts(v)
case None => logDebug(s"${MAX_APP_ATTEMPTS.key} is not set. " +
"Cluster's default value will be used.")
}
sparkConf.get(AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).foreach { interval =>
try {
val method = appContext.getClass().getMethod(
"setAttemptFailuresValidityInterval", classOf[Long])
method.invoke(appContext, interval: java.lang.Long)
} catch {
case e: NoSuchMethodException =>
logWarning(s"Ignoring ${AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS.key} because " +
"the version of YARN does not support it")
}
}
val capability = Records.newRecord(classOf[Resource])
capability.setMemory(amMemory + amMemoryOverhead)
capability.setVirtualCores(amCores)
sparkConf.get(AM_NODE_LABEL_EXPRESSION) match {
case Some(expr) =>
try {
val amRequest = Records.newRecord(classOf[ResourceRequest])
amRequest.setResourceName(ResourceRequest.ANY)
amRequest.setPriority(Priority.newInstance(0))
amRequest.setCapability(capability)
amRequest.setNumContainers(1)
val method = amRequest.getClass.getMethod("setNodeLabelExpression", classOf[String])
method.invoke(amRequest, expr)
val setResourceRequestMethod =
appContext.getClass.getMethod("setAMContainerResourceRequest", classOf[ResourceRequest])
setResourceRequestMethod.invoke(appContext, amRequest)
} catch {
case e: NoSuchMethodException =>
logWarning(s"Ignoring ${AM_NODE_LABEL_EXPRESSION.key} because the version " +
"of YARN does not support it")
appContext.setResource(capability)
}
case None =>
appContext.setResource(capability)
}
sparkConf.get(ROLLED_LOG_INCLUDE_PATTERN).foreach { includePattern =>
try {
val logAggregationContext = Records.newRecord(
Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
.asInstanceOf[Object]
val setRolledLogsIncludePatternMethod =
logAggregationContext.getClass.getMethod("setRolledLogsIncludePattern", classOf[String])
setRolledLogsIncludePatternMethod.invoke(logAggregationContext, includePattern)
sparkConf.get(ROLLED_LOG_EXCLUDE_PATTERN).foreach { excludePattern =>
val setRolledLogsExcludePatternMethod =
logAggregationContext.getClass.getMethod("setRolledLogsExcludePattern", classOf[String])
setRolledLogsExcludePatternMethod.invoke(logAggregationContext, excludePattern)
}
val setLogAggregationContextMethod =
appContext.getClass.getMethod("setLogAggregationContext",
Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
setLogAggregationContextMethod.invoke(appContext, logAggregationContext)
} catch {
case NonFatal(e) =>
logWarning(s"Ignoring ${ROLLED_LOG_INCLUDE_PATTERN.key} because the version of YARN " +
s"does not support it", e)
}
}
appContext
}
/** Set up security tokens for launching our ApplicationMaster container. */
private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = {
val dob = new DataOutputBuffer
credentials.writeTokenStorageToStream(dob)
amContainer.setTokens(ByteBuffer.wrap(dob.getData))
}
/** Get the application report from the ResourceManager for an application we have submitted. */
def getApplicationReport(appId: ApplicationId): ApplicationReport =
yarnClient.getApplicationReport(appId)
/**
* Return the security token used by this client to communicate with the ApplicationMaster.
* If no security is enabled, the token returned by the report is null.
*/
private def getClientToken(report: ApplicationReport): String =
Option(report.getClientToAMToken).map(_.toString).getOrElse("")
/**
* Fail fast if we have requested more resources per container than is available in the cluster.
*/
private def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = {
val maxMem = newAppResponse.getMaximumResourceCapability().getMemory()
logInfo("Verifying our application has not requested more than the maximum " +
s"memory capability of the cluster ($maxMem MB per container)")
val executorMem = executorMemory + executorMemoryOverhead
if (executorMem > maxMem) {
throw new IllegalArgumentException(s"Required executor memory ($executorMemory" +
s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
"Please check the values of 'yarn.scheduler.maximum-allocation-mb' and/or " +
"'yarn.nodemanager.resource.memory-mb'.")
}
val amMem = amMemory + amMemoryOverhead
if (amMem > maxMem) {
throw new IllegalArgumentException(s"Required AM memory ($amMemory" +
s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
"Please increase the value of 'yarn.scheduler.maximum-allocation-mb'.")
}
logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format(
amMem,
amMemoryOverhead))
// We could add checks to make sure the entire cluster has enough resources but that involves
// getting all the node reports and computing ourselves.
}
/**
* Copy the given file to a remote file system (e.g. HDFS) if needed.
* The file is only copied if the source and destination file systems are different. This is used
* for preparing resources for launching the ApplicationMaster container. Exposed for testing.
*/
private[yarn] def copyFileToRemote(
destDir: Path,
srcPath: Path,
replication: Short,
symlinkCache: Map[URI, Path],
force: Boolean = false,
destName: Option[String] = None): Path = {
val destFs = destDir.getFileSystem(hadoopConf)
val srcFs = srcPath.getFileSystem(hadoopConf)
var destPath = srcPath
if (force || !compareFs(srcFs, destFs)) {
destPath = new Path(destDir, destName.getOrElse(srcPath.getName()))
logInfo(s"Uploading resource $srcPath -> $destPath")
FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf)
destFs.setReplication(destPath, replication)
destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION))
} else {
logInfo(s"Source and destination file systems are the same. Not copying $srcPath")
}
// Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
// version shows the specific version in the distributed cache configuration
val qualifiedDestPath = destFs.makeQualified(destPath)
val qualifiedDestDir = qualifiedDestPath.getParent
val resolvedDestDir = symlinkCache.getOrElseUpdate(qualifiedDestDir.toUri(), {
val fc = FileContext.getFileContext(qualifiedDestDir.toUri(), hadoopConf)
fc.resolvePath(qualifiedDestDir)
})
new Path(resolvedDestDir, qualifiedDestPath.getName())
}
/**
* Upload any resources to the distributed cache if needed. If a resource is intended to be
* consumed locally, set up the appropriate config for downstream code to handle it properly.
* This is used for setting up a container launch context for our ApplicationMaster.
* Exposed for testing.
*/
def prepareLocalResources(
destDir: Path,
pySparkArchives: Seq[String]): HashMap[String, LocalResource] = {
logInfo("Preparing resources for our AM container")
// Upload Spark and the application JAR to the remote file system if necessary,
// and add them as local resources to the application master.
val fs = destDir.getFileSystem(hadoopConf)
// Merge credentials obtained from registered providers
val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(hadoopConf, credentials)
if (credentials != null) {
// Add credentials to current user's UGI, so that following operations don't need to use the
// Kerberos tgt to get delegations again in the client side.
UserGroupInformation.getCurrentUser.addCredentials(credentials)
logDebug(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n"))
}
// If we use principal and keytab to login, also credentials can be renewed some time
// after current time, we should pass the next renewal and updating time to credential
// renewer and updater.
if (loginFromKeytab && nearestTimeOfNextRenewal > System.currentTimeMillis() &&
nearestTimeOfNextRenewal != Long.MaxValue) {
// Valid renewal time is 75% of next renewal time, and the valid update time will be
// slightly later then renewal time (80% of next renewal time). This is to make sure
// credentials are renewed and updated before expired.
val currTime = System.currentTimeMillis()
val renewalTime = (nearestTimeOfNextRenewal - currTime) * 0.75 + currTime
val updateTime = (nearestTimeOfNextRenewal - currTime) * 0.8 + currTime
sparkConf.set(CREDENTIALS_RENEWAL_TIME, renewalTime.toLong)
sparkConf.set(CREDENTIALS_UPDATE_TIME, updateTime.toLong)
}
// Used to keep track of URIs added to the distributed cache. If the same URI is added
// multiple times, YARN will fail to launch containers for the app with an internal
// error.
val distributedUris = new HashSet[String]
// Used to keep track of URIs(files) added to the distribute cache have the same name. If
// same name but different path files are added multiple time, YARN will fail to launch
// containers for the app with an internal error.
val distributedNames = new HashSet[String]
val replication = sparkConf.get(STAGING_FILE_REPLICATION).map(_.toShort)
.getOrElse(fs.getDefaultReplication(destDir))
val localResources = HashMap[String, LocalResource]()
FileSystem.mkdirs(fs, destDir, new FsPermission(STAGING_DIR_PERMISSION))
val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
val symlinkCache: Map[URI, Path] = HashMap[URI, Path]()
def addDistributedUri(uri: URI): Boolean = {
val uriStr = uri.toString()
val fileName = new File(uri.getPath).getName
if (distributedUris.contains(uriStr)) {
logWarning(s"Same path resource $uri added multiple times to distributed cache.")
false
} else if (distributedNames.contains(fileName)) {
logWarning(s"Same name resource $uri added multiple times to distributed cache")
false
} else {
distributedUris += uriStr
distributedNames += fileName
true
}
}
/**
* Distribute a file to the cluster.
*
* If the file's path is a "local:" URI, it's actually not distributed. Other files are copied
* to HDFS (if not already there) and added to the application's distributed cache.
*
* @param path URI of the file to distribute.
* @param resType Type of resource being distributed.
* @param destName Name of the file in the distributed cache.
* @param targetDir Subdirectory where to place the file.
* @param appMasterOnly Whether to distribute only to the AM.
* @return A 2-tuple. First item is whether the file is a "local:" URI. Second item is the
* localized path for non-local paths, or the input `path` for local paths.
* The localized path will be null if the URI has already been added to the cache.
*/
def distribute(
path: String,
resType: LocalResourceType = LocalResourceType.FILE,
destName: Option[String] = None,
targetDir: Option[String] = None,
appMasterOnly: Boolean = false): (Boolean, String) = {
val trimmedPath = path.trim()
val localURI = Utils.resolveURI(trimmedPath)
if (localURI.getScheme != LOCAL_SCHEME) {
if (addDistributedUri(localURI)) {
val localPath = getQualifiedLocalPath(localURI, hadoopConf)
val linkname = targetDir.map(_ + "/").getOrElse("") +
destName.orElse(Option(localURI.getFragment())).getOrElse(localPath.getName())
val destPath = copyFileToRemote(destDir, localPath, replication, symlinkCache)
val destFs = FileSystem.get(destPath.toUri(), hadoopConf)
distCacheMgr.addResource(
destFs, hadoopConf, destPath, localResources, resType, linkname, statCache,
appMasterOnly = appMasterOnly)
(false, linkname)
} else {
(false, null)
}
} else {
(true, trimmedPath)
}
}
// If we passed in a keytab, make sure we copy the keytab to the staging directory on
// HDFS, and setup the relevant environment vars, so the AM can login again.
if (loginFromKeytab) {
logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" +
" via the YARN Secure Distributed Cache.")
val (_, localizedPath) = distribute(keytab,
destName = Some(amKeytabFileName),
appMasterOnly = true)
require(localizedPath != null, "Keytab file already distributed.")
}
/**
* Add Spark to the cache. There are two settings that control what files to add to the cache:
* - if a Spark archive is defined, use the archive. The archive is expected to contain
* jar files at its root directory.
* - if a list of jars is provided, filter the non-local ones, resolve globs, and
* add the found files to the cache.
*
* Note that the archive cannot be a "local" URI. If none of the above settings are found,
* then upload all files found in $SPARK_HOME/jars.
*/
val sparkArchive = sparkConf.get(SPARK_ARCHIVE)
if (sparkArchive.isDefined) {
val archive = sparkArchive.get
require(!isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.")
distribute(Utils.resolveURI(archive).toString,
resType = LocalResourceType.ARCHIVE,
destName = Some(LOCALIZED_LIB_DIR))
} else {
sparkConf.get(SPARK_JARS) match {
case Some(jars) =>
// Break the list of jars to upload, and resolve globs.
val localJars = new ArrayBuffer[String]()
jars.foreach { jar =>
if (!isLocalUri(jar)) {
val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf)
val pathFs = FileSystem.get(path.toUri(), hadoopConf)
pathFs.globStatus(path).filter(_.isFile()).foreach { entry =>
val uri = entry.getPath().toUri()
statCache.update(uri, entry)
distribute(uri.toString(), targetDir = Some(LOCALIZED_LIB_DIR))
}
} else {
localJars += jar
}
}
// Propagate the local URIs to the containers using the configuration.
sparkConf.set(SPARK_JARS, localJars)
case None =>
// No configuration, so fall back to uploading local jar files.
logWarning(s"Neither ${SPARK_JARS.key} nor ${SPARK_ARCHIVE.key} is set, falling back " +
"to uploading libraries under SPARK_HOME.")
val jarsDir = new File(YarnCommandBuilderUtils.findJarsDir(
sparkConf.getenv("SPARK_HOME")))
val jarsArchive = File.createTempFile(LOCALIZED_LIB_DIR, ".zip",
new File(Utils.getLocalDir(sparkConf)))
val jarsStream = new ZipOutputStream(new FileOutputStream(jarsArchive))
try {
jarsStream.setLevel(0)
jarsDir.listFiles().foreach { f =>
if (f.isFile && f.getName.toLowerCase().endsWith(".jar") && f.canRead) {
jarsStream.putNextEntry(new ZipEntry(f.getName))
Files.copy(f, jarsStream)
jarsStream.closeEntry()
}
}
} finally {
jarsStream.close()
}
distribute(jarsArchive.toURI.getPath,
resType = LocalResourceType.ARCHIVE,
destName = Some(LOCALIZED_LIB_DIR))
}
}
/**
* Copy user jar to the distributed cache if their scheme is not "local".
* Otherwise, set the corresponding key in our SparkConf to handle it downstream.
*/
Option(args.userJar).filter(_.trim.nonEmpty).foreach { jar =>
val (isLocal, localizedPath) = distribute(jar, destName = Some(APP_JAR_NAME))
if (isLocal) {
require(localizedPath != null, s"Path $jar already distributed")
// If the resource is intended for local use only, handle this downstream
// by setting the appropriate property
sparkConf.set(APP_JAR, localizedPath)
}
}
/**
* Do the same for any additional resources passed in through ClientArguments.
* Each resource category is represented by a 3-tuple of:
* (1) comma separated list of resources in this category,
* (2) resource type, and
* (3) whether to add these resources to the classpath
*/
val cachedSecondaryJarLinks = ListBuffer.empty[String]
List(
(sparkConf.get(JARS_TO_DISTRIBUTE), LocalResourceType.FILE, true),
(sparkConf.get(FILES_TO_DISTRIBUTE), LocalResourceType.FILE, false),
(sparkConf.get(ARCHIVES_TO_DISTRIBUTE), LocalResourceType.ARCHIVE, false)
).foreach { case (flist, resType, addToClasspath) =>
flist.foreach { file =>
val (_, localizedPath) = distribute(file, resType = resType)
// If addToClassPath, we ignore adding jar multiple times to distitrbuted cache.
if (addToClasspath) {
if (localizedPath != null) {
cachedSecondaryJarLinks += localizedPath
}
} else {
if (localizedPath == null) {
throw new IllegalArgumentException(s"Attempt to add ($file) multiple times" +
" to the distributed cache.")
}
}
}
}
if (cachedSecondaryJarLinks.nonEmpty) {
sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks)
}
if (isClusterMode && args.primaryPyFile != null) {
distribute(args.primaryPyFile, appMasterOnly = true)
}
pySparkArchives.foreach { f => distribute(f) }
// The python files list needs to be treated especially. All files that are not an
// archive need to be placed in a subdirectory that will be added to PYTHONPATH.
sparkConf.get(PY_FILES).foreach { f =>
val targetDir = if (f.endsWith(".py")) Some(LOCALIZED_PYTHON_DIR) else None
distribute(f, targetDir = targetDir)
}
// Update the configuration with all the distributed files, minus the conf archive. The
// conf archive will be handled by the AM differently so that we avoid having to send
// this configuration by other means. See SPARK-14602 for one reason of why this is needed.
distCacheMgr.updateConfiguration(sparkConf)
// Upload the conf archive to HDFS manually, and record its location in the configuration.
// This will allow the AM to know where the conf archive is in HDFS, so that it can be
// distributed to the containers.
//
// This code forces the archive to be copied, so that unit tests pass (since in that case both
// file systems are the same and the archive wouldn't normally be copied). In most (all?)
// deployments, the archive would be copied anyway, since it's a temp file in the local file
// system.
val remoteConfArchivePath = new Path(destDir, LOCALIZED_CONF_ARCHIVE)
val remoteFs = FileSystem.get(remoteConfArchivePath.toUri(), hadoopConf)
sparkConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString())
val localConfArchive = new Path(createConfArchive().toURI())
copyFileToRemote(destDir, localConfArchive, replication, symlinkCache, force = true,
destName = Some(LOCALIZED_CONF_ARCHIVE))
// Manually add the config archive to the cache manager so that the AM is launched with
// the proper files set up.
distCacheMgr.addResource(
remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE,
LOCALIZED_CONF_DIR, statCache, appMasterOnly = false)
// Clear the cache-related entries from the configuration to avoid them polluting the
// UI's environment page. This works for client mode; for cluster mode, this is handled
// by the AM.
CACHE_CONFIGS.foreach(sparkConf.remove)
localResources
}
/**
* Create an archive with the config files for distribution.
*
* These will be used by AM and executors. The files are zipped and added to the job as an
* archive, so that YARN will explode it when distributing to AM and executors. This directory
* is then added to the classpath of AM and executor process, just to make sure that everybody
* is using the same default config.
*
* This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR
* shows up in the classpath before YARN_CONF_DIR.
*
* Currently this makes a shallow copy of the conf directory. If there are cases where a
* Hadoop config directory contains subdirectories, this code will have to be fixed.
*
* The archive also contains some Spark configuration. Namely, it saves the contents of
* SparkConf in a file to be loaded by the AM process.
*/
private def createConfArchive(): File = {
val hadoopConfFiles = new HashMap[String, File]()
// Uploading $SPARK_CONF_DIR/log4j.properties file to the distributed cache to make sure that
// the executors will use the latest configurations instead of the default values. This is
// required when user changes log4j.properties directly to set the log configurations. If
// configuration file is provided through --files then executors will be taking configurations
// from --files instead of $SPARK_CONF_DIR/log4j.properties.
// Also uploading metrics.properties to distributed cache if exists in classpath.
// If user specify this file using --files then executors will use the one
// from --files instead.
for { prop <- Seq("log4j.properties", "metrics.properties")
url <- Option(Utils.getContextOrSparkClassLoader.getResource(prop))
if url.getProtocol == "file" } {
hadoopConfFiles(prop) = new File(url.getPath)
}
Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
sys.env.get(envKey).foreach { path =>
val dir = new File(path)
if (dir.isDirectory()) {
val files = dir.listFiles()
if (files == null) {
logWarning("Failed to list files under directory " + dir)
} else {
files.foreach { file =>
if (file.isFile && !hadoopConfFiles.contains(file.getName())) {
hadoopConfFiles(file.getName()) = file
}
}
}
}
}
}
val confArchive = File.createTempFile(LOCALIZED_CONF_DIR, ".zip",
new File(Utils.getLocalDir(sparkConf)))
val confStream = new ZipOutputStream(new FileOutputStream(confArchive))
try {
confStream.setLevel(0)
hadoopConfFiles.foreach { case (name, file) =>
if (file.canRead()) {
confStream.putNextEntry(new ZipEntry(name))
Files.copy(file, confStream)
confStream.closeEntry()
}
}
// Save Spark configuration to a file in the archive.
val props = new Properties()
sparkConf.getAll.foreach { case (k, v) => props.setProperty(k, v) }
// Override spark.yarn.key to point to the location in distributed cache which will be used
// by AM.
Option(amKeytabFileName).foreach { k => props.setProperty(KEYTAB.key, k) }
confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE))
val writer = new OutputStreamWriter(confStream, StandardCharsets.UTF_8)
props.store(writer, "Spark configuration.")
writer.flush()
confStream.closeEntry()
} finally {
confStream.close()
}
confArchive
}
/**
* Set up the environment for launching our ApplicationMaster container.
*/
private def setupLaunchEnv(
stagingDirPath: Path,
pySparkArchives: Seq[String]): HashMap[String, String] = {
logInfo("Setting up the launch environment for our AM container")
val env = new HashMap[String, String]()
populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH))
env("SPARK_YARN_MODE") = "true"
env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString
env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName()
if (loginFromKeytab) {
val credentialsFile = "credentials-" + UUID.randomUUID().toString
sparkConf.set(CREDENTIALS_FILE_PATH, new Path(stagingDirPath, credentialsFile).toString)
logInfo(s"Credentials file set to: $credentialsFile")
}
// Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.*
val amEnvPrefix = "spark.yarn.appMasterEnv."
sparkConf.getAll
.filter { case (k, v) => k.startsWith(amEnvPrefix) }
.map { case (k, v) => (k.substring(amEnvPrefix.length), v) }
.foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) }
// Keep this for backwards compatibility but users should move to the config
sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
// Allow users to specify some environment variables.
YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
// Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments.
env("SPARK_YARN_USER_ENV") = userEnvs
}
// If pyFiles contains any .py files, we need to add LOCALIZED_PYTHON_DIR to the PYTHONPATH
// of the container processes too. Add all non-.py files directly to PYTHONPATH.
//
// NOTE: the code currently does not handle .py files defined with a "local:" scheme.
val pythonPath = new ListBuffer[String]()
val (pyFiles, pyArchives) = sparkConf.get(PY_FILES).partition(_.endsWith(".py"))
if (pyFiles.nonEmpty) {
pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
LOCALIZED_PYTHON_DIR)
}
(pySparkArchives ++ pyArchives).foreach { path =>
val uri = Utils.resolveURI(path)
if (uri.getScheme != LOCAL_SCHEME) {
pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
new Path(uri).getName())
} else {
pythonPath += uri.getPath()
}
}
// Finally, update the Spark config to propagate PYTHONPATH to the AM and executors.
if (pythonPath.nonEmpty) {
val pythonPathStr = (sys.env.get("PYTHONPATH") ++ pythonPath)
.mkString(YarnSparkHadoopUtil.getClassPathSeparator)
env("PYTHONPATH") = pythonPathStr
sparkConf.setExecutorEnv("PYTHONPATH", pythonPathStr)
}
// In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to
// executors. But we can't just set spark.executor.extraJavaOptions, because the driver's
// SparkContext will not let that set spark* system properties, which is expected behavior for
// Yarn clients. So propagate it through the environment.
//
// Note that to warn the user about the deprecation in cluster mode, some code from
// SparkConf#validateSettings() is duplicated here (to avoid triggering the condition
// described above).
if (isClusterMode) {
sys.env.get("SPARK_JAVA_OPTS").foreach { value =>
val warning =
s"""
|SPARK_JAVA_OPTS was detected (set to '$value').
|This is deprecated in Spark 1.0+.
|
|Please instead use:
| - ./spark-submit with conf/spark-defaults.conf to set defaults for an application
| - ./spark-submit with --driver-java-options to set -X options for a driver
| - spark.executor.extraJavaOptions to set -X options for executors
""".stripMargin
logWarning(warning)
for (proc <- Seq("driver", "executor")) {
val key = s"spark.$proc.extraJavaOptions"
if (sparkConf.contains(key)) {
throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.")
}
}
env("SPARK_JAVA_OPTS") = value
}
// propagate PYSPARK_DRIVER_PYTHON and PYSPARK_PYTHON to driver in cluster mode
Seq("PYSPARK_DRIVER_PYTHON", "PYSPARK_PYTHON").foreach { envname =>
if (!env.contains(envname)) {
sys.env.get(envname).foreach(env(envname) = _)
}
}
}
sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp =>
env(ENV_DIST_CLASSPATH) = dcp
}
env
}
/**
* Set up a ContainerLaunchContext to launch our ApplicationMaster container.
* This sets up the launch environment, java options, and the command for launching the AM.
*/
private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse)
: ContainerLaunchContext = {
logInfo("Setting up container launch context for our AM")
val appId = newAppResponse.getApplicationId
val appStagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
val pySparkArchives =
if (sparkConf.get(IS_PYTHON_APP)) {
findPySparkArchives()
} else {
Nil
}
val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives)
val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives)
val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
amContainer.setLocalResources(localResources.asJava)
amContainer.setEnvironment(launchEnv.asJava)
val javaOpts = ListBuffer[String]()
// Set the environment variable through a command prefix
// to append to the existing value of the variable
var prefixEnv: Option[String] = None
// Add Xmx for AM memory
javaOpts += "-Xmx" + amMemory + "m"
val tmpDir = new Path(
YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR
)
javaOpts += "-Djava.io.tmpdir=" + tmpDir
// TODO: Remove once cpuset version is pushed out.
// The context is, default gc for server class machines ends up using all cores to do gc -
// hence if there are multiple containers in same node, Spark GC affects all other containers'
// performance (which can be that of other Spark containers)
// Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
// multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
// of cores on a node.
val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean)
if (useConcurrentAndIncrementalGC) {
// In our expts, using (default) throughput collector has severe perf ramifications in
// multi-tenant machines
javaOpts += "-XX:+UseConcMarkSweepGC"
javaOpts += "-XX:MaxTenuringThreshold=31"
javaOpts += "-XX:SurvivorRatio=8"
javaOpts += "-XX:+CMSIncrementalMode"
javaOpts += "-XX:+CMSIncrementalPacing"
javaOpts += "-XX:CMSIncrementalDutyCycleMin=0"
javaOpts += "-XX:CMSIncrementalDutyCycle=10"
}
// Include driver-specific java options if we are launching a driver
if (isClusterMode) {
val driverOpts = sparkConf.get(DRIVER_JAVA_OPTIONS).orElse(sys.env.get("SPARK_JAVA_OPTS"))
driverOpts.foreach { opts =>
javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
}
val libraryPaths = Seq(sparkConf.get(DRIVER_LIBRARY_PATH),
sys.props.get("spark.driver.libraryPath")).flatten
if (libraryPaths.nonEmpty) {
prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(libraryPaths)))
}
if (sparkConf.get(AM_JAVA_OPTIONS).isDefined) {
logWarning(s"${AM_JAVA_OPTIONS.key} will not take effect in cluster mode")
}
} else {
// Validate and include yarn am specific java options in yarn-client mode.
sparkConf.get(AM_JAVA_OPTIONS).foreach { opts =>
if (opts.contains("-Dspark")) {
val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to set Spark options (was '$opts')."
throw new SparkException(msg)
}
if (opts.contains("-Xmx")) {
val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to specify max heap memory settings " +
s"(was '$opts'). Use spark.yarn.am.memory instead."
throw new SparkException(msg)
}
javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
}
sparkConf.get(AM_LIBRARY_PATH).foreach { paths =>
prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(paths))))
}
}
// For log4j configuration to reference
javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
val userClass =
if (isClusterMode) {
Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass))
} else {
Nil
}
val userJar =
if (args.userJar != null) {
Seq("--jar", args.userJar)
} else {
Nil
}
val primaryPyFile =
if (isClusterMode && args.primaryPyFile != null) {
Seq("--primary-py-file", new Path(args.primaryPyFile).getName())
} else {
Nil
}
val primaryRFile =
if (args.primaryRFile != null) {
Seq("--primary-r-file", args.primaryRFile)
} else {
Nil
}
val amClass =
if (isClusterMode) {
Utils.classForName("org.apache.spark.deploy.yarn.ApplicationMaster").getName
} else {
Utils.classForName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName
}
if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs
}
val userArgs = args.userArgs.flatMap { arg =>
Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg))
}
val amArgs =
Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ primaryRFile ++
userArgs ++ Seq(
"--properties-file", buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
LOCALIZED_CONF_DIR, SPARK_CONF_FILE))