@@ -20,7 +20,6 @@ package org.apache.spark.deploy
2020import java .io .{File , PrintStream }
2121import java .lang .reflect .{InvocationTargetException , Modifier , UndeclaredThrowableException }
2222import java .net .URL
23- import java .nio .file .{Path => JavaPath }
2423import java .security .PrivilegedExceptionAction
2524
2625import scala .collection .mutable .{ArrayBuffer , HashMap , Map }
@@ -401,6 +400,10 @@ object SparkSubmit {
401400 OptionAssigner (args.archives, YARN , CLUSTER , clOption = " --archives" ),
402401 OptionAssigner (args.jars, YARN , CLUSTER , clOption = " --addJars" ),
403402
403+ // Yarn client or cluster
404+ OptionAssigner (args.principal, YARN , ALL_DEPLOY_MODES , clOption = " --principal" ),
405+ OptionAssigner (args.keytab, YARN , ALL_DEPLOY_MODES , clOption = " --keytab" ),
406+
404407 // Other options
405408 OptionAssigner (args.executorCores, STANDALONE , ALL_DEPLOY_MODES ,
406409 sysProp = " spark.executor.cores" ),
@@ -709,9 +712,7 @@ private[deploy] object SparkSubmitUtils {
709712 * @param artifactId the artifactId of the coordinate
710713 * @param version the version of the coordinate
711714 */
712- private [deploy] case class MavenCoordinate (groupId : String , artifactId : String , version : String ) {
713- override def toString : String = s " $groupId: $artifactId: $version"
714- }
715+ private [deploy] case class MavenCoordinate (groupId : String , artifactId : String , version : String )
715716
716717/**
717718 * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided
@@ -734,10 +735,6 @@ private[deploy] object SparkSubmitUtils {
734735 }
735736 }
736737
737- /** Path of the local Maven cache. */
738- private [spark] def m2Path : JavaPath = new File (System .getProperty(" user.home" ),
739- " .m2" + File .separator + " repository" + File .separator).toPath
740-
741738 /**
742739 * Extracts maven coordinates from a comma-delimited string
743740 * @param remoteRepos Comma-delimited string of remote repositories
@@ -751,7 +748,8 @@ private[deploy] object SparkSubmitUtils {
751748
752749 val localM2 = new IBiblioResolver
753750 localM2.setM2compatible(true )
754- localM2.setRoot(m2Path.toUri.toString)
751+ val m2Path = " .m2" + File .separator + " repository" + File .separator
752+ localM2.setRoot(new File (System .getProperty(" user.home" ), m2Path).toURI.toString)
755753 localM2.setUsepoms(true )
756754 localM2.setName(" local-m2-cache" )
757755 cr.add(localM2)
@@ -876,72 +874,69 @@ private[deploy] object SparkSubmitUtils {
876874 " "
877875 } else {
878876 val sysOut = System .out
879- try {
880- // To prevent ivy from logging to system out
881- System .setOut(printStream)
882- val artifacts = extractMavenCoordinates(coordinates)
883- // Default configuration name for ivy
884- val ivyConfName = " default"
885- // set ivy settings for location of cache
886- val ivySettings : IvySettings = new IvySettings
887- // Directories for caching downloads through ivy and storing the jars when maven coordinates
888- // are supplied to spark-submit
889- val alternateIvyCache = ivyPath.getOrElse(" " )
890- val packagesDirectory : File =
891- if (alternateIvyCache.trim.isEmpty) {
892- new File (ivySettings.getDefaultIvyUserDir, " jars" )
893- } else {
894- ivySettings.setDefaultIvyUserDir(new File (alternateIvyCache))
895- ivySettings.setDefaultCache(new File (alternateIvyCache, " cache" ))
896- new File (alternateIvyCache, " jars" )
897- }
898- printStream.println(
899- s " Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}" )
900- printStream.println(s " The jars for the packages stored in: $packagesDirectory" )
901- // create a pattern matcher
902- ivySettings.addMatcher(new GlobPatternMatcher )
903- // create the dependency resolvers
904- val repoResolver = createRepoResolvers(remoteRepos, ivySettings)
905- ivySettings.addResolver(repoResolver)
906- ivySettings.setDefaultResolver(repoResolver.getName)
907-
908- val ivy = Ivy .newInstance(ivySettings)
909- // Set resolve options to download transitive dependencies as well
910- val resolveOptions = new ResolveOptions
911- resolveOptions.setTransitive(true )
912- val retrieveOptions = new RetrieveOptions
913- // Turn downloading and logging off for testing
914- if (isTest) {
915- resolveOptions.setDownload(false )
916- resolveOptions.setLog(LogOptions .LOG_QUIET )
917- retrieveOptions.setLog(LogOptions .LOG_QUIET )
877+ // To prevent ivy from logging to system out
878+ System .setOut(printStream)
879+ val artifacts = extractMavenCoordinates(coordinates)
880+ // Default configuration name for ivy
881+ val ivyConfName = " default"
882+ // set ivy settings for location of cache
883+ val ivySettings : IvySettings = new IvySettings
884+ // Directories for caching downloads through ivy and storing the jars when maven coordinates
885+ // are supplied to spark-submit
886+ val alternateIvyCache = ivyPath.getOrElse(" " )
887+ val packagesDirectory : File =
888+ if (alternateIvyCache.trim.isEmpty) {
889+ new File (ivySettings.getDefaultIvyUserDir, " jars" )
918890 } else {
919- resolveOptions.setDownload(true )
891+ ivySettings.setDefaultIvyUserDir(new File (alternateIvyCache))
892+ ivySettings.setDefaultCache(new File (alternateIvyCache, " cache" ))
893+ new File (alternateIvyCache, " jars" )
920894 }
895+ printStream.println(
896+ s " Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}" )
897+ printStream.println(s " The jars for the packages stored in: $packagesDirectory" )
898+ // create a pattern matcher
899+ ivySettings.addMatcher(new GlobPatternMatcher )
900+ // create the dependency resolvers
901+ val repoResolver = createRepoResolvers(remoteRepos, ivySettings)
902+ ivySettings.addResolver(repoResolver)
903+ ivySettings.setDefaultResolver(repoResolver.getName)
904+
905+ val ivy = Ivy .newInstance(ivySettings)
906+ // Set resolve options to download transitive dependencies as well
907+ val resolveOptions = new ResolveOptions
908+ resolveOptions.setTransitive(true )
909+ val retrieveOptions = new RetrieveOptions
910+ // Turn downloading and logging off for testing
911+ if (isTest) {
912+ resolveOptions.setDownload(false )
913+ resolveOptions.setLog(LogOptions .LOG_QUIET )
914+ retrieveOptions.setLog(LogOptions .LOG_QUIET )
915+ } else {
916+ resolveOptions.setDownload(true )
917+ }
921918
922- // A Module descriptor must be specified. Entries are dummy strings
923- val md = getModuleDescriptor
924- md.setDefaultConf(ivyConfName)
919+ // A Module descriptor must be specified. Entries are dummy strings
920+ val md = getModuleDescriptor
921+ md.setDefaultConf(ivyConfName)
925922
926- // Add exclusion rules for Spark and Scala Library
927- addExclusionRules(ivySettings, ivyConfName, md)
928- // add all supplied maven artifacts as dependencies
929- addDependenciesToIvy(md, artifacts, ivyConfName)
923+ // Add exclusion rules for Spark and Scala Library
924+ addExclusionRules(ivySettings, ivyConfName, md)
925+ // add all supplied maven artifacts as dependencies
926+ addDependenciesToIvy(md, artifacts, ivyConfName)
930927
931- // resolve dependencies
932- val rr : ResolveReport = ivy.resolve(md, resolveOptions)
933- if (rr.hasError) {
934- throw new RuntimeException (rr.getAllProblemMessages.toString)
935- }
936- // retrieve all resolved dependencies
937- ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId,
938- packagesDirectory.getAbsolutePath + File .separator +
939- " [organization]_[artifact]-[revision].[ext]" ,
940- retrieveOptions.setConfs(Array (ivyConfName)))
941- resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory)
942- } finally {
943- System .setOut(sysOut)
928+ // resolve dependencies
929+ val rr : ResolveReport = ivy.resolve(md, resolveOptions)
930+ if (rr.hasError) {
931+ throw new RuntimeException (rr.getAllProblemMessages.toString)
944932 }
933+ // retrieve all resolved dependencies
934+ ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId,
935+ packagesDirectory.getAbsolutePath + File .separator +
936+ " [organization]_[artifact]-[revision].[ext]" ,
937+ retrieveOptions.setConfs(Array (ivyConfName)))
938+ System .setOut(sysOut)
939+ resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory)
945940 }
946941 }
947942}
0 commit comments