diff options
author | Marcelo Vanzin <vanzin@cloudera.com> | 2016-03-15 09:44:48 -0700 |
---|---|---|
committer | Marcelo Vanzin <vanzin@cloudera.com> | 2016-03-15 09:44:51 -0700 |
commit | 48978abfa4d8f2cf79a4b053cc8bc7254cc2d61b (patch) | |
tree | 009d0b0870a8eb6a185cf192acb172d2d91b540c /bin/run-example | |
parent | bd5365bbe9ff6518cde9402ee8843ec1002fff5b (diff) | |
download | spark-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 'bin/run-example')
-rwxr-xr-x | bin/run-example | 55 |
1 files changed, 2 insertions, 53 deletions
diff --git a/bin/run-example b/bin/run-example index e1b0d5789b..dd0e3c4120 100755 --- a/bin/run-example +++ b/bin/run-example @@ -21,56 +21,5 @@ if [ -z "${SPARK_HOME}" ]; then export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" fi -EXAMPLES_DIR="${SPARK_HOME}"/examples - -. "${SPARK_HOME}"/bin/load-spark-env.sh - -if [ -n "$1" ]; then - EXAMPLE_CLASS="$1" - shift -else - echo "Usage: ./bin/run-example <example-class> [example-args]" 1>&2 - echo " - set MASTER=XX to use a specific master" 1>&2 - echo " - can use abbreviated example class name relative to com.apache.spark.examples" 1>&2 - echo " (e.g. SparkPi, mllib.LinearRegression, streaming.KinesisWordCountASL)" 1>&2 - exit 1 -fi - -if [ -f "${SPARK_HOME}/RELEASE" ]; then - JAR_PATH="${SPARK_HOME}/lib" -else - JAR_PATH="${EXAMPLES_DIR}/target/scala-${SPARK_SCALA_VERSION}" -fi - -JAR_COUNT=0 - -for f in "${JAR_PATH}"/spark-examples-*hadoop*.jar; do - if [[ ! -e "$f" ]]; then - echo "Failed to find Spark examples assembly in ${SPARK_HOME}/lib or ${SPARK_HOME}/examples/target" 1>&2 - echo "You need to build Spark before running this program" 1>&2 - exit 1 - fi - SPARK_EXAMPLES_JAR="$f" - JAR_COUNT=$((JAR_COUNT+1)) -done - -if [ "$JAR_COUNT" -gt "1" ]; then - echo "Found multiple Spark examples assembly jars in ${JAR_PATH}" 1>&2 - ls "${JAR_PATH}"/spark-examples-*hadoop*.jar 1>&2 - echo "Please remove all but one jar." 1>&2 - exit 1 -fi - -export SPARK_EXAMPLES_JAR - -EXAMPLE_MASTER=${MASTER:-"local[*]"} - -if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then - EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" -fi - -exec "${SPARK_HOME}"/bin/spark-submit \ - --master $EXAMPLE_MASTER \ - --class $EXAMPLE_CLASS \ - "$SPARK_EXAMPLES_JAR" \ - "$@" +export _SPARK_CMD_USAGE="Usage: ./bin/run-example [options] example-class [example args]" +exec "${SPARK_HOME}"/bin/spark-submit run-example "$@" |