зеркало из https://github.com/microsoft/spark.git
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.
This commit is contained in:
Родитель
abdbacf252
Коммит
53cd50c069
|
@ -20,16 +20,16 @@ Spark and its example programs, run:
|
|||
|
||||
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)
|
||||
in the spark documentation..
|
||||
in the Spark documentation..
|
||||
|
||||
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've installed Scala. Scala must be accessible through one of these
|
||||
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.
|
||||
|
||||
|
|
|
@ -30,79 +30,25 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then
|
|||
. $FWDIR/conf/spark-env.sh
|
||||
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
|
||||
CLASSPATH="$SPARK_CLASSPATH"
|
||||
|
||||
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/*"
|
||||
}
|
||||
|
||||
CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
|
||||
if [ -f "$FWDIR/RELEASE" ]; then
|
||||
release_classpath
|
||||
ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
|
||||
else
|
||||
dev_classpath
|
||||
ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.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
|
||||
|
||||
# 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
|
||||
# the configurtion files.
|
||||
if [ "x" != "x$HADOOP_CONF_DIR" ]; then
|
||||
|
|
|
@ -87,7 +87,7 @@ TEST_LOG_DIR=$?
|
|||
if [ "${TEST_LOG_DIR}" = "0" ]; then
|
||||
rm -f $SPARK_LOG_DIR/.spark_test
|
||||
else
|
||||
chown $SPARK_IDENT_STRING $SPARK_LOG_DIR
|
||||
chown $SPARK_IDENT_STRING $SPARK_LOG_DIR
|
||||
fi
|
||||
|
||||
if [ "$SPARK_PID_DIR" = "" ]; then
|
||||
|
@ -109,7 +109,7 @@ fi
|
|||
case $startStop in
|
||||
|
||||
(start)
|
||||
|
||||
|
||||
mkdir -p "$SPARK_PID_DIR"
|
||||
|
||||
if [ -f $pid ]; then
|
||||
|
@ -128,11 +128,11 @@ case $startStop in
|
|||
echo starting $command, logging to $log
|
||||
echo "Spark Daemon: $command" > $log
|
||||
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
|
||||
sleep 1; head "$log"
|
||||
;;
|
||||
|
||||
|
||||
(stop)
|
||||
|
||||
if [ -f $pid ]; then
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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")))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -158,4 +158,4 @@ trait Message[K] {
|
|||
|
||||
## 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.
|
||||
|
|
|
@ -27,9 +27,9 @@ Spark also supports building using Maven. If you would like to build using Maven
|
|||
# Testing the Build
|
||||
|
||||
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).
|
||||
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.
|
||||
|
||||
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.
|
|||
|
||||
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
|
||||
`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
|
||||
without any arguments.
|
||||
|
|
|
@ -15,9 +15,9 @@ We need a consolidated spark core jar (which bundles all the required dependenci
|
|||
This can be built either through sbt or via maven.
|
||||
|
||||
- Building spark assembled jar via sbt.
|
||||
Enable YARN support by setting `SPARK_WITH_YARN=true` when invoking sbt:
|
||||
Enable YARN support by setting `SPARK_YARN=true` when invoking sbt:
|
||||
|
||||
SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true ./sbt/sbt clean assembly
|
||||
SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt clean assembly
|
||||
|
||||
The assembled jar would typically be something like :
|
||||
`./yarn/target/spark-yarn-assembly-0.8.0-SNAPSHOT.jar`
|
||||
|
@ -55,7 +55,7 @@ This would be used to connect to the cluster, write to the dfs and submit jobs t
|
|||
|
||||
The command to launch the YARN Client is as follows:
|
||||
|
||||
SPARK_JAR=<SPARK_YARN_JAR_FILE> ./run spark.deploy.yarn.Client \
|
||||
SPARK_JAR=<SPARK_YARN_JAR_FILE> ./spark-class spark.deploy.yarn.Client \
|
||||
--jar <YOUR_APP_JAR_FILE> \
|
||||
--class <APP_MAIN_CLASS> \
|
||||
--args <APP_MAIN_ARGUMENTS> \
|
||||
|
@ -67,7 +67,7 @@ The command to launch the YARN Client is as follows:
|
|||
|
||||
For example:
|
||||
|
||||
SPARK_JAR=./yarn/target/spark-yarn-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \
|
||||
SPARK_JAR=./yarn/target/spark-yarn-assembly-{{site.SPARK_VERSION}}.jar ./spark-class spark.deploy.yarn.Client \
|
||||
--jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar \
|
||||
--class spark.examples.SparkPi \
|
||||
--args yarn-standalone \
|
||||
|
|
|
@ -356,7 +356,7 @@ res2: Int = 10
|
|||
# Where to Go from Here
|
||||
|
||||
You can see some [example Spark programs](http://www.spark-project.org/examples.html) on the Spark website.
|
||||
In addition, Spark includes several sample programs in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run` script included in Spark -- for example, `./run spark.examples.SparkPi`. Each example program prints usage help when run without any arguments.
|
||||
In addition, Spark includes several sample programs in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark -- for example, `./run-example spark.examples.SparkPi`. Each example program prints usage help when run without any arguments.
|
||||
|
||||
For help on optimizing your program, the [configuration](configuration.html) and
|
||||
[tuning](tuning.html) guides provide information on best practices. They are especially important for
|
||||
|
|
|
@ -20,7 +20,7 @@ Compile Spark with `sbt package` as described in the [Getting Started Guide](ind
|
|||
|
||||
You can start a standalone master server by executing:
|
||||
|
||||
./run spark.deploy.master.Master
|
||||
./spark-class spark.deploy.master.Master
|
||||
|
||||
Once started, the master will print out a `spark://IP:PORT` URL for itself, which you can use to connect workers to it,
|
||||
or pass as the "master" argument to `SparkContext` to connect a job to the cluster. You can also find this URL on
|
||||
|
@ -28,7 +28,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by
|
|||
|
||||
Similarly, you can start one or more workers and connect them to the master via:
|
||||
|
||||
./run spark.deploy.worker.Worker spark://IP:PORT
|
||||
./spark-class spark.deploy.worker.Worker spark://IP:PORT
|
||||
|
||||
Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default).
|
||||
You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS).
|
||||
|
|
|
@ -234,7 +234,7 @@ $ nc -lk 9999
|
|||
Then, in a different terminal, you can start NetworkWordCount by using
|
||||
|
||||
{% highlight bash %}
|
||||
$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999
|
||||
$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999
|
||||
{% endhighlight %}
|
||||
|
||||
This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
|
||||
|
@ -272,7 +272,7 @@ Time: 1357008430000 ms
|
|||
</td>
|
||||
</table>
|
||||
|
||||
You can find more examples in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
|
||||
You can find more examples in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run-example spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
|
||||
|
||||
# DStream Persistence
|
||||
Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`.
|
||||
|
|
|
@ -47,6 +47,11 @@
|
|||
<artifactId>spark-mllib</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<artifactId>spark-bagel</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase</artifactId>
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.bagel.examples
|
||||
package spark.examples.bagel
|
||||
|
||||
import spark._
|
||||
import spark.SparkContext._
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.bagel.examples
|
||||
package spark.examples.bagel
|
||||
|
||||
import spark._
|
||||
import spark.SparkContext._
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.bagel.examples
|
||||
package spark.examples.bagel
|
||||
|
||||
import spark._
|
||||
import serializer.{DeserializationStream, SerializationStream, SerializerInstance}
|
|
@ -47,7 +47,7 @@ VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z
|
|||
|
||||
# Initialize defaults
|
||||
SPARK_HADOOP_VERSION=1.0.4
|
||||
SPARK_WITH_YARN=false
|
||||
SPARK_YARN=false
|
||||
MAKE_TGZ=false
|
||||
|
||||
# Parse arguments
|
||||
|
@ -58,7 +58,7 @@ while (( "$#" )); do
|
|||
shift
|
||||
;;
|
||||
--with-yarn)
|
||||
SPARK_WITH_YARN=true
|
||||
SPARK_YARN=true
|
||||
;;
|
||||
--tgz)
|
||||
MAKE_TGZ=true
|
||||
|
@ -74,7 +74,7 @@ else
|
|||
fi
|
||||
|
||||
echo "Hadoop version set to $SPARK_HADOOP_VERSION"
|
||||
if [ "$SPARK_WITH_YARN" == "true" ]; then
|
||||
if [ "$SPARK_YARN" == "true" ]; then
|
||||
echo "YARN enabled"
|
||||
else
|
||||
echo "YARN disabled"
|
||||
|
@ -82,21 +82,22 @@ fi
|
|||
|
||||
# Build fat JAR
|
||||
export SPARK_HADOOP_VERSION
|
||||
export SPARK_WITH_YARN
|
||||
"$FWDIR/sbt/sbt" "repl/assembly"
|
||||
export SPARK_YARN
|
||||
"$FWDIR/sbt/sbt" "assembly/assembly"
|
||||
|
||||
# Make directories
|
||||
rm -rf "$DISTDIR"
|
||||
mkdir -p "$DISTDIR/jars"
|
||||
echo "$VERSION" > "$DISTDIR/RELEASE"
|
||||
echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE"
|
||||
|
||||
# Copy jars
|
||||
cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/"
|
||||
cp $FWDIR/assembly/target/*/*assembly*.jar "$DISTDIR/jars/"
|
||||
|
||||
# Copy other things
|
||||
cp -r "$FWDIR/bin" "$DISTDIR"
|
||||
cp -r "$FWDIR/conf" "$DISTDIR"
|
||||
cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR"
|
||||
cp "$FWDIR/spark-class" "$DISTDIR"
|
||||
cp "$FWDIR/spark-shell" "$DISTDIR"
|
||||
cp "$FWDIR/spark-executor" "$DISTDIR"
|
||||
|
||||
|
||||
|
|
|
@ -26,30 +26,35 @@ import AssemblyKeys._
|
|||
object SparkBuild extends Build {
|
||||
// Hadoop version to build against. For example, "1.0.4" for Apache releases, or
|
||||
// "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set
|
||||
// through the environment variables SPARK_HADOOP_VERSION and SPARK_WITH_YARN.
|
||||
// through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN.
|
||||
val DEFAULT_HADOOP_VERSION = "1.0.4"
|
||||
val DEFAULT_WITH_YARN = false
|
||||
val DEFAULT_YARN = false
|
||||
|
||||
// HBase version; set as appropriate.
|
||||
val HBASE_VERSION = "0.94.6"
|
||||
|
||||
lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects:_*)
|
||||
lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects: _*)
|
||||
|
||||
lazy val core = Project("core", file("core"), settings = coreSettings)
|
||||
|
||||
lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn(core) dependsOn(bagel) dependsOn(mllib) dependsOn(maybeYarn:_*)
|
||||
lazy val repl = Project("repl", file("repl"), settings = replSettings)
|
||||
.dependsOn(core, bagel, mllib) dependsOn(maybeYarn: _*)
|
||||
|
||||
lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) dependsOn(mllib)
|
||||
lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
|
||||
.dependsOn(core, mllib, bagel, streaming)
|
||||
|
||||
lazy val tools = Project("tools", file("tools"), settings = examplesSettings) dependsOn (core) dependsOn (streaming)
|
||||
lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming)
|
||||
|
||||
lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core)
|
||||
lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core)
|
||||
|
||||
lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core)
|
||||
lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core)
|
||||
|
||||
lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn (core)
|
||||
lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
|
||||
|
||||
lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn (core)
|
||||
lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
|
||||
|
||||
lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
|
||||
.dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*)
|
||||
|
||||
// A configuration to set an alternative publishLocalConfiguration
|
||||
lazy val MavenCompile = config("m2r") extend(Compile)
|
||||
|
@ -57,15 +62,16 @@ object SparkBuild extends Build {
|
|||
|
||||
// Allows build configuration to be set through environment variables
|
||||
lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
|
||||
lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_WITH_YARN") match {
|
||||
case None => DEFAULT_WITH_YARN
|
||||
lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match {
|
||||
case None => DEFAULT_YARN
|
||||
case Some(v) => v.toBoolean
|
||||
}
|
||||
|
||||
// Conditionally include the yarn sub-project
|
||||
lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
|
||||
lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
|
||||
lazy val allProjects = Seq[ProjectReference](core, repl, examples, bagel, streaming, mllib, tools) ++ maybeYarnRef
|
||||
lazy val allProjects = Seq[ProjectReference](
|
||||
core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef
|
||||
|
||||
def sharedSettings = Defaults.defaultSettings ++ Seq(
|
||||
organization := "org.spark-project",
|
||||
|
@ -100,8 +106,8 @@ object SparkBuild extends Build {
|
|||
<url>http://spark-project.org/</url>
|
||||
<licenses>
|
||||
<license>
|
||||
<name>BSD License</name>
|
||||
<url>https://github.com/mesos/spark/blob/master/LICENSE</url>
|
||||
<name>Apache 2.0 License</name>
|
||||
<url>http://www.apache.org/licenses/LICENSE-2.0.html</url>
|
||||
<distribution>repo</distribution>
|
||||
</license>
|
||||
</licenses>
|
||||
|
@ -195,7 +201,7 @@ object SparkBuild extends Build {
|
|||
"com.twitter" % "chill_2.9.3" % "0.3.1",
|
||||
"com.twitter" % "chill-java" % "0.3.1"
|
||||
)
|
||||
) ++ assemblySettings ++ extraAssemblySettings
|
||||
)
|
||||
|
||||
def rootSettings = sharedSettings ++ Seq(
|
||||
publish := {}
|
||||
|
@ -204,7 +210,7 @@ object SparkBuild extends Build {
|
|||
def replSettings = sharedSettings ++ Seq(
|
||||
name := "spark-repl",
|
||||
libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _)
|
||||
) ++ assemblySettings ++ extraAssemblySettings
|
||||
)
|
||||
|
||||
def examplesSettings = sharedSettings ++ Seq(
|
||||
name := "spark-examples",
|
||||
|
@ -223,7 +229,7 @@ object SparkBuild extends Build {
|
|||
exclude("org.apache.cassandra.deps", "avro")
|
||||
excludeAll(excludeSnappy)
|
||||
)
|
||||
)
|
||||
) ++ assemblySettings ++ extraAssemblySettings
|
||||
|
||||
def toolsSettings = sharedSettings ++ Seq(
|
||||
name := "spark-tools"
|
||||
|
@ -251,7 +257,7 @@ object SparkBuild extends Build {
|
|||
"org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
|
||||
"com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty)
|
||||
)
|
||||
) ++ assemblySettings ++ extraAssemblySettings
|
||||
)
|
||||
|
||||
def yarnSettings = sharedSettings ++ Seq(
|
||||
name := "spark-yarn"
|
||||
|
@ -271,7 +277,13 @@ object SparkBuild extends Build {
|
|||
)
|
||||
)
|
||||
|
||||
def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq(
|
||||
def assemblyProjSettings = sharedSettings ++ Seq(
|
||||
name := "spark-assembly",
|
||||
jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }
|
||||
) ++ assemblySettings ++ extraAssemblySettings
|
||||
|
||||
def extraAssemblySettings() = Seq(
|
||||
test in assembly := {},
|
||||
mergeStrategy in assembly := {
|
||||
case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
|
||||
case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard
|
||||
|
|
|
@ -15,4 +15,4 @@
|
|||
# limitations under the License.
|
||||
#
|
||||
|
||||
sbt.version=0.12.3
|
||||
sbt.version=0.12.4
|
||||
|
|
|
@ -4,7 +4,7 @@ resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/release
|
|||
|
||||
resolvers += "Spray Repository" at "http://repo.spray.cc/"
|
||||
|
||||
addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.5")
|
||||
addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.1")
|
||||
|
||||
addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0")
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ SPARK_HOME = os.environ["SPARK_HOME"]
|
|||
def launch_gateway():
|
||||
# Launch the Py4j gateway using Spark's run command so that we pick up the
|
||||
# proper classpath and SPARK_MEM settings from spark-env.sh
|
||||
command = [os.path.join(SPARK_HOME, "run"), "py4j.GatewayServer",
|
||||
command = [os.path.join(SPARK_HOME, "spark-class"), "py4j.GatewayServer",
|
||||
"--die-on-broken-pipe", "0"]
|
||||
proc = Popen(command, stdout=PIPE, stdin=PIPE)
|
||||
# Determine which ephemeral port the server started on:
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
SCALA_VERSION=2.9.3
|
||||
|
||||
# Figure out where the Scala framework is installed
|
||||
FWDIR="$(cd `dirname $0`; pwd)"
|
||||
|
||||
# Export this as SPARK_HOME
|
||||
export SPARK_HOME="$FWDIR"
|
||||
|
||||
# Load environment variables from conf/spark-env.sh, if it exists
|
||||
if [ -e $FWDIR/conf/spark-env.sh ] ; then
|
||||
. $FWDIR/conf/spark-env.sh
|
||||
fi
|
||||
|
||||
if [ -z "$1" ]; then
|
||||
echo "Usage: run-example <example-class> [<args>]" >&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
|
||||
# to avoid the -sources and -doc packages that are built by publish-local.
|
||||
EXAMPLES_DIR="$FWDIR"/examples
|
||||
SPARK_EXAMPLES_JAR=""
|
||||
if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9T].jar ]; then
|
||||
# Use the JAR from the SBT build
|
||||
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9T].jar`
|
||||
fi
|
||||
if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9T].jar ]; then
|
||||
# Use the JAR from the Maven build
|
||||
# TODO: this also needs to become an assembly!
|
||||
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9T].jar`
|
||||
fi
|
||||
if [[ -z $SPARK_EXAMPLES_JAR ]]; then
|
||||
echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
|
||||
echo "You need to compile Spark before running this program" >&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Find java binary
|
||||
if [ -n "${JAVA_HOME}" ]; then
|
||||
RUNNER="${JAVA_HOME}/bin/java"
|
||||
else
|
||||
if [ `command -v java` ]; then
|
||||
RUNNER="java"
|
||||
else
|
||||
echo "JAVA_HOME is not set" >&2
|
||||
exit 1
|
||||
fi
|
||||
fi
|
||||
|
||||
if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
|
||||
echo -n "Spark Command: "
|
||||
echo "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@"
|
||||
echo "========================================"
|
||||
echo
|
||||
fi
|
||||
|
||||
exec "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@"
|
|
@ -66,51 +66,27 @@ case "$1" in
|
|||
;;
|
||||
esac
|
||||
|
||||
# Figure out whether to run our class with java or with the scala launcher.
|
||||
# In most cases, we'd prefer to execute our process with java because scala
|
||||
# creates a shell script as the parent of its Java process, which makes it
|
||||
# hard to kill the child with stuff like Process.destroy(). However, for
|
||||
# the Spark shell, the wrapper is necessary to properly reset the terminal
|
||||
# when we exit, so we allow it to set a variable to launch with scala.
|
||||
# We still fall back on java for the shell if this is a "release" created
|
||||
# from make-distribution.sh since it's possible scala is not installed
|
||||
# but we have everything we need to run the shell.
|
||||
if [[ "$SPARK_LAUNCH_WITH_SCALA" == "1" && ! -f "$FWDIR/RELEASE" ]]; then
|
||||
if [ "$SCALA_HOME" ]; then
|
||||
RUNNER="${SCALA_HOME}/bin/scala"
|
||||
else
|
||||
if [ `command -v scala` ]; then
|
||||
RUNNER="scala"
|
||||
else
|
||||
echo "SCALA_HOME is not set and scala is not in PATH" >&2
|
||||
exit 1
|
||||
fi
|
||||
fi
|
||||
# Find the java binary
|
||||
if [ -n "${JAVA_HOME}" ]; then
|
||||
RUNNER="${JAVA_HOME}/bin/java"
|
||||
else
|
||||
if [ -n "${JAVA_HOME}" ]; then
|
||||
RUNNER="${JAVA_HOME}/bin/java"
|
||||
if [ `command -v java` ]; then
|
||||
RUNNER="java"
|
||||
else
|
||||
if [ `command -v java` ]; then
|
||||
RUNNER="java"
|
||||
else
|
||||
echo "JAVA_HOME is not set" >&2
|
||||
exit 1
|
||||
fi
|
||||
echo "JAVA_HOME is not set" >&2
|
||||
exit 1
|
||||
fi
|
||||
if [[ ! -f "$FWDIR/RELEASE" && -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
|
||||
if [[ ! -f "$FWDIR/RELEASE" && -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
|
||||
|
||||
# Figure out how much memory to use per executor and set it as an environment
|
||||
# variable so that our process sees it and can report it to Mesos
|
||||
if [ -z "$SPARK_MEM" ] ; then
|
||||
SPARK_MEM="512m"
|
||||
fi
|
||||
# Set SPARK_MEM if it isn't already set since we also use it for this process
|
||||
SPARK_MEM=${SPARK_MEM:-512m}
|
||||
export SPARK_MEM
|
||||
|
||||
# Set JAVA_OPTS to be able to load native libraries and to set heap size
|
||||
|
@ -125,51 +101,24 @@ export JAVA_OPTS
|
|||
# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
|
||||
|
||||
if [ ! -f "$FWDIR/RELEASE" ]; then
|
||||
CORE_DIR="$FWDIR/core"
|
||||
EXAMPLES_DIR="$FWDIR/examples"
|
||||
REPL_DIR="$FWDIR/repl"
|
||||
|
||||
# Exit if the user hasn't compiled Spark
|
||||
if [ ! -e "$CORE_DIR/target" ]; then
|
||||
echo "Failed to find Spark classes in $CORE_DIR/target" >&2
|
||||
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
|
||||
|
||||
if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then
|
||||
echo "Failed to find Spark classes in $REPL_DIR/target" >&2
|
||||
echo "You need to compile Spark repl module before running this program" >&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
|
||||
# to avoid the -sources and -doc packages that are built by publish-local.
|
||||
if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then
|
||||
# Use the JAR from the SBT build
|
||||
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar`
|
||||
fi
|
||||
if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then
|
||||
# Use the JAR from the Maven build
|
||||
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
|
||||
fi
|
||||
fi
|
||||
|
||||
# Compute classpath using external script
|
||||
CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
|
||||
export CLASSPATH
|
||||
|
||||
if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
|
||||
EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS
|
||||
else
|
||||
# The JVM doesn't read JAVA_OPTS by default so we need to pass it in
|
||||
EXTRA_ARGS="$JAVA_OPTS"
|
||||
fi
|
||||
|
||||
command="$RUNNER -cp \"$CLASSPATH\" $EXTRA_ARGS $@"
|
||||
if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
|
||||
echo "Spark Command: $command"
|
||||
echo -n "Spark Command: "
|
||||
echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
|
||||
echo "========================================"
|
||||
echo
|
||||
fi
|
||||
|
||||
exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@"
|
||||
exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
|
|
@ -19,4 +19,4 @@
|
|||
|
||||
FWDIR="`dirname $0`"
|
||||
echo "Running spark-executor with framework dir = $FWDIR"
|
||||
exec $FWDIR/run spark.executor.MesosExecutorBackend
|
||||
exec $FWDIR/spark-class spark.executor.MesosExecutorBackend
|
||||
|
|
|
@ -79,8 +79,7 @@ if [[ ! $? ]]; then
|
|||
saved_stty=""
|
||||
fi
|
||||
|
||||
export SPARK_LAUNCH_WITH_SCALA=${SPARK_LAUNCH_WITH_SCALA:-1}
|
||||
$FWDIR/run $OPTIONS spark.repl.Main "$@"
|
||||
$FWDIR/spark-class $OPTIONS spark.repl.Main "$@"
|
||||
|
||||
# record the exit status lest it be overwritten:
|
||||
# then reenable echo and propagate the code.
|
||||
|
|
Загрузка…
Ссылка в новой задаче