You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by an...@apache.org on 2015/06/18 07:33:46 UTC

spark git commit: [SPARK-8095] Resolve dependencies of --packages in local ivy cache

Repository: spark
Updated Branches:
  refs/heads/master e2cdb0568 -> 3b6107704


[SPARK-8095] Resolve dependencies of --packages in local ivy cache

Dependencies of artifacts in the local ivy cache were not being resolved properly. The dependencies were not being picked up. Now they should be.

cc andrewor14

Author: Burak Yavuz <br...@gmail.com>

Closes #6788 from brkyvz/local-ivy-fix and squashes the following commits:

2875bf4 [Burak Yavuz] fix temp dir bug
48cc648 [Burak Yavuz] improve deletion
a69e3e6 [Burak Yavuz] delete cache before test as well
0037197 [Burak Yavuz] fix merge conflicts
f60772c [Burak Yavuz] use different folder for m2 cache during testing
b6ef038 [Burak Yavuz] [SPARK-8095] Resolve dependencies of Spark Packages in local ivy cache


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3b610770
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3b610770
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3b610770

Branch: refs/heads/master
Commit: 3b6107704fb946e9fcb8c1c9bc4ded1b88c571af
Parents: e2cdb05
Author: Burak Yavuz <br...@gmail.com>
Authored: Wed Jun 17 22:33:37 2015 -0700
Committer: Andrew Or <an...@databricks.com>
Committed: Wed Jun 17 22:33:37 2015 -0700

----------------------------------------------------------------------
 .../org/apache/spark/deploy/SparkSubmit.scala   |  22 ++--
 .../org/apache/spark/deploy/IvyTestUtils.scala  | 124 ++++++++++++++++---
 .../spark/deploy/SparkSubmitUtilsSuite.scala    |  22 ++--
 3 files changed, 135 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/3b610770/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
----------------------------------------------------------------------
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 cfcc6d3..abf2227 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -35,7 +35,8 @@ import org.apache.ivy.core.resolve.ResolveOptions
 import org.apache.ivy.core.retrieve.RetrieveOptions
 import org.apache.ivy.core.settings.IvySettings
 import org.apache.ivy.plugins.matcher.GlobPatternMatcher
-import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver}
+import org.apache.ivy.plugins.repository.file.FileRepository
+import org.apache.ivy.plugins.resolver.{FileSystemResolver, ChainResolver, IBiblioResolver}
 import org.apache.spark.SPARK_VERSION
 import org.apache.spark.deploy.rest._
 import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils}
@@ -735,8 +736,14 @@ private[spark] object SparkSubmitUtils {
   }
 
   /** Path of the local Maven cache. */
-  private[spark] def m2Path: File = new File(System.getProperty("user.home"),
-    ".m2" + File.separator + "repository" + File.separator)
+  private[spark] def m2Path: File = {
+    if (Utils.isTesting) {
+      // test builds delete the maven cache, and this can cause flakiness
+      new File("dummy", ".m2" + File.separator + "repository")
+    } else {
+      new File(System.getProperty("user.home"), ".m2" + File.separator + "repository")
+    }
+  }
 
   /**
    * Extracts maven coordinates from a comma-delimited string
@@ -756,12 +763,13 @@ private[spark] object SparkSubmitUtils {
     localM2.setName("local-m2-cache")
     cr.add(localM2)
 
-    val localIvy = new IBiblioResolver
-    localIvy.setRoot(new File(ivySettings.getDefaultIvyUserDir,
-      "local" + File.separator).toURI.toString)
+    val localIvy = new FileSystemResolver
+    val localIvyRoot = new File(ivySettings.getDefaultIvyUserDir, "local")
+    localIvy.setLocal(true)
+    localIvy.setRepository(new FileRepository(localIvyRoot))
     val ivyPattern = Seq("[organisation]", "[module]", "[revision]", "[type]s",
       "[artifact](-[classifier]).[ext]").mkString(File.separator)
-    localIvy.setPattern(ivyPattern)
+    localIvy.addIvyPattern(localIvyRoot.getAbsolutePath + File.separator + ivyPattern)
     localIvy.setName("local-ivy-cache")
     cr.add(localIvy)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/3b610770/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
index 7d39984..823050b 100644
--- a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
@@ -24,6 +24,8 @@ import com.google.common.io.{Files, ByteStreams}
 
 import org.apache.commons.io.FileUtils
 
+import org.apache.ivy.core.settings.IvySettings
+
 import org.apache.spark.TestUtils.{createCompiledClass, JavaSourceFromString}
 import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate
 
@@ -44,13 +46,30 @@ private[deploy] object IvyTestUtils {
       if (!useIvyLayout) {
         Seq(groupDirs, artifactDirs, artifact.version).mkString(File.separator)
       } else {
-        Seq(groupDirs, artifactDirs, artifact.version, ext + "s").mkString(File.separator)
+        Seq(artifact.groupId, artifactDirs, artifact.version, ext + "s").mkString(File.separator)
       }
     new File(prefix, artifactPath)
   }
 
-  private def artifactName(artifact: MavenCoordinate, ext: String = ".jar"): String = {
-    s"${artifact.artifactId}-${artifact.version}$ext"
+  /** Returns the artifact naming based on standard ivy or maven format. */
+  private def artifactName(
+      artifact: MavenCoordinate,
+      useIvyLayout: Boolean,
+      ext: String = ".jar"): String = {
+    if (!useIvyLayout) {
+      s"${artifact.artifactId}-${artifact.version}$ext"
+    } else {
+      s"${artifact.artifactId}$ext"
+    }
+  }
+
+  /** Returns the directory for the given groupId based on standard ivy or maven format. */
+  private def getBaseGroupDirectory(artifact: MavenCoordinate, useIvyLayout: Boolean): String = {
+    if (!useIvyLayout) {
+      artifact.groupId.replace(".", File.separator)
+    } else {
+      artifact.groupId
+    }
   }
 
   /** Write the contents to a file to the supplied directory. */
@@ -92,6 +111,22 @@ private[deploy] object IvyTestUtils {
     createCompiledClass(className, dir, sourceFile, Seq.empty)
   }
 
+  private def createDescriptor(
+      tempPath: File,
+      artifact: MavenCoordinate,
+      dependencies: Option[Seq[MavenCoordinate]],
+      useIvyLayout: Boolean): File = {
+    if (useIvyLayout) {
+      val ivyXmlPath = pathFromCoordinate(artifact, tempPath, "ivy", true)
+      Files.createParentDirs(new File(ivyXmlPath, "dummy"))
+      createIvyDescriptor(ivyXmlPath, artifact, dependencies)
+    } else {
+      val pomPath = pathFromCoordinate(artifact, tempPath, "pom", useIvyLayout)
+      Files.createParentDirs(new File(pomPath, "dummy"))
+      createPom(pomPath, artifact, dependencies)
+    }
+  }
+
   /** Helper method to write artifact information in the pom. */
   private def pomArtifactWriter(artifact: MavenCoordinate, tabCount: Int = 1): String = {
     var result = "\n" + "  " * tabCount + s"<groupId>${artifact.groupId}</groupId>"
@@ -121,15 +156,55 @@ private[deploy] object IvyTestUtils {
       "\n  <dependencies>\n" + inside + "\n  </dependencies>"
     }.getOrElse("")
     content += "\n</project>"
-    writeFile(dir, artifactName(artifact, ".pom"), content.trim)
+    writeFile(dir, artifactName(artifact, false, ".pom"), content.trim)
+  }
+
+  /** Helper method to write artifact information in the ivy.xml. */
+  private def ivyArtifactWriter(artifact: MavenCoordinate): String = {
+    s"""<dependency org="${artifact.groupId}" name="${artifact.artifactId}"
+       |            rev="${artifact.version}" force="true"
+       |            conf="compile->compile(*),master(*);runtime->runtime(*)"/>""".stripMargin
+  }
+
+  /** Create a pom file for this artifact. */
+  private def createIvyDescriptor(
+      dir: File,
+      artifact: MavenCoordinate,
+      dependencies: Option[Seq[MavenCoordinate]]): File = {
+    var content = s"""
+        |<?xml version="1.0" encoding="UTF-8"?>
+        |<ivy-module version="2.0" xmlns:m="http://ant.apache.org/ivy/maven">
+        |  <info organisation="${artifact.groupId}"
+        |        module="${artifact.artifactId}"
+        |        revision="${artifact.version}"
+        |        status="release" publication="20150405222456" />
+        |  <configurations>
+        |    <conf name="default" visibility="public" description="" extends="runtime,master"/>
+        |    <conf name="compile" visibility="public" description=""/>
+        |    <conf name="master" visibility="public" description=""/>
+        |    <conf name="runtime" visibility="public" description="" extends="compile"/>
+        |    <conf name="pom" visibility="public" description=""/>
+        |  </configurations>
+        |  <publications>
+        |     <artifact name="${artifactName(artifact, true, "")}" type="jar" ext="jar"
+        |               conf="master"/>
+        |  </publications>
+      """.stripMargin.trim
+    content += dependencies.map { deps =>
+      val inside = deps.map(ivyArtifactWriter).mkString("\n")
+      "\n  <dependencies>\n" + inside + "\n  </dependencies>"
+    }.getOrElse("")
+    content += "\n</ivy-module>"
+    writeFile(dir, "ivy.xml", content.trim)
   }
 
   /** Create the jar for the given maven coordinate, using the supplied files. */
   private def packJar(
       dir: File,
       artifact: MavenCoordinate,
-      files: Seq[(String, File)]): File = {
-    val jarFile = new File(dir, artifactName(artifact))
+      files: Seq[(String, File)],
+      useIvyLayout: Boolean): File = {
+    val jarFile = new File(dir, artifactName(artifact, useIvyLayout))
     val jarFileStream = new FileOutputStream(jarFile)
     val jarStream = new JarOutputStream(jarFileStream, new java.util.jar.Manifest())
 
@@ -187,12 +262,10 @@ private[deploy] object IvyTestUtils {
         } else {
           Seq(javaFile)
         }
-      val jarFile = packJar(jarPath, artifact, allFiles)
+      val jarFile = packJar(jarPath, artifact, allFiles, useIvyLayout)
       assert(jarFile.exists(), "Problem creating Jar file")
-      val pomPath = pathFromCoordinate(artifact, tempPath, "pom", useIvyLayout)
-      Files.createParentDirs(new File(pomPath, "dummy"))
-      val pomFile = createPom(pomPath, artifact, dependencies)
-      assert(pomFile.exists(), "Problem creating Pom file")
+      val descriptor = createDescriptor(tempPath, artifact, dependencies, useIvyLayout)
+      assert(descriptor.exists(), "Problem creating Pom file")
     } finally {
       FileUtils.deleteDirectory(root)
     }
@@ -237,7 +310,10 @@ private[deploy] object IvyTestUtils {
       dependencies: Option[String],
       rootDir: Option[File],
       useIvyLayout: Boolean = false,
-      withPython: Boolean = false)(f: String => Unit): Unit = {
+      withPython: Boolean = false,
+      ivySettings: IvySettings = new IvySettings)(f: String => Unit): Unit = {
+    val deps = dependencies.map(SparkSubmitUtils.extractMavenCoordinates)
+    purgeLocalIvyCache(artifact, deps, ivySettings)
     val repo = createLocalRepositoryForTests(artifact, dependencies, rootDir, useIvyLayout,
       withPython)
     try {
@@ -245,17 +321,29 @@ private[deploy] object IvyTestUtils {
     } finally {
       // Clean up
       if (repo.toString.contains(".m2") || repo.toString.contains(".ivy2")) {
-        FileUtils.deleteDirectory(new File(repo,
-          artifact.groupId.replace(".", File.separator) + File.separator + artifact.artifactId))
-        dependencies.map(SparkSubmitUtils.extractMavenCoordinates).foreach { seq =>
-          seq.foreach { dep =>
-            FileUtils.deleteDirectory(new File(repo,
-              dep.artifactId.replace(".", File.separator)))
+        val groupDir = getBaseGroupDirectory(artifact, useIvyLayout)
+        FileUtils.deleteDirectory(new File(repo, groupDir + File.separator + artifact.artifactId))
+        deps.foreach { _.foreach { dep =>
+            FileUtils.deleteDirectory(new File(repo, getBaseGroupDirectory(dep, useIvyLayout)))
           }
         }
       } else {
         FileUtils.deleteDirectory(repo)
       }
+      purgeLocalIvyCache(artifact, deps, ivySettings)
+    }
+  }
+
+  /** Deletes the test packages from the ivy cache */
+  private def purgeLocalIvyCache(
+      artifact: MavenCoordinate,
+      dependencies: Option[Seq[MavenCoordinate]],
+      ivySettings: IvySettings): Unit = {
+    // delete the artifact from the cache as well if it already exists
+    FileUtils.deleteDirectory(new File(ivySettings.getDefaultCache, artifact.groupId))
+    dependencies.foreach { _.foreach { dep =>
+        FileUtils.deleteDirectory(new File(ivySettings.getDefaultCache, dep.groupId))
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/3b610770/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
index 3a8da9f..12c40f0 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
@@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.ivy.core.module.descriptor.MDArtifact
 import org.apache.ivy.core.settings.IvySettings
-import org.apache.ivy.plugins.resolver.IBiblioResolver
+import org.apache.ivy.plugins.resolver.{AbstractResolver, FileSystemResolver, IBiblioResolver}
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate
@@ -68,7 +68,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll {
     // should have central and spark-packages by default
     assert(res1.getResolvers.size() === 4)
     assert(res1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "local-m2-cache")
-    assert(res1.getResolvers.get(1).asInstanceOf[IBiblioResolver].getName === "local-ivy-cache")
+    assert(res1.getResolvers.get(1).asInstanceOf[FileSystemResolver].getName === "local-ivy-cache")
     assert(res1.getResolvers.get(2).asInstanceOf[IBiblioResolver].getName === "central")
     assert(res1.getResolvers.get(3).asInstanceOf[IBiblioResolver].getName === "spark-packages")
 
@@ -76,10 +76,10 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll {
     val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos), settings)
     assert(resolver2.getResolvers.size() === 7)
     val expected = repos.split(",").map(r => s"$r/")
-    resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: IBiblioResolver, i) =>
+    resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: AbstractResolver, i) =>
       if (i > 3) {
         assert(resolver.getName === s"repo-${i - 3}")
-        assert(resolver.getRoot === expected(i - 4))
+        assert(resolver.asInstanceOf[IBiblioResolver].getRoot === expected(i - 4))
       }
     }
   }
@@ -112,28 +112,34 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll {
   }
 
   test("search for artifact at local repositories") {
-    val main = new MavenCoordinate("my.awesome.lib", "mylib", "0.1")
+    val main = new MavenCoordinate("my.great.lib", "mylib", "0.1")
+    val dep = "my.great.dep:mydep:0.5"
     // Local M2 repository
-    IvyTestUtils.withRepository(main, None, Some(SparkSubmitUtils.m2Path)) { repo =>
+    IvyTestUtils.withRepository(main, Some(dep), Some(SparkSubmitUtils.m2Path)) { repo =>
       val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None,
         isTest = true)
       assert(jarPath.indexOf("mylib") >= 0, "should find artifact")
+      assert(jarPath.indexOf("mydep") >= 0, "should find dependency")
     }
     // Local Ivy Repository
     val settings = new IvySettings
     val ivyLocal = new File(settings.getDefaultIvyUserDir, "local" + File.separator)
-    IvyTestUtils.withRepository(main, None, Some(ivyLocal), true) { repo =>
+    IvyTestUtils.withRepository(main, Some(dep), Some(ivyLocal), useIvyLayout = true) { repo =>
       val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None,
         isTest = true)
       assert(jarPath.indexOf("mylib") >= 0, "should find artifact")
+      assert(jarPath.indexOf("mydep") >= 0, "should find dependency")
     }
     // Local ivy repository with modified home
     val dummyIvyLocal = new File(tempIvyPath, "local" + File.separator)
-    IvyTestUtils.withRepository(main, None, Some(dummyIvyLocal), true) { repo =>
+    settings.setDefaultIvyUserDir(new File(tempIvyPath))
+    IvyTestUtils.withRepository(main, Some(dep), Some(dummyIvyLocal), useIvyLayout = true,
+      ivySettings = settings) { repo =>
       val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None,
         Some(tempIvyPath), isTest = true)
       assert(jarPath.indexOf("mylib") >= 0, "should find artifact")
       assert(jarPath.indexOf(tempIvyPath) >= 0, "should be in new ivy path")
+      assert(jarPath.indexOf("mydep") >= 0, "should find dependency")
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org