aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org
diff options
context:
space:
mode:
authorPatrick Wendell <patrick@databricks.com>2015-05-01 13:01:43 -0700
committerPatrick Wendell <patrick@databricks.com>2015-05-01 13:01:43 -0700
commitc6d9a429421561508e8adbb4892954381bc33a90 (patch)
tree33ad5b623d3b209eebd6b8f11990e2f2f7168bf5 /core/src/main/scala/org
parent58d6584d349d5208a994a074b4cfa8a6ec4d1665 (diff)
downloadspark-c6d9a429421561508e8adbb4892954381bc33a90.tar.gz
spark-c6d9a429421561508e8adbb4892954381bc33a90.tar.bz2
spark-c6d9a429421561508e8adbb4892954381bc33a90.zip
Revert "[SPARK-7224] added mock repository generator for --packages tests"
This reverts commit 7dacc08ab36188991a001df23880167433844767.
Diffstat (limited to 'core/src/main/scala/org')
-rw-r--r--core/src/main/scala/org/apache/spark/TestUtils.scala27
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala129
2 files changed, 69 insertions, 87 deletions
diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
index fe6320b504..398ca41e16 100644
--- a/core/src/main/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -105,18 +105,23 @@ private[spark] object TestUtils {
URI.create(s"string:///${name.replace(".", "/")}${SOURCE.extension}")
}
- private[spark] class JavaSourceFromString(val name: String, val code: String)
+ private class JavaSourceFromString(val name: String, val code: String)
extends SimpleJavaFileObject(createURI(name), SOURCE) {
override def getCharContent(ignoreEncodingErrors: Boolean): String = code
}
- /** Creates a compiled class with the source file. Class file will be placed in destDir. */
+ /** Creates a compiled class with the given name. Class file will be placed in destDir. */
def createCompiledClass(
className: String,
destDir: File,
- sourceFile: JavaSourceFromString,
- classpathUrls: Seq[URL]): File = {
+ toStringValue: String = "",
+ baseClass: String = null,
+ classpathUrls: Seq[URL] = Seq()): File = {
val compiler = ToolProvider.getSystemJavaCompiler
+ val extendsText = Option(baseClass).map { c => s" extends ${c}" }.getOrElse("")
+ val sourceFile = new JavaSourceFromString(className,
+ "public class " + className + extendsText + " implements java.io.Serializable {" +
+ " @Override public String toString() { return \"" + toStringValue + "\"; }}")
// Calling this outputs a class file in pwd. It's easier to just rename the file than
// build a custom FileManager that controls the output location.
@@ -139,18 +144,4 @@ private[spark] object TestUtils {
assert(out.exists(), "Destination file not moved: " + out.getAbsolutePath())
out
}
-
- /** Creates a compiled class with the given name. Class file will be placed in destDir. */
- def createCompiledClass(
- className: String,
- destDir: File,
- toStringValue: String = "",
- baseClass: String = null,
- classpathUrls: Seq[URL] = Seq()): File = {
- val extendsText = Option(baseClass).map { c => s" extends ${c}" }.getOrElse("")
- val sourceFile = new JavaSourceFromString(className,
- "public class " + className + extendsText + " implements java.io.Serializable {" +
- " @Override public String toString() { return \"" + toStringValue + "\"; }}")
- createCompiledClass(className, destDir, sourceFile, classpathUrls)
- }
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 0d149e703a..b8ae4af18d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -20,7 +20,6 @@ package org.apache.spark.deploy
import java.io.{File, PrintStream}
import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException}
import java.net.URL
-import java.nio.file.{Path => JavaPath}
import java.security.PrivilegedExceptionAction
import scala.collection.mutable.{ArrayBuffer, HashMap, Map}
@@ -709,9 +708,7 @@ private[deploy] object SparkSubmitUtils {
* @param artifactId the artifactId of the coordinate
* @param version the version of the coordinate
*/
- private[deploy] case class MavenCoordinate(groupId: String, artifactId: String, version: String) {
- override def toString: String = s"$groupId:$artifactId:$version"
- }
+ private[deploy] case class MavenCoordinate(groupId: String, artifactId: String, version: String)
/**
* Extracts maven coordinates from a comma-delimited string. Coordinates should be provided
@@ -734,10 +731,6 @@ private[deploy] object SparkSubmitUtils {
}
}
- /** Path of the local Maven cache. */
- private[spark] def m2Path: JavaPath = new File(System.getProperty("user.home"),
- ".m2" + File.separator + "repository" + File.separator).toPath
-
/**
* Extracts maven coordinates from a comma-delimited string
* @param remoteRepos Comma-delimited string of remote repositories
@@ -751,7 +744,8 @@ private[deploy] object SparkSubmitUtils {
val localM2 = new IBiblioResolver
localM2.setM2compatible(true)
- localM2.setRoot(m2Path.toUri.toString)
+ val m2Path = ".m2" + File.separator + "repository" + File.separator
+ localM2.setRoot(new File(System.getProperty("user.home"), m2Path).toURI.toString)
localM2.setUsepoms(true)
localM2.setName("local-m2-cache")
cr.add(localM2)
@@ -876,72 +870,69 @@ private[deploy] object SparkSubmitUtils {
""
} else {
val sysOut = System.out
- try {
- // To prevent ivy from logging to system out
- System.setOut(printStream)
- val artifacts = extractMavenCoordinates(coordinates)
- // Default configuration name for ivy
- val ivyConfName = "default"
- // set ivy settings for location of cache
- val ivySettings: IvySettings = new IvySettings
- // Directories for caching downloads through ivy and storing the jars when maven coordinates
- // are supplied to spark-submit
- val alternateIvyCache = ivyPath.getOrElse("")
- val packagesDirectory: File =
- if (alternateIvyCache.trim.isEmpty) {
- new File(ivySettings.getDefaultIvyUserDir, "jars")
- } else {
- ivySettings.setDefaultIvyUserDir(new File(alternateIvyCache))
- ivySettings.setDefaultCache(new File(alternateIvyCache, "cache"))
- new File(alternateIvyCache, "jars")
- }
- printStream.println(
- s"Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}")
- printStream.println(s"The jars for the packages stored in: $packagesDirectory")
- // create a pattern matcher
- ivySettings.addMatcher(new GlobPatternMatcher)
- // create the dependency resolvers
- val repoResolver = createRepoResolvers(remoteRepos, ivySettings)
- ivySettings.addResolver(repoResolver)
- ivySettings.setDefaultResolver(repoResolver.getName)
-
- val ivy = Ivy.newInstance(ivySettings)
- // Set resolve options to download transitive dependencies as well
- val resolveOptions = new ResolveOptions
- resolveOptions.setTransitive(true)
- val retrieveOptions = new RetrieveOptions
- // Turn downloading and logging off for testing
- if (isTest) {
- resolveOptions.setDownload(false)
- resolveOptions.setLog(LogOptions.LOG_QUIET)
- retrieveOptions.setLog(LogOptions.LOG_QUIET)
+ // To prevent ivy from logging to system out
+ System.setOut(printStream)
+ val artifacts = extractMavenCoordinates(coordinates)
+ // Default configuration name for ivy
+ val ivyConfName = "default"
+ // set ivy settings for location of cache
+ val ivySettings: IvySettings = new IvySettings
+ // Directories for caching downloads through ivy and storing the jars when maven coordinates
+ // are supplied to spark-submit
+ val alternateIvyCache = ivyPath.getOrElse("")
+ val packagesDirectory: File =
+ if (alternateIvyCache.trim.isEmpty) {
+ new File(ivySettings.getDefaultIvyUserDir, "jars")
} else {
- resolveOptions.setDownload(true)
+ ivySettings.setDefaultIvyUserDir(new File(alternateIvyCache))
+ ivySettings.setDefaultCache(new File(alternateIvyCache, "cache"))
+ new File(alternateIvyCache, "jars")
}
+ printStream.println(
+ s"Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}")
+ printStream.println(s"The jars for the packages stored in: $packagesDirectory")
+ // create a pattern matcher
+ ivySettings.addMatcher(new GlobPatternMatcher)
+ // create the dependency resolvers
+ val repoResolver = createRepoResolvers(remoteRepos, ivySettings)
+ ivySettings.addResolver(repoResolver)
+ ivySettings.setDefaultResolver(repoResolver.getName)
+
+ val ivy = Ivy.newInstance(ivySettings)
+ // Set resolve options to download transitive dependencies as well
+ val resolveOptions = new ResolveOptions
+ resolveOptions.setTransitive(true)
+ val retrieveOptions = new RetrieveOptions
+ // Turn downloading and logging off for testing
+ if (isTest) {
+ resolveOptions.setDownload(false)
+ resolveOptions.setLog(LogOptions.LOG_QUIET)
+ retrieveOptions.setLog(LogOptions.LOG_QUIET)
+ } else {
+ resolveOptions.setDownload(true)
+ }
- // A Module descriptor must be specified. Entries are dummy strings
- val md = getModuleDescriptor
- md.setDefaultConf(ivyConfName)
+ // A Module descriptor must be specified. Entries are dummy strings
+ val md = getModuleDescriptor
+ md.setDefaultConf(ivyConfName)
- // Add exclusion rules for Spark and Scala Library
- addExclusionRules(ivySettings, ivyConfName, md)
- // add all supplied maven artifacts as dependencies
- addDependenciesToIvy(md, artifacts, ivyConfName)
+ // Add exclusion rules for Spark and Scala Library
+ addExclusionRules(ivySettings, ivyConfName, md)
+ // add all supplied maven artifacts as dependencies
+ addDependenciesToIvy(md, artifacts, ivyConfName)
- // resolve dependencies
- val rr: ResolveReport = ivy.resolve(md, resolveOptions)
- if (rr.hasError) {
- throw new RuntimeException(rr.getAllProblemMessages.toString)
- }
- // retrieve all resolved dependencies
- ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId,
- packagesDirectory.getAbsolutePath + File.separator +
- "[organization]_[artifact]-[revision].[ext]",
- retrieveOptions.setConfs(Array(ivyConfName)))
- resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory)
- } finally {
- System.setOut(sysOut)
+ // resolve dependencies
+ val rr: ResolveReport = ivy.resolve(md, resolveOptions)
+ if (rr.hasError) {
+ throw new RuntimeException(rr.getAllProblemMessages.toString)
}
+ // retrieve all resolved dependencies
+ ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId,
+ packagesDirectory.getAbsolutePath + File.separator +
+ "[organization]_[artifact]-[revision].[ext]",
+ retrieveOptions.setConfs(Array(ivyConfName)))
+ System.setOut(sysOut)
+ resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory)
}
}
}