From 53cd50c0699efc8733518658100c62426b425de2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 23 Aug 2013 23:30:17 -0700 Subject: Change build and run instructions to use assemblies This commit makes Spark invocation saner by using an assembly JAR to find all of Spark's dependencies instead of adding all the JARs in lib_managed. It also packages the examples into an assembly and uses that as SPARK_EXAMPLES_JAR. Finally, it replaces the old "run" script with two better-named scripts: "run-examples" for examples, and "spark-class" for Spark internal classes (e.g. REPL, master, etc). This is also designed to minimize the confusion people have in trying to use "run" to run their own classes; it's not meant to do that, but now at least if they look at it, they can modify run-examples to do a decent job for them. As part of this, Bagel's examples are also now properly moved to the examples package instead of bagel. --- .../main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala | 2 +- core/src/test/scala/spark/DriverSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core') diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 6ebbb5ec9b..ebfc21392d 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -125,7 +125,7 @@ private[spark] class CoarseMesosSchedulerBackend( StandaloneSchedulerBackend.ACTOR_NAME) val uri = System.getProperty("spark.executor.uri") if (uri == null) { - val runScript = new File(sparkHome, "run").getCanonicalPath + val runScript = new File(sparkHome, "spark-class").getCanonicalPath command.setValue("\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format( runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index ed16b9d8ef..553c0309f6 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts { val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(30 seconds) { - Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master), + Utils.execute(Seq("./spark-class", "spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } } -- cgit v1.2.3 From ab0e625d9e0abd62a20754125952e3a00f2c275a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 22 Aug 2013 23:02:09 -0700 Subject: Fix PySpark for assembly run and include it in dist --- .gitignore | 1 + core/lib/PY4J_LICENSE.txt | 27 +++++++++++++++++++++++++++ core/lib/PY4J_VERSION.txt | 1 + core/lib/py4j0.7.jar | Bin 0 -> 103286 bytes make-distribution.sh | 5 ++++- pyspark | 12 ++++++++---- python/lib/py4j0.7.jar | Bin 103286 -> 0 bytes 7 files changed, 41 insertions(+), 5 deletions(-) create mode 100644 core/lib/PY4J_LICENSE.txt create mode 100644 core/lib/PY4J_VERSION.txt create mode 100644 core/lib/py4j0.7.jar delete mode 100644 python/lib/py4j0.7.jar (limited to 'core') diff --git a/.gitignore b/.gitignore index 00fbff6a2c..e1f64a1133 100644 --- a/.gitignore +++ b/.gitignore @@ -40,3 +40,4 @@ checkpoint derby.log dist/ spark-*-bin.tar.gz +unit-tests.log diff --git a/core/lib/PY4J_LICENSE.txt b/core/lib/PY4J_LICENSE.txt new file mode 100644 index 0000000000..a70279ca14 --- /dev/null +++ b/core/lib/PY4J_LICENSE.txt @@ -0,0 +1,27 @@ + +Copyright (c) 2009-2011, Barthelemy Dagenais All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +- Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +- Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +- The name of the author may not be used to endorse or promote products +derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. diff --git a/core/lib/PY4J_VERSION.txt b/core/lib/PY4J_VERSION.txt new file mode 100644 index 0000000000..04a0cd52a8 --- /dev/null +++ b/core/lib/PY4J_VERSION.txt @@ -0,0 +1 @@ +b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/core/lib/py4j0.7.jar b/core/lib/py4j0.7.jar new file mode 100644 index 0000000000..73b7ddb7d1 Binary files /dev/null and b/core/lib/py4j0.7.jar differ diff --git a/make-distribution.sh b/make-distribution.sh index df7bbf1e74..92b2706126 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -94,11 +94,14 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE cp $FWDIR/assembly/target/*/*assembly*.jar "$DISTDIR/jars/" # Copy other things +mkdir "$DISTDIR"/conf +cp -r "$FWDIR/conf/*.template" "$DISTDIR" cp -r "$FWDIR/bin" "$DISTDIR" -cp -r "$FWDIR/conf" "$DISTDIR" +cp -r "$FWDIR/python" "$DISTDIR" cp "$FWDIR/spark-class" "$DISTDIR" cp "$FWDIR/spark-shell" "$DISTDIR" cp "$FWDIR/spark-executor" "$DISTDIR" +cp "$FWDIR/pyspark" "$DISTDIR" if [ "$MAKE_TGZ" == "true" ]; then diff --git a/pyspark b/pyspark index 801239c108..155ccd4fdf 100755 --- a/pyspark +++ b/pyspark @@ -24,10 +24,14 @@ FWDIR="$(cd `dirname $0`; pwd)" export SPARK_HOME="$FWDIR" # Exit if the user hasn't compiled Spark -if [ ! -e "$SPARK_HOME/repl/target" ]; then - echo "Failed to find Spark classes in $SPARK_HOME/repl/target" >&2 - echo "You need to compile Spark before running this program" >&2 - exit 1 +if [ ! -f "$FWDIR/RELEASE" ]; then + # Exit if the user hasn't compiled Spark + ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar >& /dev/null + if [[ $? != 0 ]]; then + echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 + echo "You need to compile Spark before running this program" >&2 + exit 1 + fi fi # Load environment variables from conf/spark-env.sh, if it exists diff --git a/python/lib/py4j0.7.jar b/python/lib/py4j0.7.jar deleted file mode 100644 index 73b7ddb7d1..0000000000 Binary files a/python/lib/py4j0.7.jar and /dev/null differ -- cgit v1.2.3 From aab345c46318264a58ee86704567a2586de32b13 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 11:37:00 -0700 Subject: Fix finding of assembly JAR, as well as some pointers to ./run --- bin/compute-classpath.sh | 2 +- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 1 + .../scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala | 2 +- core/src/main/scala/spark/ui/UIWorkloadGenerator.scala | 2 +- .../src/main/scala/spark/streaming/examples/ActorWordCount.scala | 4 ++-- .../src/main/scala/spark/streaming/examples/HdfsWordCount.scala | 2 +- .../src/main/scala/spark/streaming/examples/KafkaWordCount.scala | 2 +- .../src/main/scala/spark/streaming/examples/NetworkWordCount.scala | 2 +- .../scala/spark/streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala | 4 ++-- .../spark/streaming/examples/clickstream/PageViewGenerator.scala | 4 ++-- .../scala/spark/streaming/examples/clickstream/PageViewStream.scala | 6 +++--- sbt/sbt | 2 +- 13 files changed, 18 insertions(+), 17 deletions(-) (limited to 'core') diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 5dc86c51a4..c7819d4932 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -35,7 +35,7 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" if [ -f "$FWDIR/RELEASE" ]; then ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar` else - ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar` + ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar` fi CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 5e53d95ac2..34665ce451 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -153,6 +153,7 @@ private[spark] class ExecutorRunner( // Launch the process val command = buildCommandSeq() + logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) val builder = new ProcessBuilder(command: _*).directory(executorDir) val env = builder.environment() for ((key, value) <- appDesc.command.environment) { diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index ebfc21392d..eef3ee1425 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -132,7 +132,7 @@ private[spark] class CoarseMesosSchedulerBackend( // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head - command.setValue("cd %s*; ./run spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + command.setValue("cd %s*; ./spark-class spark.executor.StandaloneExecutorBackend %s %s %s %d".format( basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index f96419520f..5ff0572f0a 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator { def main(args: Array[String]) { if (args.length < 2) { - println("usage: ./run spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + println("usage: ./spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } val master = args(0) diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index f97174aeae..05d3176478 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -132,9 +132,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./run spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` * and then run the example - * `$ ./run spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala index f5baec242d..30af01a26f 100644 --- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala @@ -28,7 +28,7 @@ import spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./run spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./run-example spark.streaming.examples.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 4929703ba2..d9c76d1a33 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -37,7 +37,7 @@ import spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./run spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 150fb5eb9c..b29d79aac5 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -29,7 +29,7 @@ import spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999` + * `$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index 33ab324732..b709fc3c87 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -29,7 +29,7 @@ import spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala index e264fae609..a0cae06c30 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -60,9 +60,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./run spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./run spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ object ZeroMQWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala index 375d5c9d22..dd36bbbf32 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -37,8 +37,8 @@ object PageView { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * */ object PageViewGenerator { val pages = Map("http://foo.com/" -> .7, diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index a24342bebf..152da23489 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -25,9 +25,9 @@ import spark.SparkContext._ * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 - * */ + * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + */ object PageViewStream { def main(args: Array[String]) { if (args.length != 3) { diff --git a/sbt/sbt b/sbt/sbt index 2227bc4696..c31a0280ff 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -25,4 +25,4 @@ fi export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd) export SPARK_TESTING=1 # To put test classes on classpath -java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m $EXTRA_ARGS $SBT_OPTS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m $EXTRA_ARGS $SBT_OPTS -jar "$SPARK_HOME"/sbt/sbt-launch-*.jar "$@" -- cgit v1.2.3 From 666d93c294458cb056cb590eb11bb6cf979861e5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 19:23:54 -0700 Subject: Update Maven build to create assemblies expected by new scripts This includes the following changes: - The "assembly" package now builds in Maven by default, and creates an assembly containing both hadoop-client and Spark, unlike the old BigTop distribution assembly that skipped hadoop-client - There is now a bigtop-dist package to build the old BigTop assembly - The repl-bin package is no longer built by default since the scripts don't reply on it; instead it can be enabled with -Prepl-bin - Py4J is now included in the assembly/lib folder as a local Maven repo, so that the Maven package can link to it - run-example now adds the original Spark classpath as well because the Maven examples assembly lists spark-core and such as provided - The various Maven projects add a spark-yarn dependency correctly --- assembly/lib/PY4J_LICENSE.txt | 27 ++++ assembly/lib/PY4J_VERSION.txt | 1 + assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar | Bin 0 -> 103286 bytes assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom | 9 ++ .../lib/net/sf/py4j/py4j/maven-metadata-local.xml | 12 ++ assembly/pom.xml | 138 +++++++++++++++++---- assembly/src/main/assembly/assembly.xml | 19 ++- core/lib/PY4J_LICENSE.txt | 27 ---- core/lib/PY4J_VERSION.txt | 1 - core/lib/py4j0.7.jar | Bin 103286 -> 0 bytes examples/pom.xml | 62 ++++++++- pom.xml | 16 +-- project/SparkBuild.scala | 4 +- run-example | 9 +- 14 files changed, 250 insertions(+), 75 deletions(-) create mode 100644 assembly/lib/PY4J_LICENSE.txt create mode 100644 assembly/lib/PY4J_VERSION.txt create mode 100644 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar create mode 100644 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom create mode 100644 assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml delete mode 100644 core/lib/PY4J_LICENSE.txt delete mode 100644 core/lib/PY4J_VERSION.txt delete mode 100644 core/lib/py4j0.7.jar (limited to 'core') diff --git a/assembly/lib/PY4J_LICENSE.txt b/assembly/lib/PY4J_LICENSE.txt new file mode 100644 index 0000000000..a70279ca14 --- /dev/null +++ b/assembly/lib/PY4J_LICENSE.txt @@ -0,0 +1,27 @@ + +Copyright (c) 2009-2011, Barthelemy Dagenais All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +- Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +- Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +- The name of the author may not be used to endorse or promote products +derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. diff --git a/assembly/lib/PY4J_VERSION.txt b/assembly/lib/PY4J_VERSION.txt new file mode 100644 index 0000000000..04a0cd52a8 --- /dev/null +++ b/assembly/lib/PY4J_VERSION.txt @@ -0,0 +1 @@ +b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar new file mode 100644 index 0000000000..73b7ddb7d1 Binary files /dev/null and b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar differ diff --git a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom new file mode 100644 index 0000000000..1c730e19b4 --- /dev/null +++ b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom @@ -0,0 +1,9 @@ + + + 4.0.0 + net.sf.py4j + py4j + 0.7 + POM was created from install:install-file + diff --git a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml b/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml new file mode 100644 index 0000000000..6942ff45e7 --- /dev/null +++ b/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml @@ -0,0 +1,12 @@ + + + net.sf.py4j + py4j + + 0.7 + + 0.7 + + 20130828020333 + + diff --git a/assembly/pom.xml b/assembly/pom.xml index ca20ccadba..74990b6361 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 @@ -13,29 +30,13 @@ Spark Project Assembly http://spark-project.org/ - - - - org.apache.maven.plugins - maven-assembly-plugin - 2.4 - - - dist - package - - single - - - - src/main/assembly/assembly.xml - - - - - - - + + + + lib + file://${project.basedir}/lib + + @@ -63,5 +64,96 @@ spark-streaming ${project.version} + + net.sf.py4j + py4j + 0.7 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/scala-${scala.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + + + + + + + + hadoop2-yarn + + + org.spark-project + spark-yarn + ${project.version} + + + + + bigtop-dist + + + + + org.apache.maven.plugins + maven-assembly-plugin + 2.4 + + + dist + package + + single + + + + src/main/assembly/assembly.xml + + + + + + + + + diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index 14485b7181..4543b52c93 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -1,3 +1,19 @@ + dist @@ -36,7 +52,8 @@ /bin - run* + run-example* + spark-class* spark-shell* spark-executor* diff --git a/core/lib/PY4J_LICENSE.txt b/core/lib/PY4J_LICENSE.txt deleted file mode 100644 index a70279ca14..0000000000 --- a/core/lib/PY4J_LICENSE.txt +++ /dev/null @@ -1,27 +0,0 @@ - -Copyright (c) 2009-2011, Barthelemy Dagenais All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -- Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. - -- Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -- The name of the author may not be used to endorse or promote products -derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE -ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE -LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR -CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF -SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS -INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN -CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) -ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE -POSSIBILITY OF SUCH DAMAGE. diff --git a/core/lib/PY4J_VERSION.txt b/core/lib/PY4J_VERSION.txt deleted file mode 100644 index 04a0cd52a8..0000000000 --- a/core/lib/PY4J_VERSION.txt +++ /dev/null @@ -1 +0,0 @@ -b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/core/lib/py4j0.7.jar b/core/lib/py4j0.7.jar deleted file mode 100644 index 73b7ddb7d1..0000000000 Binary files a/core/lib/py4j0.7.jar and /dev/null differ diff --git a/examples/pom.xml b/examples/pom.xml index d24bd404fa..687fbcca8f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -36,21 +36,25 @@ org.spark-project spark-core ${project.version} + provided org.spark-project spark-streaming ${project.version} + provided org.spark-project spark-mllib ${project.version} + provided org.spark-project spark-bagel ${project.version} + provided org.apache.hbase @@ -67,10 +71,6 @@ - - org.scala-lang - scala-library - org.eclipse.jetty jetty-server @@ -126,13 +126,63 @@ + + + + hadoop2-yarn + + + org.spark-project + spark-yarn + ${project.version} + provided + + + + + target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes - org.scalatest - scalatest-maven-plugin + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/scala-${scala.version}/${project.artifactId}-assembly-${project.version}.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + diff --git a/pom.xml b/pom.xml index ea9548359f..e2fd54a966 100644 --- a/pom.xml +++ b/pom.xml @@ -62,6 +62,7 @@ tools streaming repl + assembly @@ -75,7 +76,7 @@ 1.7.2 1.2.17 1.0.4 - + 0.94.6 64m 512m @@ -743,21 +744,10 @@ - assembly + repl-bin false - - assembly - - - - expensive-modules - - - !noExpensive - - repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8797e65b8d..2e26812671 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -41,7 +41,7 @@ object SparkBuild extends Build { .dependsOn(core, bagel, mllib) dependsOn(maybeYarn: _*) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, bagel, streaming) + .dependsOn(core, mllib, bagel, streaming) dependsOn(maybeYarn: _*) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) @@ -261,7 +261,7 @@ object SparkBuild extends Build { def yarnSettings = sharedSettings ++ Seq( name := "spark-yarn" - ) ++ extraYarnSettings ++ assemblySettings ++ extraAssemblySettings + ) ++ extraYarnSettings // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN). diff --git a/run-example b/run-example index e1b26257e1..ccd4356bdf 100755 --- a/run-example +++ b/run-example @@ -54,6 +54,11 @@ if [[ -z $SPARK_EXAMPLES_JAR ]]; then exit 1 fi +# Since the examples JAR ideally shouldn't include spark-core (that dependency should be +# "provided"), also add our standard Spark classpath, built using compute-classpath.sh. +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH" + # Find java binary if [ -n "${JAVA_HOME}" ]; then RUNNER="${JAVA_HOME}/bin/java" @@ -68,9 +73,9 @@ fi if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then echo -n "Spark Command: " - echo "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@" + echo "$RUNNER" -cp "$CLASSPATH" "$@" echo "========================================" echo fi -exec "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@" +exec "$RUNNER" -cp "$CLASSPATH" "$@" -- cgit v1.2.3