Skip to content
Snippets Groups Projects
Commit ca716209 authored by Matei Zaharia's avatar Matei Zaharia
Browse files

Merge pull request #857 from mateiz/assembly

Change build and run instructions to use assemblies
parents 15287766 e11bc182
No related branches found
No related tags found
No related merge requests found
Showing
with 248 additions and 145 deletions
...@@ -40,3 +40,4 @@ checkpoint ...@@ -40,3 +40,4 @@ checkpoint
derby.log derby.log
dist/ dist/
spark-*-bin.tar.gz spark-*-bin.tar.gz
unit-tests.log
...@@ -16,20 +16,20 @@ Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is ...@@ -16,20 +16,20 @@ Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is
built using Simple Build Tool (SBT), which is packaged with it. To build built using Simple Build Tool (SBT), which is packaged with it. To build
Spark and its example programs, run: Spark and its example programs, run:
sbt/sbt package assembly sbt/sbt assembly
Spark also supports building using Maven. If you would like to build using Maven, Spark also supports building using Maven. If you would like to build using Maven,
see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html)
in the spark documentation.. in the Spark documentation..
To run Spark, you will need to have Scala's bin directory in your `PATH`, or To run Spark, you will need to have Scala's bin directory in your `PATH`, or
you will need to set the `SCALA_HOME` environment variable to point to where you will need to set the `SCALA_HOME` environment variable to point to where
you've installed Scala. Scala must be accessible through one of these you've installed Scala. Scala must be accessible through one of these
methods on your cluster's worker nodes as well as its master. methods on your cluster's worker nodes as well as its master.
To run one of the examples, use `./run <class> <params>`. For example: To run one of the examples, use `./run-example <class> <params>`. For example:
./run spark.examples.SparkLR local[2] ./run-example spark.examples.SparkLR local[2]
will run the Logistic Regression example locally on 2 CPUs. will run the Logistic Regression example locally on 2 CPUs.
...@@ -52,19 +52,19 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop ...@@ -52,19 +52,19 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop
versions without YARN, use: versions without YARN, use:
# Apache Hadoop 1.2.1 # Apache Hadoop 1.2.1
$ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt package assembly $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly
# Cloudera CDH 4.2.0 with MapReduce v1 # Cloudera CDH 4.2.0 with MapReduce v1
$ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt package assembly $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly
For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions
with YARN, also set `SPARK_WITH_YARN=true`: with YARN, also set `SPARK_WITH_YARN=true`:
# Apache Hadoop 2.0.5-alpha # Apache Hadoop 2.0.5-alpha
$ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true sbt/sbt package assembly $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true sbt/sbt assembly
# Cloudera CDH 4.2.0 with MapReduce v2 # Cloudera CDH 4.2.0 with MapReduce v2
$ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_WITH_YARN=true sbt/sbt package assembly $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_WITH_YARN=true sbt/sbt assembly
For convenience, these variables may also be set through the `conf/spark-env.sh` file For convenience, these variables may also be set through the `conf/spark-env.sh` file
described below. described below.
......
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.
b7924aabe9c5e63f0a4d8bbd17019534c7ec014e
<?xml version="1.0" encoding="UTF-8"?>
<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<modelVersion>4.0.0</modelVersion>
<groupId>net.sf.py4j</groupId>
<artifactId>py4j</artifactId>
<version>0.7</version>
<description>POM was created from install:install-file</description>
</project>
<?xml version="1.0" encoding="UTF-8"?>
<metadata>
<groupId>net.sf.py4j</groupId>
<artifactId>py4j</artifactId>
<versioning>
<release>0.7</release>
<versions>
<version>0.7</version>
</versions>
<lastUpdated>20130828020333</lastUpdated>
</versioning>
</metadata>
<?xml version="1.0" encoding="UTF-8"?> <?xml version="1.0" encoding="UTF-8"?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<parent> <parent>
...@@ -13,29 +30,13 @@ ...@@ -13,29 +30,13 @@
<name>Spark Project Assembly</name> <name>Spark Project Assembly</name>
<url>http://spark-project.org/</url> <url>http://spark-project.org/</url>
<build> <repositories>
<plugins> <!-- A repository in the local filesystem for the Py4J JAR, which is not in Maven central -->
<plugin> <repository>
<groupId>org.apache.maven.plugins</groupId> <id>lib</id>
<artifactId>maven-assembly-plugin</artifactId> <url>file://${project.basedir}/lib</url>
<version>2.4</version> </repository>
<executions> </repositories>
<execution>
<id>dist</id>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
<configuration>
<descriptors>
<descriptor>src/main/assembly/assembly.xml</descriptor>
</descriptors>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
<dependencies> <dependencies>
<dependency> <dependency>
...@@ -63,5 +64,96 @@ ...@@ -63,5 +64,96 @@
<artifactId>spark-streaming</artifactId> <artifactId>spark-streaming</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
<dependency>
<groupId>net.sf.py4j</groupId>
<artifactId>py4j</artifactId>
<version>0.7</version>
</dependency>
</dependencies> </dependencies>
<build>
<plugins>
<!-- Use the shade plugin to create a big JAR with all the dependencies -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<configuration>
<shadedArtifactAttached>false</shadedArtifactAttached>
<outputFile>${project.build.directory}/scala-${scala.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
<artifactSet>
<includes>
<include>*:*</include>
</includes>
</artifactSet>
<filters>
<filter>
<artifact>*:*</artifact>
<excludes>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
</excludes>
</filter>
</filters>
</configuration>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>reference.conf</resource>
</transformer>
</transformers>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
<profiles>
<profile>
<id>hadoop2-yarn</id>
<dependencies>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-yarn</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
<id>bigtop-dist</id>
<!-- This profile uses the assembly plugin to create a special "dist" package for BigTop
that contains Spark but not the Hadoop JARs it depends on. -->
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-assembly-plugin</artifactId>
<version>2.4</version>
<executions>
<execution>
<id>dist</id>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
<configuration>
<descriptors>
<descriptor>src/main/assembly/assembly.xml</descriptor>
</descriptors>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project> </project>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<assembly> <assembly>
<id>dist</id> <id>dist</id>
<formats> <formats>
...@@ -36,7 +52,8 @@ ...@@ -36,7 +52,8 @@
</directory> </directory>
<outputDirectory>/bin</outputDirectory> <outputDirectory>/bin</outputDirectory>
<includes> <includes>
<include>run*</include> <include>run-example*</include>
<include>spark-class*</include>
<include>spark-shell*</include> <include>spark-shell*</include>
<include>spark-executor*</include> <include>spark-executor*</include>
</includes> </includes>
......
...@@ -30,79 +30,25 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then ...@@ -30,79 +30,25 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then
. $FWDIR/conf/spark-env.sh . $FWDIR/conf/spark-env.sh
fi fi
CORE_DIR="$FWDIR/core"
REPL_DIR="$FWDIR/repl"
REPL_BIN_DIR="$FWDIR/repl-bin"
EXAMPLES_DIR="$FWDIR/examples"
BAGEL_DIR="$FWDIR/bagel"
MLLIB_DIR="$FWDIR/mllib"
TOOLS_DIR="$FWDIR/tools"
YARN_DIR="$FWDIR/yarn"
STREAMING_DIR="$FWDIR/streaming"
PYSPARK_DIR="$FWDIR/python"
# Build up classpath # Build up classpath
CLASSPATH="$SPARK_CLASSPATH" CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
function dev_classpath {
CLASSPATH="$CLASSPATH:$FWDIR/conf"
CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes"
if [ -n "$SPARK_TESTING" ] ; then
CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
fi
CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources"
CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
if [ -e "$FWDIR/lib_managed" ]; then
CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*"
CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*"
fi
CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
# Add the shaded JAR for Maven builds
if [ -e $REPL_BIN_DIR/target ]; then
for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded.jar'`; do
CLASSPATH="$CLASSPATH:$jar"
done
# The shaded JAR doesn't contain examples, so include those separately
for jar in `find "$EXAMPLES_DIR/target" -name 'spark-examples*[0-9T].jar'`; do
CLASSPATH="$CLASSPATH:$jar"
done
fi
CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$TOOLS_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$YARN_DIR/target/scala-$SCALA_VERSION/classes"
for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
CLASSPATH="$CLASSPATH:$jar"
done
# Add Scala standard library
if [ -z "$SCALA_LIBRARY_PATH" ]; then
if [ -z "$SCALA_HOME" ]; then
echo "SCALA_HOME is not set" >&2
exit 1
fi
SCALA_LIBRARY_PATH="$SCALA_HOME/lib"
fi
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar"
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar"
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar"
}
function release_classpath {
CLASSPATH="$CLASSPATH:$FWDIR/jars/*"
}
if [ -f "$FWDIR/RELEASE" ]; then if [ -f "$FWDIR/RELEASE" ]; then
release_classpath ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
else else
dev_classpath ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
fi
CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1
if [[ $SPARK_TESTING == 1 ]]; then
CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes"
fi fi
# Add hadoop conf dir - else FileSystem.*, etc fail ! # Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail !
# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts # Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
# the configurtion files. # the configurtion files.
if [ "x" != "x$HADOOP_CONF_DIR" ]; then if [ "x" != "x$HADOOP_CONF_DIR" ]; then
......
...@@ -87,7 +87,7 @@ TEST_LOG_DIR=$? ...@@ -87,7 +87,7 @@ TEST_LOG_DIR=$?
if [ "${TEST_LOG_DIR}" = "0" ]; then if [ "${TEST_LOG_DIR}" = "0" ]; then
rm -f $SPARK_LOG_DIR/.spark_test rm -f $SPARK_LOG_DIR/.spark_test
else else
chown $SPARK_IDENT_STRING $SPARK_LOG_DIR chown $SPARK_IDENT_STRING $SPARK_LOG_DIR
fi fi
if [ "$SPARK_PID_DIR" = "" ]; then if [ "$SPARK_PID_DIR" = "" ]; then
...@@ -109,7 +109,7 @@ fi ...@@ -109,7 +109,7 @@ fi
case $startStop in case $startStop in
(start) (start)
mkdir -p "$SPARK_PID_DIR" mkdir -p "$SPARK_PID_DIR"
if [ -f $pid ]; then if [ -f $pid ]; then
...@@ -128,11 +128,11 @@ case $startStop in ...@@ -128,11 +128,11 @@ case $startStop in
echo starting $command, logging to $log echo starting $command, logging to $log
echo "Spark Daemon: $command" > $log echo "Spark Daemon: $command" > $log
cd "$SPARK_PREFIX" cd "$SPARK_PREFIX"
nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/run $command "$@" >> "$log" 2>&1 < /dev/null & nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
echo $! > $pid echo $! > $pid
sleep 1; head "$log" sleep 1; head "$log"
;; ;;
(stop) (stop)
if [ -f $pid ]; then if [ -f $pid ]; then
......
...@@ -153,6 +153,7 @@ private[spark] class ExecutorRunner( ...@@ -153,6 +153,7 @@ private[spark] class ExecutorRunner(
// Launch the process // Launch the process
val command = buildCommandSeq() val command = buildCommandSeq()
logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
val builder = new ProcessBuilder(command: _*).directory(executorDir) val builder = new ProcessBuilder(command: _*).directory(executorDir)
val env = builder.environment() val env = builder.environment()
for ((key, value) <- appDesc.command.environment) { for ((key, value) <- appDesc.command.environment) {
......
...@@ -125,14 +125,14 @@ private[spark] class CoarseMesosSchedulerBackend( ...@@ -125,14 +125,14 @@ private[spark] class CoarseMesosSchedulerBackend(
StandaloneSchedulerBackend.ACTOR_NAME) StandaloneSchedulerBackend.ACTOR_NAME)
val uri = System.getProperty("spark.executor.uri") val uri = System.getProperty("spark.executor.uri")
if (uri == null) { 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( command.setValue("\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
} else { } else {
// Grab everything to the first '.'. We'll use that and '*' to // Grab everything to the first '.'. We'll use that and '*' to
// glob the directory "correctly". // glob the directory "correctly".
val basename = uri.split('/').last.split('.').head 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)) basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
} }
......
...@@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator { ...@@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator {
def main(args: Array[String]) { def main(args: Array[String]) {
if (args.length < 2) { 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) System.exit(1)
} }
val master = args(0) val master = args(0)
......
...@@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts { ...@@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts {
val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
forAll(masters) { (master: String) => forAll(masters) { (master: String) =>
failAfter(30 seconds) { failAfter(30 seconds) {
Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master), Utils.execute(Seq("./spark-class", "spark.DriverWithoutCleanup", master),
new File(System.getenv("SPARK_HOME"))) new File(System.getenv("SPARK_HOME")))
} }
} }
......
...@@ -158,4 +158,4 @@ trait Message[K] { ...@@ -158,4 +158,4 @@ trait Message[K] {
## Where to Go from Here ## Where to Go from Here
Two example jobs, PageRank and shortest path, are included in `bagel/src/main/scala/spark/bagel/examples`. You can run them by passing the class name to the `run` script included in Spark -- for example, `./run spark.bagel.examples.WikipediaPageRank`. Each example program prints usage help when run without any arguments. Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark -- for example, `./run-example spark.examples.bagel.WikipediaPageRank`. Each example program prints usage help when run without any arguments.
...@@ -8,53 +8,51 @@ title: Building Spark with Maven ...@@ -8,53 +8,51 @@ title: Building Spark with Maven
Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer. Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer.
## Specifying the Hadoop version ##
To enable support for HDFS and other Hadoop-supported storage systems, specify the exact Hadoop version by setting the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. ## Setting up Maven's Memory Usage ##
For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings:
# Apache Hadoop 1.2.1 export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"
$ mvn -Dhadoop.version=1.2.1 clean install
# Cloudera CDH 4.2.0 with MapReduce v1 If you don't run this, you may see errors like the following:
$ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 clean install
For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, enable the "hadoop2-yarn" profile: [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes...
[ERROR] PermGen space -> [Help 1]
# Apache Hadoop 2.0.5-alpha
$ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean install
# Cloudera CDH 4.2.0 with MapReduce v2 [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes...
$ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 clean install [ERROR] Java heap space -> [Help 1]
You can fix this by setting the `MAVEN_OPTS` variable as discussed before.
## Spark Tests in Maven ## ## Specifying the Hadoop version ##
Tests are run by default via the scalatest-maven-plugin. With this you can do things like: Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default.
Skip test execution (but not compilation): For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use:
$ mvn -Dhadoop.version=... -DskipTests clean install # Apache Hadoop 1.2.1
$ mvn -Dhadoop.version=1.2.1 -DskipTests clean package
To run a specific test suite: # Cloudera CDH 4.2.0 with MapReduce v1
$ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package
$ mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should also enable the "hadoop2-yarn" profile:
# Apache Hadoop 2.0.5-alpha
$ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -DskipTests clean package
## Setting up JVM Memory Usage Via Maven ## # Cloudera CDH 4.2.0 with MapReduce v2
$ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package
You might run into the following errors if you're using a vanilla installation of Maven:
[INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... ## Spark Tests in Maven ##
[ERROR] PermGen space -> [Help 1]
[INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`.
[ERROR] Java heap space -> [Help 1]
To fix these, you can do the following: The ScalaTest plugin also supports running only a specific test suite as follows:
export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=128M" $ mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test
## Continuous Compilation ## ## Continuous Compilation ##
...@@ -63,8 +61,7 @@ We use the scala-maven-plugin which supports incremental and continuous compilat ...@@ -63,8 +61,7 @@ We use the scala-maven-plugin which supports incremental and continuous compilat
$ mvn scala:cc $ mvn scala:cc
…should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively.
## Using With IntelliJ IDEA ## ## Using With IntelliJ IDEA ##
...@@ -72,8 +69,8 @@ This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the ...@@ -72,8 +69,8 @@ This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the
## Building Spark Debian Packages ## ## Building Spark Debian Packages ##
It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the deb profile: It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the following profiles:
$ mvn -Pdeb clean install $ mvn -Prepl-bin -Pdeb clean package
The debian package can then be found under repl/target. We added the short commit hash to the file name so that we can distinguish individual packages build for SNAPSHOT versions. The debian package can then be found under repl/target. We added the short commit hash to the file name so that we can distinguish individual packages build for SNAPSHOT versions.
...@@ -20,16 +20,16 @@ of these methods on slave nodes on your cluster. ...@@ -20,16 +20,16 @@ of these methods on slave nodes on your cluster.
Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run
sbt/sbt package sbt/sbt assembly
Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](building-with-maven.html). Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](building-with-maven.html).
# Testing the Build # Testing the Build
Spark comes with a number of sample programs in the `examples` directory. Spark comes with a number of sample programs in the `examples` directory.
To run one of the samples, use `./run <class> <params>` in the top-level Spark directory To run one of the samples, use `./run-example <class> <params>` in the top-level Spark directory
(the `run` script sets up the appropriate paths and launches that program). (the `run` script sets up the appropriate paths and launches that program).
For example, `./run spark.examples.SparkPi` will run a sample program that estimates Pi. Each of the For example, `./run-example spark.examples.SparkPi` will run a sample program that estimates Pi. Each of the
examples prints usage help if no params are given. examples prints usage help if no params are given.
Note that all of the sample programs take a `<master>` parameter specifying the cluster URL Note that all of the sample programs take a `<master>` parameter specifying the cluster URL
......
...@@ -190,6 +190,6 @@ We hope to generate documentation with Java-style syntax in the future. ...@@ -190,6 +190,6 @@ We hope to generate documentation with Java-style syntax in the future.
Spark includes several sample programs using the Java API in Spark includes several sample programs using the Java API in
[`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/examples). You can run them by passing the class name to the [`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/examples). You can run them by passing the class name to the
`run` script included in Spark -- for example, `./run `run-example` script included in Spark -- for example, `./run-example
spark.examples.JavaWordCount`. Each example program prints usage help when run spark.examples.JavaWordCount`. Each example program prints usage help when run
without any arguments. without any arguments.
...@@ -70,7 +70,7 @@ The script automatically adds the `pyspark` package to the `PYTHONPATH`. ...@@ -70,7 +70,7 @@ The script automatically adds the `pyspark` package to the `PYTHONPATH`.
The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
{% highlight bash %} {% highlight bash %}
$ sbt/sbt package $ sbt/sbt assembly
$ ./pyspark $ ./pyspark
{% endhighlight %} {% endhighlight %}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment