diff options
author | Patrick Wendell <pwendell@gmail.com> | 2014-04-21 10:26:33 -0700 |
---|---|---|
committer | Patrick Wendell <pwendell@gmail.com> | 2014-04-21 10:26:33 -0700 |
commit | fb98488fc8e68cc84f6e0750fd4e9e29029879d2 (patch) | |
tree | eba99b56bea8ec2e357020a413bf9cf04a4e3308 /dev/audit-release | |
parent | 3a390bfd80f80739b9d847780eccc443fc2dc0ea (diff) | |
download | spark-fb98488fc8e68cc84f6e0750fd4e9e29029879d2.tar.gz spark-fb98488fc8e68cc84f6e0750fd4e9e29029879d2.tar.bz2 spark-fb98488fc8e68cc84f6e0750fd4e9e29029879d2.zip |
Clean up and simplify Spark configuration
Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements:
1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file.
2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath.
3. Adds ability to set these same variables for the driver using `spark-submit`.
4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`.
5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node.
Author: Patrick Wendell <pwendell@gmail.com>
Closes #299 from pwendell/config-cleanup and squashes the following commits:
127f301 [Patrick Wendell] Improvements to testing
a006464 [Patrick Wendell] Moving properties file template.
b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf
0086939 [Patrick Wendell] Minor style fixes
af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs
b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide
af0adf7 [Patrick Wendell] Automatically add user jar
a56b125 [Patrick Wendell] Responses to Tom's review
d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup
a762901 [Patrick Wendell] Fixing test failures
ffa00fe [Patrick Wendell] Review feedback
fda0301 [Patrick Wendell] Note
308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN
e83cd8f [Patrick Wendell] Changes to allow re-use of test applications
be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set
c2a2909 [Patrick Wendell] Test compile fixes
4ee6f9d [Patrick Wendell] Making YARN doc changes consistent
afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors.
b08893b [Patrick Wendell] Additional improvements.
ace4ead [Patrick Wendell] Responses to review feedback.
b72d183 [Patrick Wendell] Review feedback for spark env file
46555c1 [Patrick Wendell] Review feedback and import clean-ups
437aed1 [Patrick Wendell] Small fix
761ebcd [Patrick Wendell] Library path and classpath for drivers
7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script
5b0ba8e [Patrick Wendell] Don't ship executor envs
84cc5e5 [Patrick Wendell] Small clean-up
1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings
4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH
6eaf7d0 [Patrick Wendell] executorJavaOpts
0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN
ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
Diffstat (limited to 'dev/audit-release')
4 files changed, 20 insertions, 6 deletions
diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index fa2f02dfec..4a816d4101 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -114,6 +114,7 @@ os.chdir("blank_sbt_build") os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY +os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: os.environ["SPARK_MODULE"] = module ret = run_cmd("sbt clean update", exit_on_failure=False) diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 53fe43215e..a89b0d7d38 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -24,8 +24,13 @@ import org.apache.spark.SparkContext._ object SimpleApp { def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Spark App").setMaster(master) + case None => new SparkConf().setAppName("Simple Spark App") + } val logFile = "input.txt" - val sc = new SparkContext("local", "Simple App") + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala index da08e014eb..24c7f8d667 100644 --- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala +++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala @@ -17,14 +17,20 @@ package main.scala -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD object GraphXApp { def main(args: Array[String]) { - val sc = new SparkContext("local", "Simple GraphX App") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple GraphX App").setMaster(master) + case None => new SparkConf().setAppName("Simple Graphx App") + } + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) + val users: RDD[(VertexId, (String, String))] = sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index 3d0722d2ac..a1d8971abe 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -27,10 +27,12 @@ import org.apache.spark.streaming._ object SparkStreamingExample { def main(args: Array[String]) { - val conf = new SparkConf(true) - .setMaster("local[2]") - .setAppName("Streaming test") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Streaming App").setMaster(master) + case None => new SparkConf().setAppName("Simple Streaming App") + } val ssc = new StreamingContext(conf, Seconds(1)) + SparkContext.jarOfClass(this.getClass).foreach(ssc.sparkContext.addJar) val seen = ListBuffer[RDD[Int]]() val rdd1 = ssc.sparkContext.makeRDD(1 to 100, 10) |