@@ -212,14 +212,20 @@ object SparkSubmit extends CommandLineUtils {
212
212
213
213
/**
214
214
* Prepare the environment for submitting an application.
215
- * This returns a 4-tuple:
216
- * (1) the arguments for the child process,
217
- * (2) a list of classpath entries for the child,
218
- * (3) a map of system properties, and
219
- * (4) the main class for the child
215
+ *
216
+ * @param args the parsed SparkSubmitArguments used for environment preparation.
217
+ * @param conf the Hadoop Configuration, this argument will only be set in unit test.
218
+ * @return a 4-tuple:
219
+ * (1) the arguments for the child process,
220
+ * (2) a list of classpath entries for the child,
221
+ * (3) a map of system properties, and
222
+ * (4) the main class for the child
223
+ *
220
224
* Exposed for testing.
221
225
*/
222
- private [deploy] def prepareSubmitEnvironment (args : SparkSubmitArguments )
226
+ private [deploy] def prepareSubmitEnvironment (
227
+ args : SparkSubmitArguments ,
228
+ conf : Option [HadoopConfiguration ] = None )
223
229
: (Seq [String ], Seq [String ], Map [String , String ], String ) = {
224
230
// Return values
225
231
val childArgs = new ArrayBuffer [String ]()
@@ -322,7 +328,7 @@ object SparkSubmit extends CommandLineUtils {
322
328
}
323
329
}
324
330
325
- val hadoopConf = new HadoopConfiguration ()
331
+ val hadoopConf = conf.getOrElse( new HadoopConfiguration () )
326
332
val targetDir = DependencyUtils .createTempDir()
327
333
328
334
// Resolve glob path for different resources.
@@ -332,19 +338,21 @@ object SparkSubmit extends CommandLineUtils {
332
338
args.archives = Option (args.archives).map(resolveGlobPaths(_, hadoopConf)).orNull
333
339
334
340
// In client mode, download remote files.
341
+ var localPrimaryResource : String = null
342
+ var localJars : String = null
343
+ var localPyFiles : String = null
335
344
if (deployMode == CLIENT ) {
336
- args.primaryResource = Option (args.primaryResource).map {
345
+ localPrimaryResource = Option (args.primaryResource).map {
337
346
downloadFile(_, targetDir, args.sparkProperties, hadoopConf)
338
347
}.orNull
339
- args.jars = Option (args.jars).map {
348
+ localJars = Option (args.jars).map {
340
349
downloadFileList(_, targetDir, args.sparkProperties, hadoopConf)
341
350
}.orNull
342
- args.pyFiles = Option (args.pyFiles).map {
351
+ localPyFiles = Option (args.pyFiles).map {
343
352
downloadFileList(_, targetDir, args.sparkProperties, hadoopConf)
344
353
}.orNull
345
354
}
346
355
347
-
348
356
// If we're running a python app, set the main class to our specific python runner
349
357
if (args.isPython && deployMode == CLIENT ) {
350
358
if (args.primaryResource == PYSPARK_SHELL ) {
@@ -353,7 +361,7 @@ object SparkSubmit extends CommandLineUtils {
353
361
// If a python file is provided, add it to the child arguments and list of files to deploy.
354
362
// Usage: PythonAppRunner <main python file> <extra python files> [app arguments]
355
363
args.mainClass = " org.apache.spark.deploy.PythonRunner"
356
- args.childArgs = ArrayBuffer (args.primaryResource, args.pyFiles ) ++ args.childArgs
364
+ args.childArgs = ArrayBuffer (localPrimaryResource, localPyFiles ) ++ args.childArgs
357
365
if (clusterManager != YARN ) {
358
366
// The YARN backend distributes the primary file differently, so don't merge it.
359
367
args.files = mergeFileLists(args.files, args.primaryResource)
@@ -363,8 +371,8 @@ object SparkSubmit extends CommandLineUtils {
363
371
// The YARN backend handles python files differently, so don't merge the lists.
364
372
args.files = mergeFileLists(args.files, args.pyFiles)
365
373
}
366
- if (args.pyFiles != null ) {
367
- sysProps(" spark.submit.pyFiles" ) = args.pyFiles
374
+ if (localPyFiles != null ) {
375
+ sysProps(" spark.submit.pyFiles" ) = localPyFiles
368
376
}
369
377
}
370
378
@@ -418,7 +426,7 @@ object SparkSubmit extends CommandLineUtils {
418
426
// If an R file is provided, add it to the child arguments and list of files to deploy.
419
427
// Usage: RRunner <main R file> [app arguments]
420
428
args.mainClass = " org.apache.spark.deploy.RRunner"
421
- args.childArgs = ArrayBuffer (args.primaryResource ) ++ args.childArgs
429
+ args.childArgs = ArrayBuffer (localPrimaryResource ) ++ args.childArgs
422
430
args.files = mergeFileLists(args.files, args.primaryResource)
423
431
}
424
432
}
@@ -463,6 +471,7 @@ object SparkSubmit extends CommandLineUtils {
463
471
OptionAssigner (args.queue, YARN , ALL_DEPLOY_MODES , sysProp = " spark.yarn.queue" ),
464
472
OptionAssigner (args.numExecutors, YARN , ALL_DEPLOY_MODES ,
465
473
sysProp = " spark.executor.instances" ),
474
+ OptionAssigner (args.pyFiles, YARN , ALL_DEPLOY_MODES , sysProp = " spark.yarn.dist.pyFiles" ),
466
475
OptionAssigner (args.jars, YARN , ALL_DEPLOY_MODES , sysProp = " spark.yarn.dist.jars" ),
467
476
OptionAssigner (args.files, YARN , ALL_DEPLOY_MODES , sysProp = " spark.yarn.dist.files" ),
468
477
OptionAssigner (args.archives, YARN , ALL_DEPLOY_MODES , sysProp = " spark.yarn.dist.archives" ),
@@ -486,15 +495,28 @@ object SparkSubmit extends CommandLineUtils {
486
495
sysProp = " spark.driver.cores" ),
487
496
OptionAssigner (args.supervise.toString, STANDALONE | MESOS , CLUSTER ,
488
497
sysProp = " spark.driver.supervise" ),
489
- OptionAssigner (args.ivyRepoPath, STANDALONE , CLUSTER , sysProp = " spark.jars.ivy" )
498
+ OptionAssigner (args.ivyRepoPath, STANDALONE , CLUSTER , sysProp = " spark.jars.ivy" ),
499
+
500
+ // An internal option used only for spark-shell to add user jars to repl's classloader,
501
+ // previously it uses "spark.jars" or "spark.yarn.dist.jars" which now may be pointed to
502
+ // remote jars, so adding a new option to only specify local jars for spark-shell internally.
503
+ OptionAssigner (localJars, ALL_CLUSTER_MGRS , CLIENT , sysProp = " spark.repl.local.jars" )
490
504
)
491
505
492
506
// In client mode, launch the application main class directly
493
507
// In addition, add the main application jar and any added jars (if any) to the classpath
494
- // Also add the main application jar and any added jars to classpath in case YARN client
495
- // requires these jars.
496
- if (deployMode == CLIENT || isYarnCluster) {
508
+ if (deployMode == CLIENT ) {
497
509
childMainClass = args.mainClass
510
+ if (localPrimaryResource != null && isUserJar(localPrimaryResource)) {
511
+ childClasspath += localPrimaryResource
512
+ }
513
+ if (localJars != null ) { childClasspath ++= localJars.split(" ," ) }
514
+ }
515
+ // Add the main application jar and any added jars to classpath in case YARN client
516
+ // requires these jars.
517
+ // This assumes both primaryResource and user jars are local jars, otherwise it will not be
518
+ // added to the classpath of YARN client.
519
+ if (isYarnCluster) {
498
520
if (isUserJar(args.primaryResource)) {
499
521
childClasspath += args.primaryResource
500
522
}
@@ -551,10 +573,6 @@ object SparkSubmit extends CommandLineUtils {
551
573
if (args.isPython) {
552
574
sysProps.put(" spark.yarn.isPython" , " true" )
553
575
}
554
-
555
- if (args.pyFiles != null ) {
556
- sysProps(" spark.submit.pyFiles" ) = args.pyFiles
557
- }
558
576
}
559
577
560
578
// assure a keytab is available from any place in a JVM
0 commit comments