Skip to content

Commit

Permalink
[SPARK-13294][PROJECT INFRA] Remove MiMa's dependency on spark-class …
Browse files Browse the repository at this point in the history
…/ Spark assembly

This patch removes the need to build a full Spark assembly before running the `dev/mima` script.

- I modified the `tools` project to remove a direct dependency on Spark, so `sbt/sbt tools/fullClasspath` will now return the classpath for the `GenerateMIMAIgnore` class itself plus its own dependencies.
   - This required me to delete two classes full of dead code that we don't use anymore
- `GenerateMIMAIgnore` now uses [ClassUtil](http://software.clapper.org/classutil/) to find all of the Spark classes rather than our homemade JAR traversal code. The problem in our own code was that it didn't handle folders of classes properly, which is necessary in order to generate excludes with an assembly-free Spark build.
- `./dev/mima` no longer runs through `spark-class`, eliminating the need to reason about classpath ordering between `SPARK_CLASSPATH` and the assembly.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11178 from JoshRosen/remove-assembly-in-run-tests.
  • Loading branch information
JoshRosen committed Mar 11, 2016
1 parent d18276c commit 6ca990f
Show file tree
Hide file tree
Showing 8 changed files with 58 additions and 576 deletions.
23 changes: 10 additions & 13 deletions dev/mima
Original file line number Diff line number Diff line change
Expand Up @@ -24,24 +24,21 @@ set -e
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
cd "$FWDIR"

echo -e "q\n" | build/sbt oldDeps/update
TOOLS_CLASSPATH="$(build/sbt "export tools/fullClasspath" | tail -n1)"

rm -f .generated-mima*

generate_mima_ignore() {
SPARK_JAVA_OPTS="-XX:MaxPermSize=1g -Xmx2g" \
./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore
java \
-XX:MaxPermSize=1g \
-Xmx2g \
-cp "$TOOLS_CLASSPATH:$1" \
org.apache.spark.tools.GenerateMIMAIgnore
}

# Generate Mima Ignore is called twice, first with latest built jars
# on the classpath and then again with previous version jars on the classpath.
# Because of a bug in GenerateMIMAIgnore that when old jars are ahead on classpath
# it did not process the new classes (which are in assembly jar).
generate_mima_ignore

export SPARK_CLASSPATH="$(build/sbt "export oldDeps/fullClasspath" | tail -n1)"
echo "SPARK_CLASSPATH=$SPARK_CLASSPATH"

generate_mima_ignore
SPARK_PROFILES="-Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
generate_mima_ignore "$(build/sbt $SPARK_PROFILES "export assembly/fullClasspath" | tail -n1)"
generate_mima_ignore "$(build/sbt $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)"

echo -e "q\n" | build/sbt mima-report-binary-issues | grep -v -e "info.*Resolving"
ret_val=$?
Expand Down
24 changes: 18 additions & 6 deletions dev/run-tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,6 @@ def build_spark_sbt(hadoop_version):
# Enable all of the profiles for the build:
build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags
sbt_goals = ["package",
"assembly/assembly",
"streaming-kafka-assembly/assembly",
"streaming-flume-assembly/assembly",
"streaming-mqtt-assembly/assembly",
Expand All @@ -350,6 +349,16 @@ def build_spark_sbt(hadoop_version):
exec_sbt(profiles_and_goals)


def build_spark_assembly_sbt(hadoop_version):
# Enable all of the profiles for the build:
build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags
sbt_goals = ["assembly/assembly"]
profiles_and_goals = build_profiles + sbt_goals
print("[info] Building Spark assembly (w/Hive 1.2.1) using SBT with these arguments: ",
" ".join(profiles_and_goals))
exec_sbt(profiles_and_goals)


def build_apache_spark(build_tool, hadoop_version):
"""Will build Spark against Hive v1.2.1 given the passed in build tool (either `sbt` or
`maven`). Defaults to using `sbt`."""
Expand Down Expand Up @@ -561,11 +570,14 @@ def main():
# spark build
build_apache_spark(build_tool, hadoop_version)

# TODO Temporarily disable MiMA check for DF-to-DS migration prototyping
# # backwards compatibility checks
# if build_tool == "sbt":
# # Note: compatiblity tests only supported in sbt for now
# detect_binary_inop_with_mima()
# backwards compatibility checks
if build_tool == "sbt":
# Note: compatibility tests only supported in sbt for now
# TODO Temporarily disable MiMA check for DF-to-DS migration prototyping
# detect_binary_inop_with_mima()
# Since we did not build assembly/assembly before running dev/mima, we need to
# do it here because the tests still rely on it; see SPARK-13294 for details.
build_spark_assembly_sbt(hadoop_version)

# run the test suites
run_scala_tests(build_tool, hadoop_version, test_modules, excluded_tags)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,10 @@

package org.apache.spark.launcher;

import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;

import static org.apache.spark.launcher.CommandBuilderUtils.*;

Expand Down Expand Up @@ -76,26 +74,6 @@ public List<String> buildCommand(Map<String, String> env) throws IOException {
javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
javaOptsKeys.add("SPARK_SHUFFLE_OPTS");
memKey = "SPARK_DAEMON_MEMORY";
} else if (className.startsWith("org.apache.spark.tools.")) {
String sparkHome = getSparkHome();
File toolsDir = new File(join(File.separator, sparkHome, "tools", "target",
"scala-" + getScalaVersion()));
checkState(toolsDir.isDirectory(), "Cannot find tools build directory.");

Pattern re = Pattern.compile("spark-tools_.*\\.jar");
for (File f : toolsDir.listFiles()) {
if (re.matcher(f.getName()).matches()) {
extraClassPath = f.getAbsolutePath();
break;
}
}

checkState(extraClassPath != null,
"Failed to find Spark Tools Jar in %s.\n" +
"You need to run \"build/sbt tools/package\" before running %s.",
toolsDir.getAbsolutePath(), className);

javaOptsKeys.add("SPARK_JAVA_OPTS");
} else {
javaOptsKeys.add("SPARK_JAVA_OPTS");
memKey = "SPARK_DRIVER_MEMORY";
Expand Down
19 changes: 10 additions & 9 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -384,18 +384,19 @@ object OldDeps {

lazy val project = Project("oldDeps", file("dev"), settings = oldDepsSettings)

def versionArtifact(id: String): Option[sbt.ModuleID] = {
val fullId = id + "_2.11"
Some("org.apache.spark" % fullId % "1.2.0")
}

def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq(
name := "old-deps",
scalaVersion := "2.10.5",
libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq",
"spark-streaming-flume", "spark-streaming-twitter",
"spark-streaming", "spark-mllib", "spark-graphx",
"spark-core").map(versionArtifact(_).get intransitive())
libraryDependencies := Seq(
"spark-streaming-mqtt",
"spark-streaming-zeromq",
"spark-streaming-flume",
"spark-streaming-twitter",
"spark-streaming",
"spark-mllib",
"spark-graphx",
"spark-core"
).map(id => "org.apache.spark" % (id + "_2.11") % "1.2.0")
)
}

Expand Down
15 changes: 5 additions & 10 deletions tools/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -34,16 +34,6 @@
<url>http://spark.apache.org/</url>

<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-reflect</artifactId>
Expand All @@ -52,6 +42,11 @@
<groupId>org.scala-lang</groupId>
<artifactId>scala-compiler</artifactId>
</dependency>
<dependency>
<groupId>org.clapper</groupId>
<artifactId>classutil_${scala.binary.version}</artifactId>
<version>1.0.6</version>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,13 @@
// scalastyle:off classforname
package org.apache.spark.tools

import java.io.File
import java.util.jar.JarFile

import scala.collection.mutable
import scala.collection.JavaConverters._
import scala.reflect.runtime.{universe => unv}
import scala.reflect.runtime.universe.runtimeMirror
import scala.util.Try

import org.clapper.classutil.ClassFinder

/**
* A tool for generating classes to be excluded during binary checking with MIMA. It is expected
* that this tool is run with ./spark-class.
Expand All @@ -42,12 +40,13 @@ object GenerateMIMAIgnore {
private val classLoader = Thread.currentThread().getContextClassLoader
private val mirror = runtimeMirror(classLoader)

private def isDeveloperApi(sym: unv.Symbol) = sym.annotations.exists {
_.tpe =:= mirror.staticClass("org.apache.spark.annotation.DeveloperApi").toType
}

private def isDeveloperApi(sym: unv.Symbol) =
sym.annotations.exists(_.tpe =:= unv.typeOf[org.apache.spark.annotation.DeveloperApi])

private def isExperimental(sym: unv.Symbol) =
sym.annotations.exists(_.tpe =:= unv.typeOf[org.apache.spark.annotation.Experimental])
private def isExperimental(sym: unv.Symbol) = sym.annotations.exists {
_.tpe =:= mirror.staticClass("org.apache.spark.annotation.Experimental").toType
}


private def isPackagePrivate(sym: unv.Symbol) =
Expand Down Expand Up @@ -160,35 +159,13 @@ object GenerateMIMAIgnore {
* and subpackages both from directories and jars present on the classpath.
*/
private def getClasses(packageName: String): Set[String] = {
val path = packageName.replace('.', '/')
val resources = classLoader.getResources(path)

val jars = resources.asScala.filter(_.getProtocol == "jar")
.map(_.getFile.split(":")(1).split("!")(0)).toSeq

jars.flatMap(getClassesFromJar(_, path))
.map(_.getName)
.filterNot(shouldExclude).toSet
}

/**
* Get all classes in a package from a jar file.
*/
private def getClassesFromJar(jarPath: String, packageName: String) = {
import scala.collection.mutable
val jar = new JarFile(new File(jarPath))
val enums = jar.entries().asScala.map(_.getName).filter(_.startsWith(packageName))
val classes = mutable.HashSet[Class[_]]()
for (entry <- enums if entry.endsWith(".class")) {
try {
classes += Class.forName(entry.replace('/', '.').stripSuffix(".class"), false, classLoader)
} catch {
// scalastyle:off println
case _: Throwable => println("Unable to load:" + entry)
// scalastyle:on println
}
}
classes
val finder = ClassFinder()
finder
.getClasses
.map(_.name)
.filter(_.startsWith(packageName))
.filterNot(shouldExclude)
.toSet
}
}
// scalastyle:on classforname
Loading

0 comments on commit 6ca990f

Please sign in to comment.