aboutsummaryrefslogtreecommitdiff
path: root/launcher
diff options
context:
space:
mode:
authorMarcelo Vanzin <vanzin@cloudera.com>2016-03-15 09:44:48 -0700
committerMarcelo Vanzin <vanzin@cloudera.com>2016-03-15 09:44:51 -0700
commit48978abfa4d8f2cf79a4b053cc8bc7254cc2d61b (patch)
tree009d0b0870a8eb6a185cf192acb172d2d91b540c /launcher
parentbd5365bbe9ff6518cde9402ee8843ec1002fff5b (diff)
downloadspark-48978abfa4d8f2cf79a4b053cc8bc7254cc2d61b.tar.gz
spark-48978abfa4d8f2cf79a4b053cc8bc7254cc2d61b.tar.bz2
spark-48978abfa4d8f2cf79a4b053cc8bc7254cc2d61b.zip
[SPARK-13576][BUILD] Don't create assembly for examples.
As part of the goal to stop creating assemblies in Spark, this change modifies the mvn and sbt builds to not create an assembly for examples. Instead, dependencies are copied to the build directory (under target/scala-xx/jars), and in the final archive, into the "examples/jars" directory. To avoid having to deal too much with Windows batch files, I made examples run through the launcher library; the spark-submit launcher now has a special mode to run examples, which adds all the necessary jars to the spark-submit command line, and replaces the bash and batch scripts that were used to run examples. The scripts are now just a thin wrapper around spark-submit; another advantage is that now all spark-submit options are supported. There are a few glitches; in the mvn build, a lot of duplicated dependencies get copied, because they are promoted to "compile" scope due to extra dependencies in the examples module (such as HBase). In the sbt build, all dependencies are copied, because there doesn't seem to be an easy way to filter things. I plan to clean some of this up when the rest of the tasks are finished. When the main assembly is replaced with jars, we can remove duplicate jars from the examples directory during packaging. Tested by running SparkPi in: maven build, sbt build, dist created by make-distribution.sh. Finally: note that running the "assembly" target in sbt doesn't build the examples anymore. You need to run "package" for that. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #11452 from vanzin/SPARK-13576.
Diffstat (limited to 'launcher')
-rw-r--r--launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java68
-rw-r--r--launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java18
2 files changed, 80 insertions, 6 deletions
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
index b2dd6ac4c3..56e4107c5a 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
@@ -30,7 +30,8 @@ import static org.apache.spark.launcher.CommandBuilderUtils.*;
* driver-side options and special parsing behavior needed for the special-casing certain internal
* Spark applications.
* <p>
- * This class has also some special features to aid launching pyspark.
+ * This class has also some special features to aid launching shells (pyspark and sparkR) and also
+ * examples.
*/
class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
@@ -63,6 +64,17 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
static final String SPARKR_SHELL_RESOURCE = "sparkr-shell";
/**
+ * Name of app resource used to identify examples. When running examples, args[0] should be
+ * this name. The app resource will identify the example class to run.
+ */
+ static final String RUN_EXAMPLE = "run-example";
+
+ /**
+ * Prefix for example class names.
+ */
+ static final String EXAMPLE_CLASS_PREFIX = "org.apache.spark.examples.";
+
+ /**
* This map must match the class names for available special classes, since this modifies the way
* command line parsing works. This maps the class name to the resource to use when calling
* spark-submit.
@@ -78,6 +90,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
final List<String> sparkArgs;
private final boolean printInfo;
+ private final boolean isExample;
/**
* Controls whether mixing spark-submit arguments with app arguments is allowed. This is needed
@@ -89,10 +102,13 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
SparkSubmitCommandBuilder() {
this.sparkArgs = new ArrayList<>();
this.printInfo = false;
+ this.isExample = false;
}
SparkSubmitCommandBuilder(List<String> args) {
- this.sparkArgs = new ArrayList<>();
+ this.allowsMixedArguments = false;
+
+ boolean isExample = false;
List<String> submitArgs = args;
if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) {
this.allowsMixedArguments = true;
@@ -102,10 +118,14 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
this.allowsMixedArguments = true;
appResource = SPARKR_SHELL_RESOURCE;
submitArgs = args.subList(1, args.size());
- } else {
- this.allowsMixedArguments = false;
+ } else if (args.size() > 0 && args.get(0).equals(RUN_EXAMPLE)) {
+ isExample = true;
+ submitArgs = args.subList(1, args.size());
}
+ this.sparkArgs = new ArrayList<>();
+ this.isExample = isExample;
+
OptionParser parser = new OptionParser();
parser.parse(submitArgs);
this.printInfo = parser.infoRequested;
@@ -155,6 +175,10 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
args.add(propertiesFile);
}
+ if (isExample) {
+ jars.addAll(findExamplesJars());
+ }
+
if (!jars.isEmpty()) {
args.add(parser.JARS);
args.add(join(",", jars));
@@ -170,6 +194,9 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
args.add(join(",", pyFiles));
}
+ if (!printInfo) {
+ checkArgument(!isExample || mainClass != null, "Missing example class name.");
+ }
if (mainClass != null) {
args.add(parser.CLASS);
args.add(mainClass);
@@ -308,6 +335,25 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
mainClass.equals("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2"));
}
+ private List<String> findExamplesJars() {
+ List<String> examplesJars = new ArrayList<>();
+ String sparkHome = getSparkHome();
+
+ File jarsDir;
+ if (new File(sparkHome, "RELEASE").isFile()) {
+ jarsDir = new File(sparkHome, "examples/jars");
+ } else {
+ jarsDir = new File(sparkHome,
+ String.format("examples/target/scala-%s/jars", getScalaVersion()));
+ }
+ checkState(jarsDir.isDirectory(), "Examples jars directory '%s' does not exist.",
+ jarsDir.getAbsolutePath());
+
+ for (File f: jarsDir.listFiles()) {
+ examplesJars.add(f.getAbsolutePath());
+ }
+ return examplesJars;
+ }
private class OptionParser extends SparkSubmitOptionParser {
@@ -367,6 +413,14 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
if (allowsMixedArguments) {
appArgs.add(opt);
return true;
+ } else if (isExample) {
+ String className = opt;
+ if (!className.startsWith(EXAMPLE_CLASS_PREFIX)) {
+ className = EXAMPLE_CLASS_PREFIX + className;
+ }
+ mainClass = className;
+ appResource = "spark-internal";
+ return false;
} else {
checkArgument(!opt.startsWith("-"), "Unrecognized option: %s", opt);
sparkArgs.add(opt);
@@ -376,8 +430,10 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
@Override
protected void handleExtraArgs(List<String> extra) {
- for (String arg : extra) {
- sparkArgs.add(arg);
+ if (isExample) {
+ appArgs.addAll(extra);
+ } else {
+ sparkArgs.addAll(extra);
}
}
diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
index 00f967122b..b7f4f2efc5 100644
--- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
+++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
@@ -151,6 +151,24 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite {
assertEquals("arg1", cmd.get(cmd.size() - 1));
}
+ @Test
+ public void testExamplesRunner() throws Exception {
+ List<String> sparkSubmitArgs = Arrays.asList(
+ SparkSubmitCommandBuilder.RUN_EXAMPLE,
+ parser.MASTER + "=foo",
+ parser.DEPLOY_MODE + "=bar",
+ "SparkPi",
+ "42");
+
+ Map<String, String> env = new HashMap<String, String>();
+ List<String> cmd = buildCommand(sparkSubmitArgs, env);
+ assertEquals("foo", findArgValue(cmd, parser.MASTER));
+ assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE));
+ assertEquals(SparkSubmitCommandBuilder.EXAMPLE_CLASS_PREFIX + "SparkPi",
+ findArgValue(cmd, parser.CLASS));
+ assertEquals("42", cmd.get(cmd.size() - 1));
+ }
+
private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) throws Exception {
String deployMode = isDriver ? "client" : "cluster";