diff --git a/.gitignore b/.gitignore
index b3c4363af038e21e670ea5530a9ddfe0b8de8c69..39635d7eefbe7311aaaba62e9259cc6041265bf6 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,7 +1,10 @@
 *~
 *.swp
+*.ipr
 *.iml
+*.iws
 .idea/
+sbt/*.jar
 .settings
 .cache
 /build/
diff --git a/README.md b/README.md
index 1550a8b5512d976b8a22cba2fe4293211886ae79..c840a68f76b17c15dd0f6106a825ca503f43f3a9 100644
--- a/README.md
+++ b/README.md
@@ -13,20 +13,22 @@ This README file only contains basic setup instructions.
 ## Building
 
 Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT),
-which is packaged with it. To build Spark and its example programs, run:
+which can be obtained [here](http://www.scala-sbt.org). If SBT is installed we
+will use the system version of sbt otherwise we will attempt to download it
+automatically. To build Spark and its example programs, run:
 
-    sbt/sbt assembly
+    ./sbt/sbt assembly
 
 Once you've built Spark, the easiest way to start using it is the shell:
 
-    ./spark-shell
+    ./bin/spark-shell
 
-Or, for the Python API, the Python shell (`./pyspark`).
+Or, for the Python API, the Python shell (`./bin/pyspark`).
 
 Spark also comes with several sample programs in the `examples` directory.
-To run one of them, use `./run-example <class> <params>`. For example:
+To run one of them, use `./bin/run-example <class> <params>`. For example:
 
-    ./run-example org.apache.spark.examples.SparkLR local[2]
+    ./bin/run-example org.apache.spark.examples.SparkLR local[2]
 
 will run the Logistic Regression example locally on 2 CPUs.
 
@@ -36,7 +38,13 @@ All of the Spark samples take a `<master>` parameter that is the cluster URL
 to connect to. This can be a mesos:// or spark:// URL, or "local" to run
 locally with one thread, or "local[N]" to run locally with N threads.
 
+## Running tests
 
+Testing first requires [Building](#building) Spark. Once Spark is built, tests
+can be run using:
+
+`./sbt/sbt test`
+ 
 ## A Note About Hadoop Versions
 
 Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported
diff --git a/assembly/lib/PY4J_LICENSE.txt b/assembly/lib/PY4J_LICENSE.txt
deleted file mode 100644
index a70279ca14ae3cdfd3f166871eb888f4d6e112ac..0000000000000000000000000000000000000000
--- a/assembly/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/assembly/lib/PY4J_VERSION.txt b/assembly/lib/PY4J_VERSION.txt
deleted file mode 100644
index 04a0cd52a8d9c0bff32eb2d907f09b41969232ed..0000000000000000000000000000000000000000
--- a/assembly/lib/PY4J_VERSION.txt
+++ /dev/null
@@ -1 +0,0 @@
-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
deleted file mode 100644
index 73b7ddb7d115be2fb1ba4cf8003fe90c662feef6..0000000000000000000000000000000000000000
Binary files a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar and /dev/null 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
deleted file mode 100644
index 1c730e19b4b2fff945777cbd089dbfe2f70450c6..0000000000000000000000000000000000000000
--- a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-<?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>
diff --git a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml b/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml
deleted file mode 100644
index 6942ff45e75f194509598c9204752ea4560fdf7e..0000000000000000000000000000000000000000
--- a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?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>
diff --git a/assembly/pom.xml b/assembly/pom.xml
index fc2adc1fbb5d280898b8a1a21214db5e9aa19286..54a25910ced7db8f20112b9f99c945f71ddf0b8b 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -67,7 +67,7 @@
     <dependency>
       <groupId>net.sf.py4j</groupId>
       <artifactId>py4j</artifactId>
-      <version>0.7</version>
+      <version>0.8.1</version>
     </dependency>
   </dependencies>
 
@@ -108,12 +108,12 @@
                 <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
                   <resource>META-INF/services/org.apache.hadoop.fs.FileSystem</resource>
                 </transformer>
-              </transformers>
-              <transformers>
-                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
                 <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
                   <resource>reference.conf</resource>
                 </transformer>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+                  <resource>log4j.properties</resource>
+                </transformer>
               </transformers>
             </configuration>
           </execution>
@@ -124,7 +124,17 @@
 
   <profiles>
     <profile>
-      <id>hadoop2-yarn</id>
+      <id>yarn-alpha</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-yarn-alpha_${scala.binary.version}</artifactId>
+          <version>${project.version}</version>
+        </dependency>
+      </dependencies>
+    </profile>
+    <profile>
+      <id>yarn</id>
       <dependencies>
         <dependency>
           <groupId>org.apache.spark</groupId>
diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml
index 47d3fa93d07657dc2401a4f2a8365b0a83fb8edb..6af383db65d47fae74fc85063eb5bd06a490a98c 100644
--- a/assembly/src/main/assembly/assembly.xml
+++ b/assembly/src/main/assembly/assembly.xml
@@ -39,23 +39,20 @@
     </fileSet>
     <fileSet>
       <directory>
-        ${project.parent.basedir}/bin/
+        ${project.parent.basedir}/sbin/
       </directory>
-      <outputDirectory>/bin</outputDirectory>
+      <outputDirectory>/sbin</outputDirectory>
       <includes>
         <include>**/*</include>
       </includes>
     </fileSet>
     <fileSet>
       <directory>
-        ${project.parent.basedir}
+        ${project.parent.basedir}/bin/
       </directory>
       <outputDirectory>/bin</outputDirectory>
       <includes>
-        <include>run-example*</include>
-        <include>spark-class*</include>
-        <include>spark-shell*</include>
-        <include>spark-executor*</include>
+        <include>**/*</include>
       </includes>
     </fileSet>
   </fileSets>
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
index 9e3e10ecaad982f8a64a510168f2340ab3b07020..4f60bff19cb93adcef2355f3a22aa89e38b9c51c 100644
--- a/bin/compute-classpath.cmd
+++ b/bin/compute-classpath.cmd
@@ -29,7 +29,7 @@ rem Load environment variables from conf\spark-env.cmd, if it exists
 if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
 
 rem Build up classpath
-set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf
+set CLASSPATH=%FWDIR%conf
 if exist "%FWDIR%RELEASE" (
   for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
     set ASSEMBLY_JAR=%%d
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index 40555089fcfe4608f53130c0744094a06090c2c4..0c823104215aa353f8087af65a2d2b6d29d20d21 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -26,7 +26,7 @@ SCALA_VERSION=2.10
 FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
diff --git a/pyspark b/bin/pyspark
similarity index 96%
rename from pyspark
rename to bin/pyspark
index 12cc926ddafa588425f06f8c8da8bc9f64e0dc3d..d6810f4686bf56ae9f9caa1c620c2a2633135f21 100755
--- a/pyspark
+++ b/bin/pyspark
@@ -18,7 +18,7 @@
 #
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Export this as SPARK_HOME
 export SPARK_HOME="$FWDIR"
@@ -37,7 +37,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then
 fi
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
diff --git a/pyspark.cmd b/bin/pyspark.cmd
similarity index 100%
rename from pyspark.cmd
rename to bin/pyspark.cmd
diff --git a/pyspark2.cmd b/bin/pyspark2.cmd
similarity index 98%
rename from pyspark2.cmd
rename to bin/pyspark2.cmd
index 21f9a3438878ad8e99b062ee2ee171b1bb587dbb..95791095ec932e175f8b2b0e137cbb485219d2b9 100644
--- a/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.10
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
diff --git a/run-example b/bin/run-example
similarity index 88%
rename from run-example
rename to bin/run-example
index a78192d31d384194f757ee213e5f916652d9ea80..2e9d51440bd5d42e9bbc44a56f7f949ef0f446d0 100755
--- a/run-example
+++ b/bin/run-example
@@ -25,13 +25,13 @@ esac
 SCALA_VERSION=2.10
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+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
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
@@ -45,20 +45,15 @@ fi
 EXAMPLES_DIR="$FWDIR"/examples
 SPARK_EXAMPLES_JAR=""
 if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
-  # Use the JAR from the SBT build
   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
 fi
-if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].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-9Tg].jar`
-fi
 if [[ -z $SPARK_EXAMPLES_JAR ]]; then
   echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
   echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
   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`
diff --git a/run-example.cmd b/bin/run-example.cmd
similarity index 100%
rename from run-example.cmd
rename to bin/run-example.cmd
diff --git a/run-example2.cmd b/bin/run-example2.cmd
similarity index 97%
rename from run-example2.cmd
rename to bin/run-example2.cmd
index d4ad98d6e7ba85510ddec713e6d89f51073a4ab9..6861334cb07e505d01ce90e6eba968c7e5f43892 100644
--- a/run-example2.cmd
+++ b/bin/run-example2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.10
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
@@ -49,7 +49,7 @@ if "x%SPARK_EXAMPLES_JAR%"=="x" (
 
 rem Compute Spark classpath using external script
 set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
 set DONT_PRINT_CLASSPATH=0
 set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH%
 
diff --git a/spark-class b/bin/spark-class
similarity index 92%
rename from spark-class
rename to bin/spark-class
index 802e4aa1045e483fc96f061b44640d53f781df9b..c4225a392d6dae2374465f6a6f1f37c3ea2f6cee 100755
--- a/spark-class
+++ b/bin/spark-class
@@ -25,13 +25,13 @@ esac
 SCALA_VERSION=2.10
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+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
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
@@ -92,7 +92,7 @@ JAVA_OPTS="$OUR_JAVA_OPTS"
 JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
 JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM"
 # Load extra JAVA_OPTS from conf/java-opts, if it exists
-if [ -e $FWDIR/conf/java-opts ] ; then
+if [ -e "$FWDIR/conf/java-opts" ] ; then
   JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
 fi
 export JAVA_OPTS
@@ -129,11 +129,16 @@ fi
 
 # Compute classpath using external script
 CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
-CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR"
+
+if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
+  CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR"
+fi
 
 if $cygwin; then
   CLASSPATH=`cygpath -wp $CLASSPATH`
-  export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR`
+  if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
+    export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR`
+  fi
 fi
 export CLASSPATH
 
diff --git a/spark-class.cmd b/bin/spark-class.cmd
similarity index 100%
rename from spark-class.cmd
rename to bin/spark-class.cmd
diff --git a/spark-class2.cmd b/bin/spark-class2.cmd
similarity index 97%
rename from spark-class2.cmd
rename to bin/spark-class2.cmd
index a60c17d0502eef84800609d5d0a9aaa87be2a4d3..460e6614766f8220e02efd2dc037f461df8b98e0 100644
--- a/spark-class2.cmd
+++ b/bin/spark-class2.cmd
@@ -17,10 +17,10 @@ rem See the License for the specific language governing permissions and
 rem limitations under the License.
 rem
 
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
@@ -73,7 +73,7 @@ for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar"
 
 rem Compute classpath using external script
 set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
 set DONT_PRINT_CLASSPATH=0
 set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR%
 
diff --git a/spark-shell b/bin/spark-shell
similarity index 93%
rename from spark-shell
rename to bin/spark-shell
index d20af0fb39f2b6ae39f3d085734ac511d0ecb8b1..e6885b51ef5676c834c6c13340a95b44875a07de 100755
--- a/spark-shell
+++ b/bin/spark-shell
@@ -32,7 +32,7 @@ esac
 # Enter posix mode for bash
 set -o posix
 
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 for o in "$@"; do
   if [ "$1" = "-c" -o "$1" = "--cores" ]; then
@@ -90,10 +90,10 @@ if $cygwin; then
     # "Backspace sends ^H" setting in "Keys" section of the Mintty options
     # (see https://github.com/sbt/sbt/issues/562).
     stty -icanon min 1 -echo > /dev/null 2>&1
-    $FWDIR/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
+    $FWDIR/bin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
     stty icanon echo > /dev/null 2>&1
 else
-    $FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
+    $FWDIR/bin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
 fi
 
 # record the exit status lest it be overwritten:
diff --git a/spark-shell.cmd b/bin/spark-shell.cmd
similarity index 87%
rename from spark-shell.cmd
rename to bin/spark-shell.cmd
index 3e52bf835e5beac773336bf3ec160e3ad0a5d703..23973e3e3dd432c0e19dc48332ebbf0f6217a3ea 100644
--- a/spark-shell.cmd
+++ b/bin/spark-shell.cmd
@@ -17,6 +17,7 @@ rem See the License for the specific language governing permissions and
 rem limitations under the License.
 rem
 
-set FWDIR=%~dp0
+rem Find the path of sbin
+set SBIN=%~dp0..\sbin\
 
-cmd /V /E /C %FWDIR%spark-class2.cmd org.apache.spark.repl.Main %*
+cmd /V /E /C %SBIN%spark-class2.cmd org.apache.spark.repl.Main %*
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index 0a35ee7c79b12da7d4ab0f3036964dcd31764344..e2071e2ade8cd87fefdb60a8aebd57db177219c1 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -18,4 +18,4 @@
 # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g)
 # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT
 # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node
-
+# - SPARK_WORKER_DIR, to set the working directory of worker processes
diff --git a/core/pom.xml b/core/pom.xml
index 043f6cf68d15e3114c2640be66df2e60e393ebf5..9e5a450d57a477ec3b7020d518b5f461b72c2bdb 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -17,215 +17,229 @@
   -->
 
 <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>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent</artifactId>
-    <version>0.9.0-incubating-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent</artifactId>
+        <version>0.9.0-incubating-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
 
-  <groupId>org.apache.spark</groupId>
-  <artifactId>spark-core_2.10</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project Core</name>
-  <url>http://spark.incubator.apache.org/</url>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-core_2.10</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Core</name>
+    <url>http://spark.incubator.apache.org/</url>
 
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>net.java.dev.jets3t</groupId>
-      <artifactId>jets3t</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-ipc</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.ning</groupId>
-      <artifactId>compress-lzf</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.xerial.snappy</groupId>
-      <artifactId>snappy-java</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.ow2.asm</groupId>
-      <artifactId>asm</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>chill_${scala.binary.version}</artifactId>
-      <version>0.3.1</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>chill-java</artifactId>
-      <version>0.3.1</version>
-    </dependency>
-    <dependency>
-      <groupId>${akka.group}</groupId>
-      <artifactId>akka-remote_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>${akka.group}</groupId>
-      <artifactId>akka-slf4j_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.scala-lang</groupId>
-      <artifactId>scala-library</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>net.liftweb</groupId>
-      <artifactId>lift-json_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>it.unimi.dsi</groupId>
-      <artifactId>fastutil</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>colt</groupId>
-      <artifactId>colt</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.mesos</groupId>
-      <artifactId>mesos</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-jvm</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-json</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-ganglia</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-graphite</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.derby</groupId>
-      <artifactId>derby</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala.binary.version}</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.easymock</groupId>
-      <artifactId>easymock</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.novocode</groupId>
-      <artifactId>junit-interface</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-  <build>
-    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>test</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <exportAntProperties>true</exportAntProperties>
-              <tasks>
-                <property name="spark.classpath" refid="maven.test.classpath" />
-                <property environment="env" />
-                <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">
-                  <condition>
-                    <not>
-                      <or>
-                        <isset property="env.SCALA_HOME" />
-                        <isset property="env.SCALA_LIBRARY_PATH" />
-                      </or>
-                    </not>
-                  </condition>
-                </fail>
-              </tasks>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.scalatest</groupId>
-        <artifactId>scalatest-maven-plugin</artifactId>
-        <configuration>
-          <environmentVariables>
-            <SPARK_HOME>${basedir}/..</SPARK_HOME>
-            <SPARK_TESTING>1</SPARK_TESTING>
-            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
-          </environmentVariables>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>net.java.dev.jets3t</groupId>
+            <artifactId>jets3t</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro-ipc</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.ning</groupId>
+            <artifactId>compress-lzf</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.ow2.asm</groupId>
+            <artifactId>asm</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.twitter</groupId>
+            <artifactId>chill_${scala.binary.version}</artifactId>
+            <version>0.3.1</version>
+        </dependency>
+        <dependency>
+            <groupId>com.twitter</groupId>
+            <artifactId>chill-java</artifactId>
+            <version>0.3.1</version>
+        </dependency>
+        <dependency>
+            <groupId>${akka.group}</groupId>
+            <artifactId>akka-remote_${scala.binary.version}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>${akka.group}</groupId>
+            <artifactId>akka-slf4j_${scala.binary.version}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>${akka.group}</groupId>
+            <artifactId>akka-testkit_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>net.liftweb</groupId>
+            <artifactId>lift-json_${scala.binary.version}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>it.unimi.dsi</groupId>
+            <artifactId>fastutil</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>colt</groupId>
+            <artifactId>colt</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.mesos</groupId>
+            <artifactId>mesos</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.clearspring.analytics</groupId>
+            <artifactId>stream</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-jvm</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-json</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-ganglia</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-graphite</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.derby</groupId>
+            <artifactId>derby</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalatest</groupId>
+            <artifactId>scalatest_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.easymock</groupId>
+            <artifactId>easymock</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.novocode</groupId>
+            <artifactId>junit-interface</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                        <configuration>
+                            <exportAntProperties>true</exportAntProperties>
+                            <tasks>
+                                <property name="spark.classpath" refid="maven.test.classpath" />
+                                <property environment="env" />
+                                <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">
+                                    <condition>
+                                        <not>
+                                            <or>
+                                                <isset property="env.SCALA_HOME" />
+                                                <isset property="env.SCALA_LIBRARY_PATH" />
+                                            </or>
+                                        </not>
+                                    </condition>
+                                </fail>
+                            </tasks>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>
+                <configuration>
+                    <environmentVariables>
+                        <SPARK_HOME>${basedir}/..</SPARK_HOME>
+                        <SPARK_TESTING>1</SPARK_TESTING>
+                        <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
+                    </environmentVariables>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 </project>
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
index edd0fc56f861abbc0603975d0cd12e9944c9d447..d2d778b7567bcd925d299c6273cd58be4ddaf63f 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
@@ -20,28 +20,35 @@ package org.apache.spark.network.netty;
 import io.netty.bootstrap.Bootstrap;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
 import io.netty.channel.oio.OioEventLoopGroup;
 import io.netty.channel.socket.oio.OioSocketChannel;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.concurrent.TimeUnit;
+
 class FileClient {
 
-  private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());
-  private FileClientHandler handler = null;
+  private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName());
+
+  private final FileClientHandler handler;
   private Channel channel = null;
   private Bootstrap bootstrap = null;
-  private int connectTimeout = 60*1000; // 1 min
+  private EventLoopGroup group = null;
+  private final int connectTimeout;
+  private final int sendTimeout = 60; // 1 min
 
-  public FileClient(FileClientHandler handler, int connectTimeout) {
+  FileClient(FileClientHandler handler, int connectTimeout) {
     this.handler = handler;
     this.connectTimeout = connectTimeout;
   }
 
   public void init() {
+    group = new OioEventLoopGroup();
     bootstrap = new Bootstrap();
-    bootstrap.group(new OioEventLoopGroup())
+    bootstrap.group(group)
       .channel(OioSocketChannel.class)
       .option(ChannelOption.SO_KEEPALIVE, true)
       .option(ChannelOption.TCP_NODELAY, true)
@@ -56,6 +63,7 @@ class FileClient {
       // ChannelFuture cf = channel.closeFuture();
       //cf.addListener(new ChannelCloseListener(this));
     } catch (InterruptedException e) {
+      LOG.warn("FileClient interrupted while trying to connect", e);
       close();
     }
   }
@@ -71,16 +79,21 @@ class FileClient {
   public void sendRequest(String file) {
     //assert(file == null);
     //assert(channel == null);
-    channel.write(file + "\r\n");
+      try {
+          // Should be able to send the message to network link channel.
+          boolean bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS);
+          if (!bSent) {
+              throw new RuntimeException("Failed to send");
+          }
+      } catch (InterruptedException e) {
+          LOG.error("Error", e);
+      }
   }
 
   public void close() {
-    if(channel != null) {
-      channel.close();
-      channel = null;
-    }
-    if ( bootstrap!=null) {
-      bootstrap.shutdown();
+    if (group != null) {
+      group.shutdownGracefully();
+      group = null;
       bootstrap = null;
     }
   }
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
index 65ee15d63b8548b27275419a2c1507cf0e23847b..264cf97d0209fec170633168131c88390ac33513 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
@@ -17,17 +17,15 @@
 
 package org.apache.spark.network.netty;
 
-import io.netty.buffer.BufType;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.handler.codec.string.StringEncoder;
 
-
 class FileClientChannelInitializer extends ChannelInitializer<SocketChannel> {
 
-  private FileClientHandler fhandler;
+  private final FileClientHandler fhandler;
 
-  public FileClientChannelInitializer(FileClientHandler handler) {
+  FileClientChannelInitializer(FileClientHandler handler) {
     fhandler = handler;
   }
 
@@ -35,7 +33,7 @@ class FileClientChannelInitializer extends ChannelInitializer<SocketChannel> {
   public void initChannel(SocketChannel channel) {
     // file no more than 2G
     channel.pipeline()
-      .addLast("encoder", new StringEncoder(BufType.BYTE))
+      .addLast("encoder", new StringEncoder())
       .addLast("handler", fhandler);
   }
 }
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
index 8a09210245fff7eccfdd50fa551be2a9d53e9954..63d3d927255f9613f79a2d5894bb337ba83a1d54 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
@@ -19,11 +19,11 @@ package org.apache.spark.network.netty;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundByteHandlerAdapter;
+import io.netty.channel.SimpleChannelInboundHandler;
 
 import org.apache.spark.storage.BlockId;
 
-abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
+abstract class FileClientHandler extends SimpleChannelInboundHandler<ByteBuf> {
 
   private FileHeader currentHeader = null;
 
@@ -37,13 +37,7 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
   public abstract void handleError(BlockId blockId);
 
   @Override
-  public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) {
-    // Use direct buffer if possible.
-    return ctx.alloc().ioBuffer();
-  }
-
-  @Override
-  public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) {
+  public void channelRead0(ChannelHandlerContext ctx, ByteBuf in) {
     // get header
     if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) {
       currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE()));
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
index a99af348ce78207d156d51855038c1e63b5f3b24..c93425e2787dc339a788b42655147a7148230518 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
@@ -22,31 +22,33 @@ import java.net.InetSocketAddress;
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
 import io.netty.channel.oio.OioEventLoopGroup;
 import io.netty.channel.socket.oio.OioServerSocketChannel;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 /**
  * Server that accept the path of a file an echo back its content.
  */
 class FileServer {
 
-  private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());
+  private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName());
 
-  private ServerBootstrap bootstrap = null;
+  private EventLoopGroup bossGroup = null;
+  private EventLoopGroup workerGroup = null;
   private ChannelFuture channelFuture = null;
   private int port = 0;
-  private Thread blockingThread = null;
 
-  public FileServer(PathResolver pResolver, int port) {
+  FileServer(PathResolver pResolver, int port) {
     InetSocketAddress addr = new InetSocketAddress(port);
 
     // Configure the server.
-    bootstrap = new ServerBootstrap();
-    bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup())
+    bossGroup = new OioEventLoopGroup();
+    workerGroup = new OioEventLoopGroup();
+
+    ServerBootstrap bootstrap = new ServerBootstrap();
+    bootstrap.group(bossGroup, workerGroup)
         .channel(OioServerSocketChannel.class)
         .option(ChannelOption.SO_BACKLOG, 100)
         .option(ChannelOption.SO_RCVBUF, 1500)
@@ -67,7 +69,8 @@ class FileServer {
    * Start the file server asynchronously in a new thread.
    */
   public void start() {
-    blockingThread = new Thread() {
+    Thread blockingThread = new Thread() {
+      @Override
       public void run() {
         try {
           channelFuture.channel().closeFuture().sync();
@@ -89,13 +92,19 @@ class FileServer {
   public void stop() {
     // Close the bound channel.
     if (channelFuture != null) {
-      channelFuture.channel().close();
+      channelFuture.channel().close().awaitUninterruptibly();
       channelFuture = null;
     }
-    // Shutdown bootstrap.
-    if (bootstrap != null) {
-      bootstrap.shutdown();
-      bootstrap = null;
+
+    // Shutdown event groups
+    if (bossGroup != null) {
+       bossGroup.shutdownGracefully();
+       bossGroup = null;
+    }
+
+    if (workerGroup != null) {
+       workerGroup.shutdownGracefully();
+       workerGroup = null;
     }
     // TODO: Shutdown all accepted channels as well ?
   }
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
index 833af1632de9ca2b87d0b51f5732ae359c4e2d75..46efec8f8d96355ac470ed06fc68e5e32bb2d921 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
@@ -23,12 +23,11 @@ import io.netty.handler.codec.DelimiterBasedFrameDecoder;
 import io.netty.handler.codec.Delimiters;
 import io.netty.handler.codec.string.StringDecoder;
 
-
 class FileServerChannelInitializer extends ChannelInitializer<SocketChannel> {
 
-  PathResolver pResolver;
+  private final PathResolver pResolver;
 
-  public FileServerChannelInitializer(PathResolver pResolver) {
+  FileServerChannelInitializer(PathResolver pResolver) {
     this.pResolver = pResolver;
   }
 
@@ -36,7 +35,7 @@ class FileServerChannelInitializer extends ChannelInitializer<SocketChannel> {
   public void initChannel(SocketChannel channel) {
     channel.pipeline()
       .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter()))
-      .addLast("strDecoder", new StringDecoder())
+      .addLast("stringDecoder", new StringDecoder())
       .addLast("handler", new FileServerHandler(pResolver));
   }
 }
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
index 172c6e4b1cce198020a314b4f6fb3062b3dd5cff..3ac045f9444f281c041f3f3b2a999370490782ac 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
@@ -21,22 +21,26 @@ import java.io.File;
 import java.io.FileInputStream;
 
 import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundMessageHandlerAdapter;
+import io.netty.channel.SimpleChannelInboundHandler;
 import io.netty.channel.DefaultFileRegion;
 
 import org.apache.spark.storage.BlockId;
 import org.apache.spark.storage.FileSegment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
+class FileServerHandler extends SimpleChannelInboundHandler<String> {
 
-  PathResolver pResolver;
+  private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName());
 
-  public FileServerHandler(PathResolver pResolver){
+  private final PathResolver pResolver;
+
+  FileServerHandler(PathResolver pResolver){
     this.pResolver = pResolver;
   }
 
   @Override
-  public void messageReceived(ChannelHandlerContext ctx, String blockIdString) {
+  public void channelRead0(ChannelHandlerContext ctx, String blockIdString) {
     BlockId blockId = BlockId.apply(blockIdString);
     FileSegment fileSegment = pResolver.getBlockLocation(blockId);
     // if getBlockLocation returns null, close the channel
@@ -57,13 +61,13 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
         ctx.flush();
         return;
       }
-      int len = new Long(length).intValue();
+      int len = (int) length;
       ctx.write((new FileHeader(len, blockId)).buffer());
       try {
-        ctx.sendFile(new DefaultFileRegion(new FileInputStream(file)
+        ctx.write(new DefaultFileRegion(new FileInputStream(file)
           .getChannel(), fileSegment.offset(), fileSegment.length()));
       } catch (Exception e) {
-        e.printStackTrace();
+          LOG.error("Exception: ", e);
       }
     } else {
       ctx.write(new FileHeader(0, blockId).buffer());
@@ -73,7 +77,7 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
 
   @Override
   public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-    cause.printStackTrace();
+    LOG.error("Exception: ", cause);
     ctx.close();
   }
 }
diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
index 9f7ced44cf93da741b20d75e1b7f2f454272f8b4..7ad8d03efbadc78b31c2347609401dc01c599028 100755
--- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
+++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
@@ -1,26 +1,26 @@
-/*
- * 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.
- */
-
-package org.apache.spark.network.netty;
-
-import org.apache.spark.storage.BlockId;
-import org.apache.spark.storage.FileSegment;
-
-public interface PathResolver {
-  /** Get the file segment in which the given block resides. */
-  public FileSegment getBlockLocation(BlockId blockId);
-}
+/*
+ * 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.
+ */
+
+package org.apache.spark.network.netty;
+
+import org.apache.spark.storage.BlockId;
+import org.apache.spark.storage.FileSegment;
+
+public interface PathResolver {
+  /** Get the file segment in which the given block resides. */
+  FileSegment getBlockLocation(BlockId blockId);
+}
diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
new file mode 100644
index 0000000000000000000000000000000000000000..d72dbadc3904f327effddf99594045067be2f529
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
@@ -0,0 +1,8 @@
+# Set everything to be logged to the console
+log4j.rootCategory=INFO, console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala
index 6e922a612a0799afd3d36d2cb7721cd73056add1..5f73d234aa0506447f23d59ca0e91b68c3ab5fb7 100644
--- a/core/src/main/scala/org/apache/spark/Accumulators.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulators.scala
@@ -41,7 +41,7 @@ class Accumulable[R, T] (
     @transient initialValue: R,
     param: AccumulableParam[R, T])
   extends Serializable {
-  
+
   val id = Accumulators.newId
   @transient private var value_ = initialValue // Current value on master
   val zero = param.zero(initialValue)  // Zero value to be passed to workers
@@ -113,7 +113,7 @@ class Accumulable[R, T] (
   def setValue(newValue: R) {
     this.value = newValue
   }
- 
+
   // Called by Java when deserializing an object
   private def readObject(in: ObjectInputStream) {
     in.defaultReadObject()
@@ -177,7 +177,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
   def zero(initialValue: R): R = {
     // We need to clone initialValue, but it's hard to specify that R should also be Cloneable.
     // Instead we'll serialize it to a buffer and load it back.
-    val ser = new JavaSerializer().newInstance()
+    val ser = new JavaSerializer(new SparkConf(false)).newInstance()
     val copy = ser.deserialize[R](ser.serialize(initialValue))
     copy.clear()   // In case it contained stuff
     copy
@@ -215,7 +215,7 @@ private object Accumulators {
   val originals = Map[Long, Accumulable[_, _]]()
   val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]()
   var lastId: Long = 0
-  
+
   def newId: Long = synchronized {
     lastId += 1
     return lastId
diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala
index cdfc9dd54e06a82b01846066f686e70a858963a5..69a738dc4446ac23965a8cbd3c7823f014a1fba2 100644
--- a/core/src/main/scala/org/apache/spark/HttpServer.scala
+++ b/core/src/main/scala/org/apache/spark/HttpServer.scala
@@ -46,6 +46,7 @@ private[spark] class HttpServer(resourceBase: File) extends Logging {
     if (server != null) {
       throw new ServerStateException("Server is already started")
     } else {
+      logInfo("Starting HTTP Server")
       server = new Server()
       val connector = new SocketConnector
       connector.setMaxIdleTime(60*1000)
diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index 6a973ea4951c3a389a202f6c5dfd77f355061648..4a34989e50e57255a6520a2a8e0a852ddda6e2e2 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -17,8 +17,8 @@
 
 package org.apache.spark
 
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
+import org.apache.log4j.{LogManager, PropertyConfigurator}
+import org.slf4j.{Logger, LoggerFactory}
 
 /**
  * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
@@ -33,6 +33,7 @@ trait Logging {
   // Method to get or create the logger for this object
   protected def log: Logger = {
     if (log_ == null) {
+      initializeIfNecessary()
       var className = this.getClass.getName
       // Ignore trailing $'s in the class names for Scala objects
       if (className.endsWith("$")) {
@@ -89,7 +90,39 @@ trait Logging {
     log.isTraceEnabled
   }
 
-  // Method for ensuring that logging is initialized, to avoid having multiple
-  // threads do it concurrently (as SLF4J initialization is not thread safe).
-  protected def initLogging() { log }
+  private def initializeIfNecessary() {
+    if (!Logging.initialized) {
+      Logging.initLock.synchronized {
+        if (!Logging.initialized) {
+          initializeLogging()
+        }
+      }
+    }
+  }
+
+  private def initializeLogging() {
+    // If Log4j doesn't seem initialized, load a default properties file
+    val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
+    if (!log4jInitialized) {
+      val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
+      val classLoader = this.getClass.getClassLoader
+      Option(classLoader.getResource(defaultLogProps)) match {
+        case Some(url) => 
+          PropertyConfigurator.configure(url)
+          log.info(s"Using Spark's default log4j profile: $defaultLogProps")
+        case None => 
+          System.err.println(s"Spark was unable to load $defaultLogProps")
+      }
+    }
+    Logging.initialized = true
+
+    // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads
+    // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
+    log
+  }
+}
+
+object Logging {
+  @volatile private var initialized = false
+  val initLock = new Object()
 }
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index ccffcc356ca045952eca413201eb192550d0c31f..77b8ca1cce80b58b3d248fc81dc5d4e157b7cf09 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -50,12 +50,12 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster
   }
 }
 
-private[spark] class MapOutputTracker extends Logging {
+private[spark] class MapOutputTracker(conf: SparkConf) extends Logging {
 
-  private val timeout = AkkaUtils.askTimeout
+  private val timeout = AkkaUtils.askTimeout(conf)
 
   // Set to the MapOutputTrackerActor living on the driver
-  var trackerActor: Either[ActorRef, ActorSelection] = _
+  var trackerActor: ActorRef = _
 
   protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
 
@@ -65,23 +65,13 @@ private[spark] class MapOutputTracker extends Logging {
   protected val epochLock = new java.lang.Object
 
   private val metadataCleaner =
-    new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup)
+    new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf)
 
   // Send a message to the trackerActor and get its result within a default timeout, or
   // throw a SparkException if this fails.
   private def askTracker(message: Any): Any = {
     try {
-      /*
-        The difference between ActorRef and ActorSelection is well explained here:
-        http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#Use_actorSelection_instead_of_actorFor
-        In spark a map output tracker can be either started on Driver where it is created which
-        is an ActorRef or it can be on executor from where it is looked up which is an
-        actorSelection.
-       */
-      val future = trackerActor match {
-        case Left(a: ActorRef) => a.ask(message)(timeout)
-        case Right(b: ActorSelection) => b.ask(message)(timeout)
-      }
+      val future = trackerActor.ask(message)(timeout)
       Await.result(future, timeout)
     } catch {
       case e: Exception =>
@@ -129,7 +119,7 @@ private[spark] class MapOutputTracker extends Logging {
       if (fetchedStatuses == null) {
         // We won the race to fetch the output locs; do so
         logInfo("Doing the fetch; tracker actor = " + trackerActor)
-        val hostPort = Utils.localHostPort()
+        val hostPort = Utils.localHostPort(conf)
         // This try-finally prevents hangs due to timeouts:
         try {
           val fetchedBytes =
@@ -192,7 +182,8 @@ private[spark] class MapOutputTracker extends Logging {
   }
 }
 
-private[spark] class MapOutputTrackerMaster extends MapOutputTracker {
+private[spark] class MapOutputTrackerMaster(conf: SparkConf)
+  extends MapOutputTracker(conf) {
 
   // Cache a serialized version of the output statuses for each shuffle to send them out faster
   private var cacheEpoch = epoch
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index bcec41c439afb6c3634450049f4cfef164d67026..31b0773bfe06c6ca4cebc75e4a13430857940dd9 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -52,7 +52,7 @@ object Partitioner {
     for (r <- bySize if r.partitioner != None) {
       return r.partitioner.get
     }
-    if (System.getProperty("spark.default.parallelism") != null) {
+    if (rdd.context.conf.contains("spark.default.parallelism")) {
       return new HashPartitioner(rdd.context.defaultParallelism)
     } else {
       return new HashPartitioner(bySize.head.partitions.size)
@@ -90,7 +90,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
 class RangePartitioner[K <% Ordered[K]: ClassTag, V](
     partitions: Int,
     @transient rdd: RDD[_ <: Product2[K,V]],
-    private val ascending: Boolean = true) 
+    private val ascending: Boolean = true)
   extends Partitioner {
 
   // An array of upper bounds for the first (partitions - 1) partitions
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2de32231e87142c085fbab5de40bc9587d5fcfb2
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -0,0 +1,198 @@
+package org.apache.spark
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.HashMap
+
+import com.typesafe.config.ConfigFactory
+
+/**
+ * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs.
+ *
+ * Most of the time, you would create a SparkConf object with `new SparkConf()`, which will load
+ * values from both the `spark.*` Java system properties and any `spark.conf` on your application's
+ * classpath (if it has one). In this case, system properties take priority over `spark.conf`, and
+ * any parameters you set directly on the `SparkConf` object take priority over both of those.
+ *
+ * For unit tests, you can also call `new SparkConf(false)` to skip loading external settings and
+ * get the same configuration no matter what is on the classpath.
+ *
+ * All setter methods in this class support chaining. For example, you can write
+ * `new SparkConf().setMaster("local").setAppName("My app")`.
+ *
+ * Note that once a SparkConf object is passed to Spark, it is cloned and can no longer be modified
+ * by the user. Spark does not support modifying the configuration at runtime.
+ *
+ * @param loadDefaults whether to load values from the system properties and classpath
+ */
+class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with Logging {
+
+  /** Create a SparkConf that loads defaults from system properties and the classpath */
+  def this() = this(true)
+
+  private val settings = new HashMap[String, String]()
+
+  if (loadDefaults) {
+    ConfigFactory.invalidateCaches()
+    val typesafeConfig = ConfigFactory.systemProperties()
+      .withFallback(ConfigFactory.parseResources("spark.conf"))
+    for (e <- typesafeConfig.entrySet().asScala if e.getKey.startsWith("spark.")) {
+      settings(e.getKey) = e.getValue.unwrapped.toString
+    }
+  }
+
+  /** Set a configuration variable. */
+  def set(key: String, value: String): SparkConf = {
+    if (key == null) {
+      throw new NullPointerException("null key")
+    }
+    if (value == null) {
+      throw new NullPointerException("null value")
+    }
+    settings(key) = value
+    this
+  }
+
+  /**
+   * The master URL to connect to, such as "local" to run locally with one thread, "local[4]" to
+   * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster.
+   */
+  def setMaster(master: String): SparkConf = {
+    set("spark.master", master)
+  }
+
+  /** Set a name for your application. Shown in the Spark web UI. */
+  def setAppName(name: String): SparkConf = {
+    set("spark.app.name", name)
+  }
+
+  /** Set JAR files to distribute to the cluster. */
+  def setJars(jars: Seq[String]): SparkConf = {
+    for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor")
+    set("spark.jars", jars.filter(_ != null).mkString(","))
+  }
+
+  /** Set JAR files to distribute to the cluster. (Java-friendly version.) */
+  def setJars(jars: Array[String]): SparkConf = {
+    setJars(jars.toSeq)
+  }
+
+  /**
+   * Set an environment variable to be used when launching executors for this application.
+   * These variables are stored as properties of the form spark.executorEnv.VAR_NAME
+   * (for example spark.executorEnv.PATH) but this method makes them easier to set.
+   */
+  def setExecutorEnv(variable: String, value: String): SparkConf = {
+    set("spark.executorEnv." + variable, value)
+  }
+
+  /**
+   * Set multiple environment variables to be used when launching executors.
+   * These variables are stored as properties of the form spark.executorEnv.VAR_NAME
+   * (for example spark.executorEnv.PATH) but this method makes them easier to set.
+   */
+  def setExecutorEnv(variables: Seq[(String, String)]): SparkConf = {
+    for ((k, v) <- variables) {
+      setExecutorEnv(k, v)
+    }
+    this
+  }
+
+  /**
+   * Set multiple environment variables to be used when launching executors.
+   * (Java-friendly version.)
+   */
+  def setExecutorEnv(variables: Array[(String, String)]): SparkConf = {
+    setExecutorEnv(variables.toSeq)
+  }
+
+  /**
+   * Set the location where Spark is installed on worker nodes.
+   */
+  def setSparkHome(home: String): SparkConf = {
+    set("spark.home", home)
+  }
+
+  /** Set multiple parameters together */
+  def setAll(settings: Traversable[(String, String)]) = {
+    this.settings ++= settings
+    this
+  }
+
+  /** Set a parameter if it isn't already configured */
+  def setIfMissing(key: String, value: String): SparkConf = {
+    if (!settings.contains(key)) {
+      settings(key) = value
+    }
+    this
+  }
+
+  /** Remove a parameter from the configuration */
+  def remove(key: String): SparkConf = {
+    settings.remove(key)
+    this
+  }
+
+  /** Get a parameter; throws a NoSuchElementException if it's not set */
+  def get(key: String): String = {
+    settings.getOrElse(key, throw new NoSuchElementException(key))
+  }
+
+  /** Get a parameter, falling back to a default if not set */
+  def get(key: String, defaultValue: String): String = {
+    settings.getOrElse(key, defaultValue)
+  }
+
+  /** Get a parameter as an Option */
+  def getOption(key: String): Option[String] = {
+    settings.get(key)
+  }
+
+  /** Get all parameters as a list of pairs */
+  def getAll: Array[(String, String)] = settings.clone().toArray
+
+  /** Get a parameter as an integer, falling back to a default if not set */
+  def getInt(key: String, defaultValue: Int): Int = {
+    getOption(key).map(_.toInt).getOrElse(defaultValue)
+  }
+
+  /** Get a parameter as a long, falling back to a default if not set */
+  def getLong(key: String, defaultValue: Long): Long = {
+    getOption(key).map(_.toLong).getOrElse(defaultValue)
+  }
+
+  /** Get a parameter as a double, falling back to a default if not set */
+  def getDouble(key: String, defaultValue: Double): Double = {
+    getOption(key).map(_.toDouble).getOrElse(defaultValue)
+  }
+
+  /** Get a parameter as a boolean, falling back to a default if not set */
+  def getBoolean(key: String, defaultValue: Boolean): Boolean = {
+    getOption(key).map(_.toBoolean).getOrElse(defaultValue)
+  }
+
+  /** Get all executor environment variables set on this SparkConf */
+  def getExecutorEnv: Seq[(String, String)] = {
+    val prefix = "spark.executorEnv."
+    getAll.filter{case (k, v) => k.startsWith(prefix)}
+          .map{case (k, v) => (k.substring(prefix.length), v)}
+  }
+
+  /** Get all akka conf variables set on this SparkConf */
+  def getAkkaConf: Seq[(String, String)] =  getAll.filter {case (k, v) => k.startsWith("akka.")}
+
+  /** Does the configuration contain a given parameter? */
+  def contains(key: String): Boolean = settings.contains(key)
+
+  /** Copy this object */
+  override def clone: SparkConf = {
+    new SparkConf(false).setAll(settings)
+  }
+
+  /**
+   * Return a string listing all keys and values, one per line. This is useful to print the
+   * configuration out for debugging.
+   */
+  def toDebugString: String = {
+    settings.toArray.sorted.map{case (k, v) => k + "=" + v}.mkString("\n")
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 304e85f1c035455527cf1cb2b50520b7b06dbb29..101a7e6db9df7877f5cb1e1156890f3fb3b250eb 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -22,33 +22,20 @@ import java.net.URI
 import java.util.{UUID, Properties}
 import java.util.concurrent.atomic.AtomicInteger
 
-import scala.collection.Map
+import scala.collection.{Map, Set}
 import scala.collection.generic.Growable
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
+
+import scala.collection.mutable.{ArrayBuffer, HashMap}
 import scala.reflect.{ClassTag, classTag}
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{LocalFileSystem, Path}
-import org.apache.hadoop.io.ArrayWritable
-import org.apache.hadoop.io.BooleanWritable
-import org.apache.hadoop.io.BytesWritable
-import org.apache.hadoop.io.DoubleWritable
-import org.apache.hadoop.io.FloatWritable
-import org.apache.hadoop.io.IntWritable
-import org.apache.hadoop.io.LongWritable
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.io.Writable
-import org.apache.hadoop.mapred.FileInputFormat
-import org.apache.hadoop.mapred.InputFormat
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapred.SequenceFileInputFormat
-import org.apache.hadoop.mapred.TextInputFormat
-import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable,
+FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable}
+import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat,
+TextInputFormat}
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob}
 import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
-
 import org.apache.mesos.MesosNativeLibrary
 
 import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
@@ -61,53 +48,101 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me
 import org.apache.spark.scheduler.local.LocalBackend
 import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
 import org.apache.spark.ui.SparkUI
-import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType,
-  TimeStampedHashMap, Utils}
+import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType,
+ClosureCleaner}
 
 /**
  * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
  * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
  *
- * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param appName A name for your application, to display on the cluster web UI.
- * @param sparkHome Location where Spark is installed on cluster nodes.
- * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
- *             system or HDFS, HTTP, HTTPS, or FTP URLs.
- * @param environment Environment variables to set on worker nodes.
+ * @param config a Spark Config object describing the application configuration. Any settings in
+ *   this config overrides the default configs as well as system properties.
+ * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can
+ *   be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
+ *   from a list of input files or InputFormats for the application.
  */
 class SparkContext(
-    val master: String,
-    val appName: String,
-    val sparkHome: String = null,
-    val jars: Seq[String] = Nil,
-    val environment: Map[String, String] = Map(),
+    config: SparkConf,
     // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc)
-    // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set
-    // of data-local splits on host
-    val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
-      scala.collection.immutable.Map())
+    // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains
+    // a map from hostname to a list of input format splits on the host.
+    val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map())
   extends Logging {
 
-  // Ensure logging is initialized before we spawn any threads
-  initLogging()
+  /**
+   * Alternative constructor that allows setting common Spark properties directly
+   *
+   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+   * @param appName A name for your application, to display on the cluster web UI
+   * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
+   */
+  def this(master: String, appName: String, conf: SparkConf) =
+    this(SparkContext.updatedConf(conf, master, appName))
 
-  // Set Spark driver host and port system properties
-  if (System.getProperty("spark.driver.host") == null) {
-    System.setProperty("spark.driver.host", Utils.localHostName())
+  /**
+   * Alternative constructor that allows setting common Spark properties directly
+   *
+   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+   * @param appName A name for your application, to display on the cluster web UI.
+   * @param sparkHome Location where Spark is installed on cluster nodes.
+   * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
+   *             system or HDFS, HTTP, HTTPS, or FTP URLs.
+   * @param environment Environment variables to set on worker nodes.
+   */
+  def this(
+      master: String,
+      appName: String,
+      sparkHome: String = null,
+      jars: Seq[String] = Nil,
+      environment: Map[String, String] = Map(),
+      preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) =
+  {
+    this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment),
+      preferredNodeLocationData)
   }
-  if (System.getProperty("spark.driver.port") == null) {
-    System.setProperty("spark.driver.port", "0")
+
+  private[spark] val conf = config.clone()
+
+  /**
+   * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be
+   * changed at runtime.
+   */
+  def getConf: SparkConf = conf.clone()
+
+  if (!conf.contains("spark.master")) {
+    throw new SparkException("A master URL must be set in your configuration")
+  }
+  if (!conf.contains("spark.app.name")) {
+    throw new SparkException("An application must be set in your configuration")
+  }
+
+  if (conf.get("spark.logConf", "false").toBoolean) {
+    logInfo("Spark configuration:\n" + conf.toDebugString)
+  }
+
+  // Set Spark driver host and port system properties
+  conf.setIfMissing("spark.driver.host", Utils.localHostName())
+  conf.setIfMissing("spark.driver.port", "0")
+
+  val jars: Seq[String] = if (conf.contains("spark.jars")) {
+    conf.get("spark.jars").split(",").filter(_.size != 0)
+  } else {
+    null
   }
 
+  val master = conf.get("spark.master")
+  val appName = conf.get("spark.app.name")
+
   val isLocal = (master == "local" || master.startsWith("local["))
 
   // Create the Spark execution environment (cache, map output tracker, etc)
-  private[spark] val env = SparkEnv.createFromSystemProperties(
+  private[spark] val env = SparkEnv.create(
+    conf,
     "<driver>",
-    System.getProperty("spark.driver.host"),
-    System.getProperty("spark.driver.port").toInt,
-    true,
-    isLocal)
+    conf.get("spark.driver.host"),
+    conf.get("spark.driver.port").toInt,
+    isDriver = true,
+    isLocal = isLocal)
   SparkEnv.set(env)
 
   // Used to store a URL for each static file/jar together with the file's local timestamp
@@ -116,7 +151,8 @@ class SparkContext(
 
   // Keeps track of all persisted RDDs
   private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
-  private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup)
+  private[spark] val metadataCleaner =
+    new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf)
 
   // Initialize the Spark UI
   private[spark] val ui = new SparkUI(this)
@@ -126,23 +162,30 @@ class SparkContext(
 
   // Add each JAR given through the constructor
   if (jars != null) {
-    jars.foreach { addJar(_) }
+    jars.foreach(addJar)
   }
 
+  private[spark] val executorMemory = conf.getOption("spark.executor.memory")
+    .orElse(Option(System.getenv("SPARK_MEM")))
+    .map(Utils.memoryStringToMb)
+    .getOrElse(512)
+
   // Environment variables to pass to our executors
   private[spark] val executorEnvs = HashMap[String, String]()
   // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
-  for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) {
-    val value = System.getenv(key)
-    if (value != null) {
-      executorEnvs(key) = value
-    }
+  for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS");
+      value <- Option(System.getenv(key))) {
+    executorEnvs(key) = value
   }
-  // Since memory can be set with a system property too, use that
-  executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m"
-  if (environment != null) {
-    executorEnvs ++= environment
+  // Convert java options to env vars as a work around
+  // since we can't set env vars directly in sbt.
+  for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing"))
+    value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} {
+    executorEnvs(envKey) = value
   }
+  // Since memory can be set with a system property too, use that
+  executorEnvs("SPARK_MEM") = executorMemory + "m"
+  executorEnvs ++= conf.getExecutorEnv
 
   // Set SPARK_USER for user who is running SparkContext.
   val sparkUser = Option {
@@ -164,24 +207,24 @@ class SparkContext(
   /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
   val hadoopConfiguration = {
     val env = SparkEnv.get
-    val conf = SparkHadoopUtil.get.newConfiguration()
+    val hadoopConf = SparkHadoopUtil.get.newConfiguration()
     // Explicitly check for S3 environment variables
     if (System.getenv("AWS_ACCESS_KEY_ID") != null &&
         System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
-      conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
-      conf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
-      conf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
-      conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
+      hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
+      hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
+      hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
+      hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
     }
     // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar"
-    Utils.getSystemProperties.foreach { case (key, value) =>
+    conf.getAll.foreach { case (key, value) =>
       if (key.startsWith("spark.hadoop.")) {
-        conf.set(key.substring("spark.hadoop.".length), value)
+        hadoopConf.set(key.substring("spark.hadoop.".length), value)
       }
     }
-    val bufferSize = System.getProperty("spark.buffer.size", "65536")
-    conf.set("io.file.buffer.size", bufferSize)
-    conf
+    val bufferSize = conf.get("spark.buffer.size", "65536")
+    hadoopConf.set("io.file.buffer.size", bufferSize)
+    hadoopConf
   }
 
   private[spark] var checkpointDir: Option[String] = None
@@ -191,7 +234,7 @@ class SparkContext(
     override protected def childValue(parent: Properties): Properties = new Properties(parent)
   }
 
-  private[spark] def getLocalProperties(): Properties = localProperties.get()
+  private[spark] def getLocalProperties: Properties = localProperties.get()
 
   private[spark] def setLocalProperties(props: Properties) {
     localProperties.set(props)
@@ -522,7 +565,7 @@ class SparkContext(
     addedFiles(key) = System.currentTimeMillis
 
     // Fetch the file locally in case a job is executed using DAGScheduler.runLocally().
-    Utils.fetchFile(path, new File(SparkFiles.getRootDirectory))
+    Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf)
 
     logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
   }
@@ -692,15 +735,27 @@ class SparkContext(
    * (in that order of preference). If neither of these is set, return None.
    */
   private[spark] def getSparkHome(): Option[String] = {
-    if (sparkHome != null) {
-      Some(sparkHome)
-    } else if (System.getProperty("spark.home") != null) {
-      Some(System.getProperty("spark.home"))
-    } else if (System.getenv("SPARK_HOME") != null) {
-      Some(System.getenv("SPARK_HOME"))
-    } else {
-      None
-    }
+    conf.getOption("spark.home").orElse(Option(System.getenv("SPARK_HOME")))
+  }
+
+  /**
+   * Support function for API backtraces.
+   */
+  def setCallSite(site: String) {
+    setLocalProperty("externalCallSite", site)
+  }
+
+  /**
+   * Support function for API backtraces.
+   */
+  def clearCallSite() {
+    setLocalProperty("externalCallSite", null)
+  }
+
+  private[spark] def getCallSite(): String = {
+    val callSite = getLocalProperty("externalCallSite")
+    if (callSite == null) return Utils.formatSparkCallSite
+    callSite
   }
 
   /**
@@ -715,7 +770,7 @@ class SparkContext(
       partitions: Seq[Int],
       allowLocal: Boolean,
       resultHandler: (Int, U) => Unit) {
-    val callSite = Utils.formatSparkCallSite
+    val callSite = getCallSite
     val cleanedFunc = clean(func)
     logInfo("Starting job: " + callSite)
     val start = System.nanoTime
@@ -799,7 +854,7 @@ class SparkContext(
       func: (TaskContext, Iterator[T]) => U,
       evaluator: ApproximateEvaluator[U, R],
       timeout: Long): PartialResult[R] = {
-    val callSite = Utils.formatSparkCallSite
+    val callSite = getCallSite
     logInfo("Starting job: " + callSite)
     val start = System.nanoTime
     val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout,
@@ -819,7 +874,7 @@ class SparkContext(
       resultFunc: => R): SimpleFutureAction[R] =
   {
     val cleanF = clean(processPartition)
-    val callSite = Utils.formatSparkCallSite
+    val callSite = getCallSite
     val waiter = dagScheduler.submitJob(
       rdd,
       (context: TaskContext, iter: Iterator[T]) => cleanF(iter),
@@ -987,7 +1042,7 @@ object SparkContext {
 
   /**
    * Find the JAR from which a given class was loaded, to make it easy for users to pass
-   * their JARs to SparkContext
+   * their JARs to SparkContext.
    */
   def jarOfClass(cls: Class[_]): Seq[String] = {
     val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class")
@@ -1004,21 +1059,44 @@ object SparkContext {
     }
   }
 
-  /** Find the JAR that contains the class of a particular object */
+  /**
+   * Find the JAR that contains the class of a particular object, to make it easy for users
+   * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in
+   * your driver program.
+   */
   def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass)
 
-  /** Get the amount of memory per executor requested through system properties or SPARK_MEM */
-  private[spark] val executorMemoryRequested = {
-    // TODO: Might need to add some extra memory for the non-heap parts of the JVM
-    Option(System.getProperty("spark.executor.memory"))
-      .orElse(Option(System.getenv("SPARK_MEM")))
-      .map(Utils.memoryStringToMb)
-      .getOrElse(512)
+  /**
+   * Creates a modified version of a SparkConf with the parameters that can be passed separately
+   * to SparkContext, to make it easier to write SparkContext's constructors. This ignores
+   * parameters that are passed as the default value of null, instead of throwing an exception
+   * like SparkConf would.
+   */
+  private[spark] def updatedConf(
+      conf: SparkConf,
+      master: String,
+      appName: String,
+      sparkHome: String = null,
+      jars: Seq[String] = Nil,
+      environment: Map[String, String] = Map()): SparkConf =
+  {
+    val res = conf.clone()
+    res.setMaster(master)
+    res.setAppName(appName)
+    if (sparkHome != null) {
+      res.setSparkHome(sparkHome)
+    }
+    if (!jars.isEmpty) {
+      res.setJars(jars)
+    }
+    res.setExecutorEnv(environment.toSeq)
+    res
   }
 
-  // Creates a task scheduler based on a given master URL. Extracted for testing.
-  private
-  def createTaskScheduler(sc: SparkContext, master: String, appName: String): TaskScheduler = {
+  /** Creates a task scheduler based on a given master URL. Extracted for testing. */
+  private def createTaskScheduler(sc: SparkContext, master: String, appName: String)
+      : TaskScheduler =
+  {
     // Regular expression used for local[N] master format
     val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r
     // Regular expression for local[N, maxRetries], used in tests with failing tasks
@@ -1064,10 +1142,10 @@ object SparkContext {
       case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
         // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
         val memoryPerSlaveInt = memoryPerSlave.toInt
-        if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) {
+        if (sc.executorMemory > memoryPerSlaveInt) {
           throw new SparkException(
             "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
-              memoryPerSlaveInt, SparkContext.executorMemoryRequested))
+              memoryPerSlaveInt, sc.executorMemory))
         }
 
         val scheduler = new TaskSchedulerImpl(sc)
@@ -1125,7 +1203,7 @@ object SparkContext {
       case mesosUrl @ MESOS_REGEX(_) =>
         MesosNativeLibrary.load()
         val scheduler = new TaskSchedulerImpl(sc)
-        val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean
+        val coarseGrained = sc.conf.get("spark.mesos.coarse", "false").toBoolean
         val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs
         val backend = if (coarseGrained) {
           new CoarseMesosSchedulerBackend(scheduler, sc, url, appName)
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 826f5c2d8c6bde5ec81905906931556e216da1b6..2e36ccb9a0f076b92c59ac00793b91237320f5a2 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -17,11 +17,10 @@
 
 package org.apache.spark
 
-import collection.mutable
-import serializer.Serializer
+import scala.collection.mutable
+import scala.concurrent.Await
 
 import akka.actor._
-import akka.remote.RemoteActorRefProvider
 
 import org.apache.spark.broadcast.BroadcastManager
 import org.apache.spark.metrics.MetricsSystem
@@ -40,7 +39,7 @@ import com.google.common.collect.MapMaker
  * objects needs to have the right SparkEnv set. You can get the current environment with
  * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set.
  */
-class SparkEnv (
+class SparkEnv private[spark] (
     val executorId: String,
     val actorSystem: ActorSystem,
     val serializerManager: SerializerManager,
@@ -54,7 +53,8 @@ class SparkEnv (
     val connectionManager: ConnectionManager,
     val httpFileServer: HttpFileServer,
     val sparkFilesDir: String,
-    val metricsSystem: MetricsSystem) {
+    val metricsSystem: MetricsSystem,
+    val conf: SparkConf) {
 
   private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()
 
@@ -62,7 +62,7 @@ class SparkEnv (
   // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
   private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]()
 
-  def stop() {
+  private[spark] def stop() {
     pythonWorkers.foreach { case(key, worker) => worker.stop() }
     httpFileServer.stop()
     mapOutputTracker.stop()
@@ -78,6 +78,7 @@ class SparkEnv (
     //actorSystem.awaitTermination()
   }
 
+  private[spark]
   def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = {
     synchronized {
       val key = (pythonExec, envVars)
@@ -106,33 +107,35 @@ object SparkEnv extends Logging {
   /**
    * Returns the ThreadLocal SparkEnv.
    */
-  def getThreadLocal : SparkEnv = {
+  def getThreadLocal: SparkEnv = {
 	  env.get()
   }
 
-  def createFromSystemProperties(
+  private[spark] def create(
+      conf: SparkConf,
       executorId: String,
       hostname: String,
       port: Int,
       isDriver: Boolean,
       isLocal: Boolean): SparkEnv = {
 
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port)
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port,
+      conf = conf)
 
     // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port),
     // figure out which port number Akka actually bound to and set spark.driver.port to it.
     if (isDriver && port == 0) {
-      System.setProperty("spark.driver.port", boundPort.toString)
+      conf.set("spark.driver.port",  boundPort.toString)
     }
 
     // set only if unset until now.
-    if (System.getProperty("spark.hostPort", null) == null) {
+    if (!conf.contains("spark.hostPort")) {
       if (!isDriver){
         // unexpected
         Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set")
       }
       Utils.checkHost(hostname)
-      System.setProperty("spark.hostPort", hostname + ":" + boundPort)
+      conf.set("spark.hostPort",  hostname + ":" + boundPort)
     }
 
     val classLoader = Thread.currentThread.getContextClassLoader
@@ -140,49 +143,51 @@ object SparkEnv extends Logging {
     // Create an instance of the class named by the given Java system property, or by
     // defaultClassName if the property is not set, and return it as a T
     def instantiateClass[T](propertyName: String, defaultClassName: String): T = {
-      val name = System.getProperty(propertyName, defaultClassName)
+      val name = conf.get(propertyName,  defaultClassName)
       Class.forName(name, true, classLoader).newInstance().asInstanceOf[T]
     }
 
     val serializerManager = new SerializerManager
 
     val serializer = serializerManager.setDefault(
-      System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer"))
+      conf.get("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf)
 
     val closureSerializer = serializerManager.get(
-      System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"))
+      conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"),
+      conf)
 
-    def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = {
+    def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
       if (isDriver) {
         logInfo("Registering " + name)
-        Left(actorSystem.actorOf(Props(newActor), name = name))
+        actorSystem.actorOf(Props(newActor), name = name)
       } else {
-        val driverHost: String = System.getProperty("spark.driver.host", "localhost")
-        val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt
+        val driverHost: String = conf.get("spark.driver.host", "localhost")
+        val driverPort: Int = conf.get("spark.driver.port", "7077").toInt
         Utils.checkHost(driverHost, "Expected hostname")
-        val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name)
-        logInfo("Connecting to " + name + ": " + url)
-        Right(actorSystem.actorSelection(url))
+        val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name"
+        val timeout = AkkaUtils.lookupTimeout(conf)
+        logInfo(s"Connecting to $name: $url")
+        Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout)
       }
     }
 
     val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
       "BlockManagerMaster",
-      new BlockManagerMasterActor(isLocal)))
-    val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer)
+      new BlockManagerMasterActor(isLocal, conf)), conf)
+    val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf)
 
     val connectionManager = blockManager.connectionManager
 
-    val broadcastManager = new BroadcastManager(isDriver)
+    val broadcastManager = new BroadcastManager(isDriver, conf)
 
     val cacheManager = new CacheManager(blockManager)
 
     // Have to assign trackerActor after initialization as MapOutputTrackerActor
     // requires the MapOutputTracker itself
     val mapOutputTracker =  if (isDriver) {
-      new MapOutputTrackerMaster()
+      new MapOutputTrackerMaster(conf)
     } else {
-      new MapOutputTracker()
+      new MapOutputTracker(conf)
     }
     mapOutputTracker.trackerActor = registerOrLookup(
       "MapOutputTracker",
@@ -193,12 +198,12 @@ object SparkEnv extends Logging {
 
     val httpFileServer = new HttpFileServer()
     httpFileServer.initialize()
-    System.setProperty("spark.fileserver.uri", httpFileServer.serverUri)
+    conf.set("spark.fileserver.uri",  httpFileServer.serverUri)
 
     val metricsSystem = if (isDriver) {
-      MetricsSystem.createMetricsSystem("driver")
+      MetricsSystem.createMetricsSystem("driver", conf)
     } else {
-      MetricsSystem.createMetricsSystem("executor")
+      MetricsSystem.createMetricsSystem("executor", conf)
     }
     metricsSystem.start()
 
@@ -212,7 +217,7 @@ object SparkEnv extends Logging {
     }
 
     // Warn about deprecated spark.cache.class property
-    if (System.getProperty("spark.cache.class") != null) {
+    if (conf.contains("spark.cache.class")) {
       logWarning("The spark.cache.class property is no longer being used! Specify storage " +
         "levels using the RDD.persist() method instead.")
     }
@@ -231,6 +236,7 @@ object SparkEnv extends Logging {
       connectionManager,
       httpFileServer,
       sparkFilesDir,
-      metricsSystem)
+      metricsSystem,
+      conf)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index c1e5e04b31e6035193828c06a9b664f9baf87c65..faf6dcd6186237b5d58a4612d78dc81bda2b54da 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -53,5 +53,3 @@ private[spark] case class ExceptionFailure(
 private[spark] case object TaskResultLost extends TaskEndReason
 
 private[spark] case object TaskKilled extends TaskEndReason
-
-private[spark] case class OtherFailure(message: String) extends TaskEndReason
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index 363667fa863534ee9ad26105595f30d41a364d02..55c87450ac65ae5256cba054cf4ce7684773c592 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -611,6 +611,42 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K
    * Return an RDD with the values of each tuple.
    */
   def values(): JavaRDD[V] = JavaRDD.fromRDD[V](rdd.map(_._2))
+
+  /**
+   * Return approximate number of distinct values for each key in this RDD.
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. Uses the provided
+   * Partitioner to partition the output RDD.
+   */
+  def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaRDD[(K, Long)] = {
+    rdd.countApproxDistinctByKey(relativeSD, partitioner)
+  }
+
+  /**
+   * Return approximate number of distinct values for each key this RDD.
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. The default value of
+   * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism
+   * level.
+   */
+  def countApproxDistinctByKey(relativeSD: Double = 0.05): JavaRDD[(K, Long)] = {
+    rdd.countApproxDistinctByKey(relativeSD)
+  }
+
+
+  /**
+   * Return approximate number of distinct values for each key in this RDD.
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. HashPartitions the
+   * output RDD into numPartitions.
+   *
+   */
+  def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = {
+    rdd.countApproxDistinctByKey(relativeSD, numPartitions)
+  }
 }
 
 object JavaPairRDD {
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index f344804b4cf6881b2091b72b07158bf04e4b4f2a..924d8af0602f4a7fa8e33b637ebd5af23059ad81 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -444,4 +444,15 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
     val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]]
     takeOrdered(num, comp)
   }
+
+  /**
+   * Return approximate number of distinct elements in the RDD.
+   *
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. The default value of
+   * relativeSD is 0.05.
+   */
+  def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD)
+
 }
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 50f2021d0194da58de8fccdeb7a45df15a3be049..e93b10fd7eecb71b59908a54a765d0e23b7da7b1 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -29,17 +29,22 @@ import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 import com.google.common.base.Optional
 
-import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, SparkContext}
+import org.apache.spark._
 import org.apache.spark.SparkContext.IntAccumulatorParam
 import org.apache.spark.SparkContext.DoubleAccumulatorParam
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.rdd.RDD
+import scala.Tuple2
 
 /**
  * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and
  * works with Java collections instead of Scala ones.
  */
 class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
+  /**
+   * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters
+   */
+  def this(conf: SparkConf) = this(new SparkContext(conf))
 
   /**
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
@@ -47,6 +52,14 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
    */
   def this(master: String, appName: String) = this(new SparkContext(master, appName))
 
+  /**
+   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+   * @param appName A name for your application, to display on the cluster web UI
+   * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
+   */
+  def this(master: String, appName: String, conf: SparkConf) =
+    this(conf.setMaster(master).setAppName(appName))
+
   /**
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
    * @param appName A name for your application, to display on the cluster web UI
@@ -392,10 +405,36 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
     new JavaRDD(sc.checkpointFile(path))
   }
+
+  /**
+   * Return a copy of this JavaSparkContext's configuration. The configuration ''cannot'' be
+   * changed at runtime.
+   */
+  def getConf: SparkConf = sc.getConf
+
+  /**
+   * Pass-through to SparkContext.setCallSite.  For API support only.
+   */
+  def setCallSite(site: String) {
+    sc.setCallSite(site)
+  }
+
+  /**
+   * Pass-through to SparkContext.setCallSite.  For API support only.
+   */
+  def clearCallSite() {
+    sc.clearCallSite()
+  }
 }
 
 object JavaSparkContext {
   implicit def fromSparkContext(sc: SparkContext): JavaSparkContext = new JavaSparkContext(sc)
 
   implicit def toSparkContext(jsc: JavaSparkContext): SparkContext = jsc.sc
+
+  /**
+   * Find the JAR from which a given class was loaded, to make it easy for users to pass
+   * their JARs to SparkContext.
+   */
+  def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray
 }
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index ca42c769286e6532b3c8e74c09d4995656fdba1e..32cc70e8c9dda4d98e44b8d5165ad5375507e460 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag](
     accumulator: Accumulator[JList[Array[Byte]]])
   extends RDD[Array[Byte]](parent) {
 
-  val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+  val bufferSize = conf.get("spark.buffer.size", "65536").toInt
 
   override def getPartitions = parent.partitions
 
@@ -250,7 +250,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort:
 
   Utils.checkHost(serverHost, "Expected hostname")
 
-  val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+  val bufferSize = SparkEnv.get.conf.get("spark.buffer.size", "65536").toInt
 
   override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
 
diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
index 43c18294c552b69a77e404ca3511634b60a94612..0fc478a41967c720f23da4f2d8a44d3ed26abd87 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -31,8 +31,8 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
   override def toString = "Broadcast(" + id + ")"
 }
 
-private[spark] 
-class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable {
+private[spark]
+class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable {
 
   private var initialized = false
   private var broadcastFactory: BroadcastFactory = null
@@ -43,14 +43,14 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable
   private def initialize() {
     synchronized {
       if (!initialized) {
-        val broadcastFactoryClass = System.getProperty(
+        val broadcastFactoryClass = conf.get(
           "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
 
         broadcastFactory =
           Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
 
         // Initialize appropriate BroadcastFactory and BroadcastObject
-        broadcastFactory.initialize(isDriver)
+        broadcastFactory.initialize(isDriver, conf)
 
         initialized = true
       }
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
index 68bff75b908c73fe11dea8a46d5ccff51258a6a3..fb161ce69d40b160a81d4ad1e389d029a16b923a 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.broadcast
 
+import org.apache.spark.SparkConf
+
 /**
  * An interface for all the broadcast implementations in Spark (to allow 
  * multiple broadcast implementations). SparkContext uses a user-specified
@@ -24,7 +26,7 @@ package org.apache.spark.broadcast
  * entire Spark job.
  */
 private[spark] trait BroadcastFactory {
-  def initialize(isDriver: Boolean): Unit
+  def initialize(isDriver: Boolean, conf: SparkConf): Unit
   def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T]
   def stop(): Unit
 }
diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
index 47db720416447700dc34fabcfc272dc921e73675..db596d5fcc05413debab51d1ca9dd8dca0321090 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -24,14 +24,14 @@ import java.util.concurrent.TimeUnit
 import it.unimi.dsi.fastutil.io.FastBufferedInputStream
 import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
 
-import org.apache.spark.{HttpServer, Logging, SparkEnv}
+import org.apache.spark.{SparkConf, HttpServer, Logging, SparkEnv}
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
 import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils}
 
 private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
   extends Broadcast[T](id) with Logging with Serializable {
-  
+
   def value = value_
 
   def blockId = BroadcastBlockId(id)
@@ -40,7 +40,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea
     SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
   }
 
-  if (!isLocal) { 
+  if (!isLocal) {
     HttpBroadcast.write(id, value_)
   }
 
@@ -64,7 +64,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea
 }
 
 private[spark] class HttpBroadcastFactory extends BroadcastFactory {
-  def initialize(isDriver: Boolean) { HttpBroadcast.initialize(isDriver) }
+  def initialize(isDriver: Boolean, conf: SparkConf) { HttpBroadcast.initialize(isDriver, conf) }
 
   def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
     new HttpBroadcast[T](value_, isLocal, id)
@@ -81,44 +81,51 @@ private object HttpBroadcast extends Logging {
   private var serverUri: String = null
   private var server: HttpServer = null
 
+  // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist
   private val files = new TimeStampedHashSet[String]
-  private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup)
+  private var cleaner: MetadataCleaner = null
 
-  private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5,TimeUnit.MINUTES).toInt
+  private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt
 
-  private lazy val compressionCodec = CompressionCodec.createCodec()
+  private var compressionCodec: CompressionCodec = null
 
-  def initialize(isDriver: Boolean) {
+  def initialize(isDriver: Boolean, conf: SparkConf) {
     synchronized {
       if (!initialized) {
-        bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
-        compress = System.getProperty("spark.broadcast.compress", "true").toBoolean
+        bufferSize = conf.get("spark.buffer.size", "65536").toInt
+        compress = conf.get("spark.broadcast.compress", "true").toBoolean
         if (isDriver) {
-          createServer()
+          createServer(conf)
+          conf.set("spark.httpBroadcast.uri",  serverUri)
         }
-        serverUri = System.getProperty("spark.httpBroadcast.uri")
+        serverUri = conf.get("spark.httpBroadcast.uri")
+        cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup, conf)
+        compressionCodec = CompressionCodec.createCodec(conf)
         initialized = true
       }
     }
   }
-  
+
   def stop() {
     synchronized {
       if (server != null) {
         server.stop()
         server = null
       }
+      if (cleaner != null) {
+        cleaner.cancel()
+        cleaner = null
+      }
+      compressionCodec = null
       initialized = false
-      cleaner.cancel()
     }
   }
 
-  private def createServer() {
-    broadcastDir = Utils.createTempDir(Utils.getLocalDir)
+  private def createServer(conf: SparkConf) {
+    broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf))
     server = new HttpServer(broadcastDir)
     server.start()
     serverUri = server.uri
-    System.setProperty("spark.httpBroadcast.uri", serverUri)
     logInfo("Broadcast server started at " + serverUri)
   }
 
@@ -143,7 +150,7 @@ private object HttpBroadcast extends Logging {
     val in = {
       val httpConnection = new URL(url).openConnection()
       httpConnection.setReadTimeout(httpReadTimeout)
-      val inputStream = httpConnection.getInputStream()
+      val inputStream = httpConnection.getInputStream
       if (compress) {
         compressionCodec.compressedInputStream(inputStream)
       } else {
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index 073a0a50297d068963bc3759a557921ce6ee47a4..95309382786a902fceedfeac5e712e8ca021b952 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -83,13 +83,13 @@ extends Broadcast[T](id) with Logging with Serializable {
         case None =>
           val start = System.nanoTime
           logInfo("Started reading broadcast variable " + id)
-          
+
           // Initialize @transient variables that will receive garbage values from the master.
           resetWorkerVariables()
 
           if (receiveBroadcast(id)) {
             value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
-            
+
             // Store the merged copy in cache so that the next worker doesn't need to rebuild it.
             // This creates a tradeoff between memory usage and latency.
             // Storing copy doubles the memory footprint; not storing doubles deserialization cost.
@@ -122,14 +122,14 @@ extends Broadcast[T](id) with Logging with Serializable {
     while (attemptId > 0 && totalBlocks == -1) {
       TorrentBroadcast.synchronized {
         SparkEnv.get.blockManager.getSingle(metaId) match {
-          case Some(x) => 
+          case Some(x) =>
             val tInfo = x.asInstanceOf[TorrentInfo]
             totalBlocks = tInfo.totalBlocks
             totalBytes = tInfo.totalBytes
             arrayOfBlocks = new Array[TorrentBlock](totalBlocks)
             hasBlocks = 0
-          
-          case None => 
+
+          case None =>
             Thread.sleep(500)
         }
       }
@@ -145,13 +145,13 @@ extends Broadcast[T](id) with Logging with Serializable {
       val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid)
       TorrentBroadcast.synchronized {
         SparkEnv.get.blockManager.getSingle(pieceId) match {
-          case Some(x) => 
+          case Some(x) =>
             arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock]
             hasBlocks += 1
             SparkEnv.get.blockManager.putSingle(
               pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true)
-          
-          case None => 
+
+          case None =>
             throw new SparkException("Failed to get " + pieceId + " of " + broadcastId)
         }
       }
@@ -166,21 +166,22 @@ private object TorrentBroadcast
 extends Logging {
 
   private var initialized = false
-
-  def initialize(_isDriver: Boolean) {
+  private var conf: SparkConf = null
+  def initialize(_isDriver: Boolean, conf: SparkConf) {
+    TorrentBroadcast.conf = conf //TODO: we might have to fix it in tests
     synchronized {
       if (!initialized) {
         initialized = true
       }
     }
   }
-  
+
   def stop() {
     initialized = false
   }
 
-  val BLOCK_SIZE = System.getProperty("spark.broadcast.blockSize", "4096").toInt * 1024
-  
+  lazy val BLOCK_SIZE = conf.get("spark.broadcast.blockSize", "4096").toInt * 1024
+
   def blockifyObject[T](obj: T): TorrentInfo = {
     val byteArray = Utils.serialize[T](obj)
     val bais = new ByteArrayInputStream(byteArray)
@@ -209,7 +210,7 @@ extends Logging {
   }
 
   def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock],
-                            totalBytes: Int, 
+                            totalBytes: Int,
                             totalBlocks: Int): T = {
     var retByteArray = new Array[Byte](totalBytes)
     for (i <- 0 until totalBlocks) {
@@ -222,23 +223,23 @@ extends Logging {
 }
 
 private[spark] case class TorrentBlock(
-    blockID: Int, 
-    byteArray: Array[Byte]) 
+    blockID: Int,
+    byteArray: Array[Byte])
   extends Serializable
 
 private[spark] case class TorrentInfo(
     @transient arrayOfBlocks : Array[TorrentBlock],
-    totalBlocks: Int, 
-    totalBytes: Int) 
+    totalBlocks: Int,
+    totalBytes: Int)
   extends Serializable {
-  
-  @transient var hasBlocks = 0 
+
+  @transient var hasBlocks = 0
 }
 
 private[spark] class TorrentBroadcastFactory
   extends BroadcastFactory {
-  
-  def initialize(isDriver: Boolean) { TorrentBroadcast.initialize(isDriver) }
+
+  def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) }
 
   def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
     new TorrentBroadcast[T](value_, isLocal, id)
diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
index 19d393a0dbce7ac92f58a91be04334fd47fe02a4..e38459b883b66b7fd0d7e7dbee05cbeac3645b16 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
@@ -19,7 +19,7 @@ package org.apache.spark.deploy
 
 private[spark] class ApplicationDescription(
     val name: String,
-    val maxCores: Int, /* Integer.MAX_VALUE denotes an unlimited number of cores */
+    val maxCores: Option[Int],
     val memoryPerSlave: Int,
     val command: Command,
     val sparkHome: String,
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e133893f6ca5bab05fea99d4319265fa4c43db74
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.deploy
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.Map
+import scala.concurrent._
+
+import akka.actor._
+import akka.pattern.ask
+import org.apache.log4j.{Level, Logger}
+
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.{DriverState, Master}
+import org.apache.spark.util.{AkkaUtils, Utils}
+import akka.actor.Actor.emptyBehavior
+import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
+
+/**
+ * Proxy that relays messages to the driver.
+ */
+class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with Logging {
+  var masterActor: ActorSelection = _
+  val timeout = AkkaUtils.askTimeout(conf)
+
+  override def preStart() = {
+    masterActor = context.actorSelection(Master.toAkkaUrl(driverArgs.master))
+
+    context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+
+    println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}")
+
+    driverArgs.cmd match {
+      case "launch" =>
+        // TODO: We could add an env variable here and intercept it in `sc.addJar` that would
+        //       truncate filesystem paths similar to what YARN does. For now, we just require
+        //       people call `addJar` assuming the jar is in the same directory.
+        val env = Map[String, String]()
+        System.getenv().foreach{case (k, v) => env(k) = v}
+
+        val mainClass = "org.apache.spark.deploy.worker.DriverWrapper"
+        val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++
+          driverArgs.driverOptions, env)
+
+        val driverDescription = new DriverDescription(
+          driverArgs.jarUrl,
+          driverArgs.memory,
+          driverArgs.cores,
+          driverArgs.supervise,
+          command)
+
+        masterActor ! RequestSubmitDriver(driverDescription)
+
+      case "kill" =>
+        val driverId = driverArgs.driverId
+        val killFuture = masterActor ! RequestKillDriver(driverId)
+    }
+  }
+
+  /* Find out driver status then exit the JVM */
+  def pollAndReportStatus(driverId: String) {
+    println(s"... waiting before polling master for driver state")
+    Thread.sleep(5000)
+    println("... polling master for driver state")
+    val statusFuture = (masterActor ? RequestDriverStatus(driverId))(timeout)
+      .mapTo[DriverStatusResponse]
+    val statusResponse = Await.result(statusFuture, timeout)
+
+    statusResponse.found match {
+      case false =>
+        println(s"ERROR: Cluster master did not recognize $driverId")
+        System.exit(-1)
+      case true =>
+        println(s"State of $driverId is ${statusResponse.state.get}")
+        // Worker node, if present
+        (statusResponse.workerId, statusResponse.workerHostPort, statusResponse.state) match {
+          case (Some(id), Some(hostPort), Some(DriverState.RUNNING)) =>
+            println(s"Driver running on $hostPort ($id)")
+          case _ =>
+        }
+        // Exception, if present
+        statusResponse.exception.map { e =>
+          println(s"Exception from cluster was: $e")
+          System.exit(-1)
+        }
+        System.exit(0)
+    }
+  }
+
+  override def receive = {
+
+    case SubmitDriverResponse(success, driverId, message) =>
+      println(message)
+      if (success) pollAndReportStatus(driverId.get) else System.exit(-1)
+
+    case KillDriverResponse(driverId, success, message) =>
+      println(message)
+      if (success) pollAndReportStatus(driverId) else System.exit(-1)
+
+    case DisassociatedEvent(_, remoteAddress, _) =>
+      println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
+      System.exit(-1)
+
+    case AssociationErrorEvent(cause, _, remoteAddress, _) =>
+      println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
+      println(s"Cause was: $cause")
+      System.exit(-1)
+  }
+}
+
+/**
+ * Executable utility for starting and terminating drivers inside of a standalone cluster.
+ */
+object Client {
+  def main(args: Array[String]) {
+    val conf = new SparkConf()
+    val driverArgs = new ClientArguments(args)
+
+    if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
+      conf.set("spark.akka.logLifecycleEvents", "true")
+    }
+    conf.set("spark.akka.askTimeout", "10")
+    conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING"))
+    Logger.getRootLogger.setLevel(driverArgs.logLevel)
+
+    // TODO: See if we can initialize akka so return messages are sent back using the same TCP
+    //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
+    val (actorSystem, _) = AkkaUtils.createActorSystem(
+      "driverClient", Utils.localHostName(), 0, false, conf)
+
+    actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf))
+
+    actorSystem.awaitTermination()
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
similarity index 64%
rename from core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
rename to core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index 6a15422c6cd6d8f486a8ce0b1ce2c36931bd127e..db67c6d1bb55c58069a23a7d2d3abb01a4d2a1ea 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -15,23 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy.client
+package org.apache.spark.deploy
+
+import java.net.URL
 
 import scala.collection.mutable.ListBuffer
 
+import org.apache.log4j.Level
+
 /**
  * Command-line parser for the driver client.
  */
-private[spark] class DriverClientArguments(args: Array[String]) {
+private[spark] class ClientArguments(args: Array[String]) {
+  val defaultCores = 1
+  val defaultMemory = 512
+
   var cmd: String = "" // 'launch' or 'kill'
+  var logLevel = Level.WARN
 
   // launch parameters
   var master: String = ""
   var jarUrl: String = ""
   var mainClass: String = ""
   var supervise: Boolean = false
-  var memory: Int = 512
-  var cores: Int = 1
+  var memory: Int = defaultMemory
+  var cores: Int = defaultCores
   private var _driverOptions = ListBuffer[String]()
   def driverOptions = _driverOptions.toSeq
 
@@ -56,10 +64,24 @@ private[spark] class DriverClientArguments(args: Array[String]) {
     case ("--help" | "-h") :: tail =>
       printUsageAndExit(0)
 
+    case ("--verbose" | "-v") :: tail =>
+      logLevel = Level.INFO
+      parse(tail)
+
     case "launch" :: _master :: _jarUrl :: _mainClass :: tail =>
       cmd = "launch"
-      master = _master
+
+      try {
+        new URL(_jarUrl)
+      } catch {
+        case e: Exception =>
+          println(s"Jar url '${_jarUrl}' is not a valid URL.")
+          println(s"Jar must be in URL format (e.g. hdfs://XX, file://XX)")
+          printUsageAndExit(-1)
+      }
+
       jarUrl = _jarUrl
+      master = _master
       mainClass = _mainClass
       _driverOptions ++= tail
 
@@ -78,14 +100,18 @@ private[spark] class DriverClientArguments(args: Array[String]) {
   def printUsageAndExit(exitCode: Int) {
     // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
     //       separately similar to in the YARN client.
-    System.err.println(
-      "usage: DriverClient [options] launch <active-master> <jar-url> <main-class> " +
-        "[driver options]\n" +
-      "usage: DriverClient kill <active-master> <driver-id>\n\n" +
-      "Options:\n" +
-      "  -c CORES, --cores CORES                Number of cores to request \n" +
-      "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n" +
-      "  -s, --supervise                        Whether to restart the driver on failure\n")
+    val usage =
+      s"""
+        |Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options]
+        |Usage: DriverClient kill <active-master> <driver-id>
+        |
+        |Options:
+        |   -c CORES, --cores CORES        Number of cores to request (default: $defaultCores)
+        |   -m MEMORY, --memory MEMORY     Megabytes of memory to request (default: $defaultMemory)
+        |   -s, --supervise                Whether to restart the driver on failure
+        |   -v, --verbose                  Print more debugging output
+      """.stripMargin
+    System.err.println(usage)
     System.exit(exitCode)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 34460d359d2e8e39868af4d2c87a6f6f5fda3fbe..5e824e1a678b647c1ef95387ae8674736e6565cd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -112,11 +112,18 @@ private[deploy] object DeployMessages {
 
   case class RequestSubmitDriver(driverDescription: DriverDescription) extends DeployMessage
 
-  case class SubmitDriverResponse(success: Boolean, message: String) extends DeployMessage
+  case class SubmitDriverResponse(success: Boolean, driverId: Option[String], message: String)
+    extends DeployMessage
 
   case class RequestKillDriver(driverId: String) extends DeployMessage
 
-  case class KillDriverResponse(success: Boolean, message: String) extends DeployMessage
+  case class KillDriverResponse(driverId: String, success: Boolean, message: String)
+    extends DeployMessage
+
+  case class RequestDriverStatus(driverId: String) extends DeployMessage
+
+  case class DriverStatusResponse(found: Boolean, state: Option[DriverState],
+    workerId: Option[String], workerHostPort: Option[String], exception: Option[Exception])
 
   // Internal message in AppClient
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
index 0aa8852649e058adf514224fd07ba4b78b7c92e8..4dfb19ed8adb6865c6a5912eada5f292d5b3d624 100644
--- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -190,7 +190,7 @@ private[spark] object FaultToleranceTest extends App with Logging {
   /** Creates a SparkContext, which constructs a Client to interact with our cluster. */
   def createClient() = {
     if (sc != null) { sc.stop() }
-    // Counter-hack: Because of a hack in SparkEnv#createFromSystemProperties() that changes this
+    // Counter-hack: Because of a hack in SparkEnv#create() that changes this
     // property, we need to reset it.
     System.setProperty("spark.driver.port", "0")
     sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome)
@@ -417,4 +417,4 @@ private[spark] object Docker extends Logging {
     "docker ps -l -q".!(ProcessLogger(line => id = line))
     new DockerId(id)
   }
-}
\ No newline at end of file
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 59d12a3e6f842904f17caee6a45d255db2f5a95a..ffc0cb09032fb6e324fa33e828f256a7672b1033 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -22,7 +22,7 @@ import akka.actor.ActorSystem
 import org.apache.spark.deploy.worker.Worker
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.util.Utils
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 import scala.collection.mutable.ArrayBuffer
 
@@ -43,7 +43,8 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
     logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
 
     /* Start the Master */
-    val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0)
+    val conf = new SparkConf(false)
+    val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0, conf)
     masterActorSystems += masterSystem
     val masterUrl = "spark://" + localHostname + ":" + masterPort
     val masters = Array(masterUrl)
@@ -55,7 +56,7 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
       workerActorSystems += workerSystem
     }
 
-    return masters
+    masters
   }
 
   def stop() {
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index fc1537f7963c44b4adb8e2e72c8786b92e6aaf70..27dc42bf7e50e260e1f43c5c4fc85da5a3c38cd3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -34,10 +34,10 @@ class SparkHadoopUtil {
   UserGroupInformation.setConfiguration(conf)
 
   def runAsUser(user: String)(func: () => Unit) {
-    // if we are already running as the user intended there is no reason to do the doAs. It 
+    // if we are already running as the user intended there is no reason to do the doAs. It
     // will actually break secure HDFS access as it doesn't fill in the credentials. Also if
-    // the user is UNKNOWN then we shouldn't be creating a remote unknown user 
-    // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only 
+    // the user is UNKNOWN then we shouldn't be creating a remote unknown user
+    // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only
     // in SparkContext.
     val currentUser = Option(System.getProperty("user.name")).
       getOrElse(SparkContext.SPARK_UNKNOWN_USER)
@@ -67,11 +67,15 @@ class SparkHadoopUtil {
 }
 
 object SparkHadoopUtil {
+
   private val hadoop = {
-    val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
+    val yarnMode = java.lang.Boolean.valueOf(
+        System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
     if (yarnMode) {
       try {
-        Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil]
+        Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
+          .newInstance()
+          .asInstanceOf[SparkHadoopUtil]
       } catch {
        case th: Throwable => throw new SparkException("Unable to load YARN support", th)
       }
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index 4779c75137d4ddcb1703e5ed3ea86d7d7d2743f5..1415e2f3d1886974cdc6a725b973ffbbe6ccc7e8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -19,20 +19,19 @@ package org.apache.spark.deploy.client
 
 import java.util.concurrent.TimeoutException
 
-import scala.concurrent.duration._
 import scala.concurrent.Await
+import scala.concurrent.duration._
 
 import akka.actor._
 import akka.pattern.ask
-import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent}
+import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
 
-import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.{Logging, SparkConf, SparkException}
 import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.util.AkkaUtils
 
-
 /**
  * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
  * an app description, and a listener for cluster events, and calls back the listener when various
@@ -44,7 +43,8 @@ private[spark] class AppClient(
     actorSystem: ActorSystem,
     masterUrls: Array[String],
     appDescription: ApplicationDescription,
-    listener: AppClientListener)
+    listener: AppClientListener,
+    conf: SparkConf)
   extends Logging {
 
   val REGISTRATION_TIMEOUT = 20.seconds
@@ -112,6 +112,12 @@ private[spark] class AppClient(
       }
     }
 
+    private def isPossibleMaster(remoteUrl: Address) = {
+      masterUrls.map(s => Master.toAkkaUrl(s))
+        .map(u => AddressFromURIString(u).hostPort)
+        .contains(remoteUrl.hostPort)
+    }
+
     override def receive = {
       case RegisteredApplication(appId_, masterUrl) =>
         appId = appId_
@@ -147,6 +153,9 @@ private[spark] class AppClient(
         logWarning(s"Connection to $address failed; waiting for master to reconnect...")
         markDisconnected()
 
+      case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) =>
+        logWarning(s"Could not connect to $address: $cause")
+
       case StopAppClient =>
         markDead()
         sender ! true
@@ -179,7 +188,7 @@ private[spark] class AppClient(
   def stop() {
     if (actor != null) {
       try {
-        val timeout = AkkaUtils.askTimeout
+        val timeout = AkkaUtils.askTimeout(conf)
         val future = actor.ask(StopAppClient)(timeout)
         Await.result(future, timeout)
       } catch {
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
deleted file mode 100644
index 8a4cdf07bbbaf169fb783ad5ad1e776bf8e9f1d9..0000000000000000000000000000000000000000
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.deploy.client
-
-import scala.collection.JavaConversions._
-import scala.collection.mutable.Map
-import scala.concurrent._
-
-import akka.actor._
-
-import org.apache.spark.Logging
-import org.apache.spark.deploy.{Command, DriverDescription}
-import org.apache.spark.deploy.DeployMessages._
-import org.apache.spark.deploy.master.Master
-import org.apache.spark.util.{AkkaUtils, Utils}
-
-/**
- * Actor that sends a single message to the standalone master and returns the response in the
- * given promise.
- */
-class DriverActor(master: String, response: Promise[(Boolean, String)]) extends Actor with Logging {
-  override def receive = {
-    case SubmitDriverResponse(success, message) => {
-      response.success((success, message))
-    }
-
-    case KillDriverResponse(success, message) => {
-      response.success((success, message))
-    }
-
-    // Relay all other messages to the master.
-    case message => {
-      logInfo(s"Sending message to master $master...")
-      val masterActor = context.actorSelection(Master.toAkkaUrl(master))
-      masterActor ! message
-    }
-  }
-}
-
-/**
- * Executable utility for starting and terminating drivers inside of a standalone cluster.
- */
-object DriverClient extends Logging {
-
-  def main(args: Array[String]) {
-    val driverArgs = new DriverClientArguments(args)
-
-    // TODO: See if we can initialize akka so return messages are sent back using the same TCP
-    //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
-      "driverClient", Utils.localHostName(), 0)
-    val master = driverArgs.master
-    val response = promise[(Boolean, String)]
-    val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
-
-    driverArgs.cmd match {
-      case "launch" =>
-        // TODO: We could add an env variable here and intercept it in `sc.addJar` that would
-        //       truncate filesystem paths similar to what YARN does. For now, we just require
-        //       people call `addJar` assuming the jar is in the same directory.
-        val env = Map[String, String]()
-        System.getenv().foreach{case (k, v) => env(k) = v}
-
-        val mainClass = "org.apache.spark.deploy.worker.DriverWrapper"
-        val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++
-          driverArgs.driverOptions, env)
-
-        val driverDescription = new DriverDescription(
-          driverArgs.jarUrl,
-          driverArgs.memory,
-          driverArgs.cores,
-          driverArgs.supervise,
-          command)
-        driver ! RequestSubmitDriver(driverDescription)
-
-      case "kill" =>
-        val driverId = driverArgs.driverId
-        driver ! RequestKillDriver(driverId)
-    }
-
-    val (success, message) =
-      try {
-        Await.result(response.future, AkkaUtils.askTimeout)
-      } catch {
-        case e: TimeoutException => (false, s"Master $master failed to respond in time")
-      }
-    if (success) logInfo(message) else logError(message)
-    actorSystem.shutdown()
-    actorSystem.awaitTermination()
-  }
-}
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index 9359bf1a4b6723eaa3875cd6698ab01ebaa6e712..ffa909c26b64aa26c725b57cf9dfc80a7cd749fc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.deploy.client
 
 import org.apache.spark.util.{Utils, AkkaUtils}
-import org.apache.spark.{Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.deploy.{Command, ApplicationDescription}
 
 private[spark] object TestClient {
@@ -45,11 +45,13 @@ private[spark] object TestClient {
 
   def main(args: Array[String]) {
     val url = args(0)
-    val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
+    val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0,
+      conf = new SparkConf)
     val desc = new ApplicationDescription(
-      "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
+      "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()),
+      "dummy-spark-home", "ignored")
     val listener = new TestListener
-    val client = new AppClient(actorSystem, Array(url), desc, listener)
+    val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf)
     client.start()
     actorSystem.awaitTermination()
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index 5150b7c7dec6ab135ba65b1d7c992901e164f5e0..3e26379166885fac81d7320a0d8f5ce8ec095889 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -28,7 +28,8 @@ private[spark] class ApplicationInfo(
     val desc: ApplicationDescription,
     val submitDate: Date,
     val driver: ActorRef,
-    val appUiUrl: String)
+    val appUiUrl: String,
+    defaultCores: Int)
   extends Serializable {
 
   @transient var state: ApplicationState.Value = _
@@ -81,7 +82,9 @@ private[spark] class ApplicationInfo(
     }
   }
 
-  def coresLeft: Int = desc.maxCores - coresGranted
+  private val myMaxCores = desc.maxCores.getOrElse(defaultCores)
+
+  def coresLeft: Int = myMaxCores - coresGranted
 
   private var _retryCount = 0
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
index 93b260740e81817e759d7dbcbdbd4d7a45e9caa0..26a68bade3c60eb5003b9a1fc0dfab4485cb6dc7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
@@ -27,6 +27,7 @@ private[spark] object DriverState extends Enumeration {
   // RELAUNCHING: Exited non-zero or due to worker failure, but has not yet started running again
   // UNKNOWN: The state of the driver is temporarily not known due to master failure recovery
   // KILLED: A user manually killed this driver
-  // FAILED: Unable to run due to an unrecoverable error (e.g. missing jar file)
-  val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED = Value
+  // FAILED: The driver exited non-zero and was not supervised
+  // ERROR: Unable to run or restart due to an unrecoverable error (e.g. missing jar file)
+  val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED, ERROR = Value
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index b8655c78db2b53d99a479b7d5616157bd264765f..cd3f3ebefc1390536ff104f30b4969bcdbec74c2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -30,7 +30,8 @@ import akka.pattern.ask
 import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
 import akka.serialization.SerializationExtension
 
-import org.apache.spark.{Logging, SparkException}
+
+import org.apache.spark.{SparkConf, Logging, SparkException}
 import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.MasterMessages._
@@ -40,14 +41,16 @@ import org.apache.spark.util.{AkkaUtils, Utils}
 import org.apache.spark.deploy.master.DriverState.DriverState
 
 private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
-  import context.dispatcher
+  import context.dispatcher   // to use Akka's scheduler.schedule()
+
+  val conf = new SparkConf
 
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
-  val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
-  val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt
-  val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt
-  val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "")
-  val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE")
+  val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000
+  val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt
+  val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt
+  val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "")
+  val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE")
 
   val workers = new HashSet[WorkerInfo]
   val idToWorker = new HashMap[String, WorkerInfo]
@@ -69,8 +72,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 
   Utils.checkHost(host, "Expected hostname")
 
-  val masterMetricsSystem = MetricsSystem.createMetricsSystem("master")
-  val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications")
+  val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf)
+  val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf)
   val masterSource = new MasterSource(this)
 
   val webUi = new MasterWebUI(this, webUiPort)
@@ -92,7 +95,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
   // As a temporary workaround before better ways of configuring memory, we allow users to set
   // a flag that will perform round-robin scheduling across the nodes (spreading out each app
   // among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
-  val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
+  val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true)
+
+  // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue)
+  val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue)
+  if (defaultCores < 1) {
+    throw new SparkException("spark.deploy.defaultCores must be positive")
+  }
 
   override def preStart() {
     logInfo("Starting Spark master at " + masterUrl)
@@ -109,7 +118,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     persistenceEngine = RECOVERY_MODE match {
       case "ZOOKEEPER" =>
         logInfo("Persisting recovery state to ZooKeeper")
-        new ZooKeeperPersistenceEngine(SerializationExtension(context.system))
+        new ZooKeeperPersistenceEngine(SerializationExtension(context.system), conf)
       case "FILESYSTEM" =>
         logInfo("Persisting recovery state to directory: " + RECOVERY_DIR)
         new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system))
@@ -119,7 +128,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 
     leaderElectionAgent = RECOVERY_MODE match {
         case "ZOOKEEPER" =>
-          context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl))
+          context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl, conf))
         case _ =>
           context.actorOf(Props(classOf[MonarchyLeaderAgent], self))
       }
@@ -177,7 +186,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     case RequestSubmitDriver(description) => {
       if (state != RecoveryState.ALIVE) {
         val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state."
-        sender ! SubmitDriverResponse(false, msg)
+        sender ! SubmitDriverResponse(false, None, msg)
       } else {
         logInfo("Driver submitted " + description.command.mainClass)
         val driver = createDriver(description)
@@ -189,14 +198,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         // TODO: It might be good to instead have the submission client poll the master to determine
         //       the current status of the driver. For now it's simply "fire and forget".
 
-        sender ! SubmitDriverResponse(true, s"Driver successfully submitted as ${driver.id}")
+        sender ! SubmitDriverResponse(true, Some(driver.id),
+          s"Driver successfully submitted as ${driver.id}")
       }
     }
 
     case RequestKillDriver(driverId) => {
       if (state != RecoveryState.ALIVE) {
         val msg = s"Can only kill drivers in ALIVE state. Current state: $state."
-        sender ! KillDriverResponse(false, msg)
+        sender ! KillDriverResponse(driverId, success = false, msg)
       } else {
         logInfo("Asked to kill driver " + driverId)
         val driver = drivers.find(_.id == driverId)
@@ -217,15 +227,25 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
             // TODO: It would be nice for this to be a synchronous response
             val msg = s"Kill request for $driverId submitted"
             logInfo(msg)
-            sender ! KillDriverResponse(true, msg)
+            sender ! KillDriverResponse(driverId, success = true, msg)
           case None =>
-            val msg = s"Could not find running driver $driverId"
+            val msg = s"Driver $driverId has already finished or does not exist"
             logWarning(msg)
-            sender ! KillDriverResponse(false, msg)
+            sender ! KillDriverResponse(driverId, success = false, msg)
         }
       }
     }
 
+    case RequestDriverStatus(driverId) => {
+      (drivers ++ completedDrivers).find(_.id == driverId) match {
+        case Some(driver) =>
+          sender ! DriverStatusResponse(found = true, Some(driver.state),
+            driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception)
+        case None =>
+          sender ! DriverStatusResponse(found = false, None, None, None, None)
+      }
+    }
+
     case RegisterApplication(description) => {
       if (state == RecoveryState.STANDBY) {
         // ignore, don't send response
@@ -270,7 +290,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 
     case DriverStateChanged(driverId, state, exception) => {
       state match {
-        case DriverState.FAILED | DriverState.FINISHED | DriverState.KILLED =>
+        case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED =>
           removeDriver(driverId, state, exception)
         case _ =>
           throw new Exception(s"Received unexpected state update for driver $driverId: $state")
@@ -396,8 +416,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 
     // Reschedule drivers which were not claimed by any workers
     drivers.filter(_.worker.isEmpty).foreach { d =>
-      logWarning(s"Driver ${d.id} was not found after master recovery, re-launching")
-      relaunchDriver(d)
+      logWarning(s"Driver ${d.id} was not found after master recovery")
+      if (d.desc.supervise) {
+        logWarning(s"Re-launching ${d.id}")
+        relaunchDriver(d)
+      } else {
+        removeDriver(d.id, DriverState.ERROR, None)
+        logWarning(s"Did not re-launch ${d.id} because it was not supervised")
+      }
     }
 
     state = RecoveryState.ALIVE
@@ -519,7 +545,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       exec.application.removeExecutor(exec)
     }
     for (driver <- worker.drivers.values) {
-      relaunchDriver(driver)
+      if (driver.desc.supervise) {
+        logInfo(s"Re-launching ${driver.id}")
+        relaunchDriver(driver)
+      } else {
+        logInfo(s"Not re-launching ${driver.id} because it was not supervised")
+        removeDriver(driver.id, DriverState.ERROR, None)
+      }
     }
     persistenceEngine.removeWorker(worker)
   }
@@ -534,7 +566,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
   def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
     val now = System.currentTimeMillis()
     val date = new Date(now)
-    new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+    new ApplicationInfo(
+      now, newApplicationId(date), desc, date, driver, desc.appUiUrl, defaultCores)
   }
 
   def registerApplication(app: ApplicationInfo): Unit = {
@@ -605,7 +638,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         removeWorker(worker)
       } else {
         if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT))
-          workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it 
+          workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it
       }
     }
   }
@@ -652,8 +685,9 @@ private[spark] object Master {
   val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
 
   def main(argStrings: Array[String]) {
-    val args = new MasterArguments(argStrings)
-    val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort)
+    val conf = new SparkConf
+    val args = new MasterArguments(argStrings, conf)
+    val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf)
     actorSystem.awaitTermination()
   }
 
@@ -667,10 +701,12 @@ private[spark] object Master {
     }
   }
 
-  def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = {
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
+  def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf)
+      : (ActorSystem, Int, Int) =
+  {
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf)
     val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName)
-    val timeout = AkkaUtils.askTimeout
+    val timeout = AkkaUtils.askTimeout(conf)
     val respFuture = actor.ask(RequestWebUIPort)(timeout)
     val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse]
     (actorSystem, boundPort, resp.webUIBoundPort)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 9d89b455fb961e6ddd1d9bc0883b408b3516ab38..e7f3224091ca8b4475a041de30f53afadd00f5f3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -18,16 +18,17 @@
 package org.apache.spark.deploy.master
 
 import org.apache.spark.util.{Utils, IntParam}
+import org.apache.spark.SparkConf
 
 /**
  * Command-line parser for the master.
  */
-private[spark] class MasterArguments(args: Array[String]) {
+private[spark] class MasterArguments(args: Array[String], conf: SparkConf) {
   var host = Utils.localHostName()
   var port = 7077
   var webUiPort = 8080
-  
-  // Check for settings in environment variables 
+
+  // Check for settings in environment variables
   if (System.getenv("SPARK_MASTER_HOST") != null) {
     host = System.getenv("SPARK_MASTER_HOST")
   }
@@ -37,8 +38,8 @@ private[spark] class MasterArguments(args: Array[String]) {
   if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) {
     webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt
   }
-  if (System.getProperty("master.ui.port") != null) {
-    webUiPort = System.getProperty("master.ui.port").toInt
+  if (conf.contains("master.ui.port")) {
+    webUiPort = conf.get("master.ui.port").toInt
   }
 
   parse(args.toList)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
index 6cc7fd2ff498f483dc5868bce4d67802b3da42fc..999090ad7464dd5becec6c7a5ee4d2e8cb784fca 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
@@ -23,7 +23,7 @@ import org.apache.zookeeper._
 import org.apache.zookeeper.Watcher.Event.KeeperState
 import org.apache.zookeeper.data.Stat
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 /**
  * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry
@@ -35,8 +35,9 @@ import org.apache.spark.Logging
  * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many
  * times or a semantic exception is thrown (e.g., "node already exists").
  */
-private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging {
-  val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "")
+private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher,
+    conf: SparkConf) extends Logging {
+  val ZK_URL = conf.get("spark.deploy.zookeeper.url", "")
 
   val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE
   val ZK_TIMEOUT_MILLIS = 30000
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
index 7d535b08def1650fd7b5f1b4fc0a0489ee9db76e..77c23fb9fbeea5b23266e66fb627a964cdaae2b3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
@@ -21,16 +21,17 @@ import akka.actor.ActorRef
 import org.apache.zookeeper._
 import org.apache.zookeeper.Watcher.Event.EventType
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 import org.apache.spark.deploy.master.MasterMessages._
 
-private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String)
+private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef,
+    masterUrl: String, conf: SparkConf)
   extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging  {
 
-  val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
+  val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
 
   private val watcher = new ZooKeeperWatcher()
-  private val zk = new SparkZooKeeperSession(this)
+  private val zk = new SparkZooKeeperSession(this, conf)
   private var status = LeadershipStatus.NOT_LEADER
   private var myLeaderFile: String = _
   private var leaderUrl: String = _
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index 52df1738507f391f2faf6409b7f5c99d87277210..f24f49ea8ad9ff939593da4bdcc4db552e3f1fff 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -17,19 +17,19 @@
 
 package org.apache.spark.deploy.master
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 import org.apache.zookeeper._
 
 import akka.serialization.Serialization
 
-class ZooKeeperPersistenceEngine(serialization: Serialization)
+class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
   extends PersistenceEngine
   with SparkZooKeeperWatcher
   with Logging
 {
-  val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
+  val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
 
-  val zk = new SparkZooKeeperSession(this)
+  val zk = new SparkZooKeeperSession(this, conf)
 
   zk.connect()
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index a72d76be521cab37787ca904da63a6b216023a2a..a9af8df5525d68375b552be8526e3f1a685da111 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -63,6 +63,10 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
     val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers)
 
+    // For now we only show driver information if the user has submitted drivers to the cluster.
+    // This is until we integrate the notion of drivers and applications in the UI.
+    def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0
+
     val content =
         <div class="row-fluid">
           <div class="span12">
@@ -98,26 +102,35 @@ private[spark] class IndexPage(parent: MasterWebUI) {
           </div>
         </div>
 
-        <div class="row-fluid">
-          <div class="span12">
-            <h4> Completed Applications </h4>
-            {completedAppsTable}
+        <div>
+          {if (hasDrivers)
+          <div class="row-fluid">
+            <div class="span12">
+              <h4> Running Drivers </h4>
+              {activeDriversTable}
+            </div>
           </div>
+          }
         </div>
 
         <div class="row-fluid">
           <div class="span12">
-            <h4> Active Drivers </h4>
-            {activeDriversTable}
+            <h4> Completed Applications </h4>
+            {completedAppsTable}
           </div>
         </div>
 
-        <div class="row-fluid">
-          <div class="span12">
-            <h4> Completed Drivers </h4>
-            {completedDriversTable}
+        <div>
+          {if (hasDrivers)
+          <div class="row-fluid">
+            <div class="span12">
+              <h4> Completed Drivers </h4>
+              {completedDriversTable}
+            </div>
           </div>
+          }
         </div>;
+
     UIUtils.basicSparkPage(content, "Spark Master at " + state.uri)
   }
 
@@ -162,7 +175,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     <tr>
       <td>{driver.id} </td>
       <td>{driver.submitDate}</td>
-      <td>{driver.worker.map(w => w.id.toString).getOrElse("None")}</td>
+      <td>{driver.worker.map(w => <a href={w.webUiAddress}>{w.id.toString}</a>).getOrElse("None")}</td>
       <td>{driver.state}</td>
       <td sorttable_customkey={driver.desc.cores.toString}>
         {driver.desc.cores}
@@ -170,7 +183,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
       <td sorttable_customkey={driver.desc.mem.toString}>
         {Utils.megabytesToString(driver.desc.mem.toLong)}
       </td>
-      <td>{driver.desc.command.mainClass}</td>
+      <td>{driver.desc.command.arguments(1)}</td>
     </tr>
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index 9ab594b682c43ce45b5262ab5bd40ffdd8b8d637..ead35662fc75ae0c35b0da33056dc74e486876d9 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -31,7 +31,7 @@ import org.apache.spark.util.{AkkaUtils, Utils}
  */
 private[spark]
 class MasterWebUI(val master: Master, requestedPort: Int) extends Logging {
-  val timeout = AkkaUtils.askTimeout
+  val timeout = AkkaUtils.askTimeout(master.conf)
   val host = Utils.localHostName()
   val port = requestedPort
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
index 785aecf1fe4719aa42087130abc5fc877c5199a4..7507bf8ad0e6c56b8f09381066ed5168add24d24 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -21,7 +21,7 @@ object CommandUtils extends Logging {
   }
 
   private def getEnv(key: String, command: Command): Option[String] =
-    command.environment.get(key).orElse(Option(getenv(key)))
+    command.environment.get(key).orElse(Option(System.getenv(key)))
 
   /**
    * Attention: this must always be aligned with the environment variables in the run scripts and
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 2d567b7a411b66e2c5d9e0e0a468527143f08826..b4df1a0dd47184c12b24d9a460faa63eb3547ca1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -19,6 +19,7 @@ package org.apache.spark.deploy.worker
 
 import java.io._
 
+import scala.collection.JavaConversions._
 import scala.collection.mutable.Map
 
 import akka.actor.ActorRef
@@ -31,6 +32,7 @@ import org.apache.spark.Logging
 import org.apache.spark.deploy.{Command, DriverDescription}
 import org.apache.spark.deploy.DeployMessages.DriverStateChanged
 import org.apache.spark.deploy.master.DriverState
+import org.apache.spark.deploy.master.DriverState.DriverState
 
 /**
  * Manages the execution of one driver, including automatically restarting the driver on failure.
@@ -47,12 +49,25 @@ private[spark] class DriverRunner(
   @volatile var process: Option[Process] = None
   @volatile var killed = false
 
+  // Populated once finished
+  var finalState: Option[DriverState] = None
+  var finalException: Option[Exception] = None
+  var finalExitCode: Option[Int] = None
+
+  // Decoupled for testing
+  private[deploy] def setClock(_clock: Clock) = clock = _clock
+  private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper
+  private var clock = new Clock {
+    def currentTimeMillis(): Long = System.currentTimeMillis()
+  }
+  private var sleeper = new Sleeper {
+    def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed})
+  }
+
   /** Starts a thread to run and manage the driver. */
   def start() = {
     new Thread("DriverRunner for " + driverId) {
       override def run() {
-        var exn: Option[Exception] = None
-
         try {
           val driverDir = createWorkingDirectory()
           val localJarFilename = downloadUserJar(driverDir)
@@ -63,21 +78,27 @@ private[spark] class DriverRunner(
           env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename"
           val newCommand = Command(driverDesc.command.mainClass,
             driverDesc.command.arguments.map(substituteVariables), env)
-
           val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem,
             sparkHome.getAbsolutePath)
-          runCommand(command, env, driverDir, driverDesc.supervise)
+          launchDriver(command, env, driverDir, driverDesc.supervise)
         }
         catch {
-          case e: Exception => exn = Some(e)
+          case e: Exception => finalException = Some(e)
         }
 
-        val finalState =
+        val state =
           if (killed) { DriverState.KILLED }
-          else if (exn.isDefined) { DriverState.FAILED }
-          else { DriverState.FINISHED }
+          else if (finalException.isDefined) { DriverState.ERROR }
+          else {
+            finalExitCode match {
+              case Some(0) => DriverState.FINISHED
+              case _ => DriverState.FAILED
+            }
+          }
 
-        worker ! DriverStateChanged(driverId, finalState, exn)
+        finalState = Some(state)
+
+        worker ! DriverStateChanged(driverId, state, finalException)
       }
     }.start()
   }
@@ -116,11 +137,10 @@ private[spark] class DriverRunner(
 
     val jarPath = new Path(driverDesc.jarUrl)
 
-    val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path
+    val emptyConf = new Configuration()
     val jarFileSystem = jarPath.getFileSystem(emptyConf)
 
-    val destPath = new File(driverDir.getAbsolutePath(), jarPath.getName())
-    // val destFileSystem = destPath.getFileSystem(emptyConf)
+    val destPath = new File(driverDir.getAbsolutePath, jarPath.getName)
     val jarFileName = jarPath.getName
     val localJarFile = new File(driverDir, jarFileName)
     val localJarFilename = localJarFile.getAbsolutePath
@@ -137,43 +157,78 @@ private[spark] class DriverRunner(
     localJarFilename
   }
 
-  /** Launch the supplied command. */
-  private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File,
-      supervise: Boolean) {
+  private def launchDriver(command: Seq[String], envVars: Map[String, String], baseDir: File,
+                           supervise: Boolean) {
+    val builder = new ProcessBuilder(command: _*).directory(baseDir)
+    envVars.map{ case(k,v) => builder.environment().put(k, v) }
+
+    def initialize(process: Process) = {
+      // Redirect stdout and stderr to files
+      val stdout = new File(baseDir, "stdout")
+      CommandUtils.redirectStream(process.getInputStream, stdout)
+
+      val stderr = new File(baseDir, "stderr")
+      val header = "Launch Command: %s\n%s\n\n".format(
+        command.mkString("\"", "\" \"", "\""), "=" * 40)
+      Files.append(header, stderr, Charsets.UTF_8)
+      CommandUtils.redirectStream(process.getErrorStream, stderr)
+    }
+    runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise)
+  }
+
+  private[deploy] def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit,
+    supervise: Boolean) {
     // Time to wait between submission retries.
     var waitSeconds = 1
+    // A run of this many seconds resets the exponential back-off.
+    val successfulRunDuration = 5
+
     var keepTrying = !killed
 
     while (keepTrying) {
-      logInfo("Launch Command: " + command.mkString("\"", "\" \"", "\""))
-      val builder = new ProcessBuilder(command: _*).directory(baseDir)
-      envVars.map{ case(k,v) => builder.environment().put(k, v) }
+      logInfo("Launch Command: " + command.command.mkString("\"", "\" \"", "\""))
 
       synchronized {
         if (killed) { return }
-
-        process = Some(builder.start())
-
-        // Redirect stdout and stderr to files
-        val stdout = new File(baseDir, "stdout")
-        CommandUtils.redirectStream(process.get.getInputStream, stdout)
-
-        val stderr = new File(baseDir, "stderr")
-        val header = "Launch Command: %s\n%s\n\n".format(
-          command.mkString("\"", "\" \"", "\""), "=" * 40)
-        Files.append(header, stderr, Charsets.UTF_8)
-        CommandUtils.redirectStream(process.get.getErrorStream, stderr)
+        process = Some(command.start())
+        initialize(process.get)
       }
 
+      val processStart = clock.currentTimeMillis()
       val exitCode = process.get.waitFor()
+      if (clock.currentTimeMillis() - processStart > successfulRunDuration * 1000) {
+        waitSeconds = 1
+      }
 
       if (supervise && exitCode != 0 && !killed) {
-        waitSeconds = waitSeconds * 1 // exponential back-off
         logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.")
-        (0 until waitSeconds).takeWhile(f => {Thread.sleep(1000); !killed})
+        sleeper.sleep(waitSeconds)
+        waitSeconds = waitSeconds * 2 // exponential back-off
       }
 
       keepTrying = supervise && exitCode != 0 && !killed
+      finalExitCode = Some(exitCode)
     }
   }
 }
+
+private[deploy] trait Clock {
+  def currentTimeMillis(): Long
+}
+
+private[deploy] trait Sleeper {
+  def sleep(seconds: Int)
+}
+
+// Needed because ProcessBuilder is a final class and cannot be mocked
+private[deploy] trait ProcessBuilderLike {
+  def start(): Process
+  def command: Seq[String]
+}
+
+private[deploy] object ProcessBuilderLike {
+  def apply(processBuilder: ProcessBuilder) = new ProcessBuilderLike {
+    def start() = processBuilder.start()
+    def command = processBuilder.command()
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index 8c13b10c5152e1b12fc41de58208790afd3b441a..1640d5fee0f77404fc93c884bd2c8ecd8212d54b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -2,6 +2,7 @@ package org.apache.spark.deploy.worker
 
 import akka.actor._
 
+import org.apache.spark.SparkConf
 import org.apache.spark.util.{AkkaUtils, Utils}
 
 /**
@@ -11,8 +12,8 @@ object DriverWrapper {
   def main(args: Array[String]) {
     args.toList match {
       case workerUrl :: mainClass :: extraArgs =>
-        val (actorSystem, boundPort) = AkkaUtils.createActorSystem("Driver",
-          Utils.localHostName(), 0)
+        val (actorSystem, _) = AkkaUtils.createActorSystem("Driver",
+          Utils.localHostName(), 0, false, new SparkConf())
         actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher")
 
         // Delegate to supplied main class
@@ -20,7 +21,7 @@ object DriverWrapper {
         val mainMethod = clazz.getMethod("main", classOf[Array[String]])
         mainMethod.invoke(null, extraArgs.toArray[String])
 
-        actorSystem.awaitTermination()
+        actorSystem.shutdown()
 
       case _ =>
         System.err.println("Usage: DriverWrapper <workerUrl> <driverMainClass> [options]")
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index fdc9a34886eb8037f98e8e155d19fd663641fa0a..18885d7ca6daa2ea4124c9e838fc631da7a8be5b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -98,6 +98,12 @@ private[spark] class ExecutorRunner(
     case other => other
   }
 
+  def getCommandSeq = {
+    val command = Command(appDesc.command.mainClass,
+      appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment)
+    CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath)
+  }
+
   /**
    * Download and run the executor described in our ApplicationDescription
    */
@@ -110,9 +116,7 @@ private[spark] class ExecutorRunner(
       }
 
       // Launch the process
-      val fullCommand = new Command(appDesc.command.mainClass,
-        appDesc.command.arguments.map(substituteVariables), appDesc.command.environment)
-      val command = CommandUtils.buildCommandSeq(fullCommand, memory, sparkHome.getAbsolutePath)
+      val command = getCommandSeq
       logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
       val builder = new ProcessBuilder(command: _*).directory(executorDir)
       val env = builder.environment()
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 4e23e0d1eb567ddca555b0cbbd3d54c7c9cb9a06..273bacded6fb4fdf28b33af63054314cbc7ba0a4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -27,10 +27,11 @@ import scala.concurrent.duration._
 import akka.actor._
 import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
 
-import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.{Logging, SparkConf, SparkException}
 import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.{DriverState, Master}
+import org.apache.spark.deploy.master.DriverState.DriverState
 import org.apache.spark.deploy.worker.ui.WorkerWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{AkkaUtils, Utils}
@@ -47,7 +48,8 @@ private[spark] class Worker(
     masterUrls: Array[String],
     actorSystemName: String,
     actorName: String,
-    workDirPath: String = null)
+    workDirPath: String = null,
+    val conf: SparkConf)
   extends Actor with Logging {
   import context.dispatcher
 
@@ -57,7 +59,7 @@ private[spark] class Worker(
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For worker and executor IDs
 
   // Send a heartbeat every (heartbeat timeout) / 4 milliseconds
-  val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4
+  val HEARTBEAT_MILLIS = conf.get("spark.worker.timeout", "60").toLong * 1000 / 4
 
   val REGISTRATION_TIMEOUT = 20.seconds
   val REGISTRATION_RETRIES = 3
@@ -90,7 +92,7 @@ private[spark] class Worker(
   var coresUsed = 0
   var memoryUsed = 0
 
-  val metricsSystem = MetricsSystem.createMetricsSystem("worker")
+  val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf)
   val workerSource = new WorkerSource(this)
 
   def coresFree: Int = cores - coresUsed
@@ -260,8 +262,8 @@ private[spark] class Worker(
 
     case KillDriver(driverId) => {
       logInfo(s"Asked to kill driver $driverId")
-      drivers.find(_._1 == driverId) match {
-        case Some((id, runner)) =>
+      drivers.get(driverId) match {
+        case Some(runner) =>
           runner.kill()
         case None =>
           logError(s"Asked to kill unknown driver $driverId")
@@ -270,7 +272,7 @@ private[spark] class Worker(
 
     case DriverStateChanged(driverId, state, exception) => {
       state match {
-        case DriverState.FAILED =>
+        case DriverState.ERROR =>
           logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}")
         case DriverState.FINISHED =>
           logInfo(s"Driver $driverId exited successfully")
@@ -280,8 +282,7 @@ private[spark] class Worker(
       masterLock.synchronized {
         master ! DriverStateChanged(driverId, state, exception)
       }
-      val driver = drivers(driverId)
-      drivers -= driverId
+      val driver = drivers.remove(driverId).get
       finishedDrivers(driverId) = driver
       memoryUsed -= driver.driverDesc.mem
       coresUsed -= driver.driverDesc.cores
@@ -316,6 +317,7 @@ private[spark] class Worker(
 }
 
 private[spark] object Worker {
+
   def main(argStrings: Array[String]) {
     val args = new WorkerArguments(argStrings)
     val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
@@ -324,14 +326,17 @@ private[spark] object Worker {
   }
 
   def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
-    masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
-    : (ActorSystem, Int) = {
+      masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
+      : (ActorSystem, Int) =
+  {
     // The LocalSparkCluster runs multiple local sparkWorkerX actor systems
+    val conf = new SparkConf
     val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
     val actorName = "Worker"
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
+      conf = conf)
     actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
-      masterUrls, systemName, actorName, workDir), name = actorName)
+      masterUrls, systemName, actorName,  workDir, conf), name = actorName)
     (actorSystem, boundPort)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
index f4184bc5db17b288e9c4f674adb6dd613971a2c2..0e0d0cd6264cfdf7bd7cdaf03f20bfb126acc92e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
@@ -10,7 +10,8 @@ import org.apache.spark.deploy.DeployMessages.SendHeartbeat
  * Actor which connects to a worker process and terminates the JVM if the connection is severed.
  * Provides fate sharing between a worker and its associated child processes.
  */
-private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging {
+private[spark] class WorkerWatcher(workerUrl: String) extends Actor
+    with Logging {
   override def preStart() {
     context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
 
@@ -19,10 +20,17 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging
     worker ! SendHeartbeat // need to send a message here to initiate connection
   }
 
+  // Used to avoid shutting down JVM during tests
+  private[deploy] var isShutDown = false
+  private[deploy] def setTesting(testing: Boolean) = isTesting = testing
+  private var isTesting = false
+
   // Lets us filter events only from the worker's actor system
   private val expectedHostPort = AddressFromURIString(workerUrl).hostPort
   private def isWorker(address: Address) = address.hostPort == expectedHostPort
 
+  def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1)
+
   override def receive = {
     case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
       logInfo(s"Successfully connected to $workerUrl")
@@ -32,12 +40,12 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging
       // These logs may not be seen if the worker (and associated pipe) has died
       logError(s"Could not initialize connection to worker $workerUrl. Exiting.")
       logError(s"Error was: $cause")
-      System.exit(-1)
+      exitNonZero()
 
     case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
       // This log message will never be seen
       logError(s"Lost connection to worker actor $workerUrl. Exiting.")
-      System.exit(-1)
+      exitNonZero()
 
     case e: AssociationEvent =>
       // pass through association events relating to other remote actor systems
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 93c6ad49d7a123f43ec2e3a74c379fddc5fdd504..925c6fb1832d7eff6adea8d51c91baf937ef0a97 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -18,7 +18,6 @@
 package org.apache.spark.deploy.worker.ui
 
 import scala.concurrent.Await
-import scala.concurrent.duration._
 import scala.xml.Node
 
 import akka.pattern.ask
@@ -27,6 +26,7 @@ import net.liftweb.json.JsonAST.JValue
 
 import org.apache.spark.deploy.JsonProtocol
 import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
+import org.apache.spark.deploy.master.DriverState
 import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
@@ -52,12 +52,16 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
     val finishedExecutorTable =
       UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors)
 
-    val driverHeaders = Seq("DriverID", "Main Class", "Cores", "Memory", "Logs")
+    val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes")
     val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse
     val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers)
     val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse
     def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers)
 
+    // For now we only show driver information if the user has submitted drivers to the cluster.
+    // This is until we integrate the notion of drivers and applications in the UI.
+    def hasDrivers = runningDrivers.length > 0 || finishedDrivers.length > 0
+
     val content =
         <div class="row-fluid"> <!-- Worker Details -->
           <div class="span12">
@@ -81,6 +85,17 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
           </div>
         </div>
 
+        <div>
+          {if (hasDrivers)
+            <div class="row-fluid"> <!-- Running Drivers -->
+              <div class="span12">
+                <h4> Running Drivers {workerState.drivers.size} </h4>
+                {runningDriverTable}
+              </div>
+            </div>
+          }
+        </div>
+
         <div class="row-fluid"> <!-- Finished Executors  -->
           <div class="span12">
             <h4> Finished Executors </h4>
@@ -88,18 +103,15 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
           </div>
         </div>
 
-        <div class="row-fluid"> <!-- Running Drivers -->
-          <div class="span12">
-            <h4> Running Drivers {workerState.drivers.size} </h4>
-            {runningDriverTable}
-          </div>
-        </div>
-
-        <div class="row-fluid"> <!-- Finished Drivers  -->
-          <div class="span12">
-            <h4> Finished Drivers </h4>
-            {finishedDriverTable}
-          </div>
+        <div>
+          {if (hasDrivers)
+            <div class="row-fluid"> <!-- Finished Drivers  -->
+              <div class="span12">
+                <h4> Finished Drivers </h4>
+                {finishedDriverTable}
+              </div>
+            </div>
+          }
         </div>;
 
     UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format(
@@ -133,7 +145,8 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
   def driverRow(driver: DriverRunner): Seq[Node] = {
     <tr>
       <td>{driver.driverId}</td>
-      <td>{driver.driverDesc.command.mainClass}</td>
+      <td>{driver.driverDesc.command.arguments(1)}</td>
+      <td>{driver.finalState.getOrElse(DriverState.RUNNING)}</td>
       <td sorttable_customkey={driver.driverDesc.cores.toString}>
         {driver.driverDesc.cores.toString}
       </td>
@@ -144,6 +157,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
         <a href={s"logPage?driverId=${driver.driverId}&logType=stdout"}>stdout</a>
         <a href={s"logPage?driverId=${driver.driverId}&logType=stderr"}>stderr</a>
       </td>
+      <td>
+        {driver.finalException.getOrElse("")}
+      </td>
     </tr>
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 2fd862c4c41d78047f172d4341d19911490083e6..8daa47b2b24352ab6925175f19db6bfa014d332e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -22,7 +22,7 @@ import java.io.File
 import javax.servlet.http.HttpServletRequest
 import org.eclipse.jetty.server.{Handler, Server}
 
-import org.apache.spark.Logging
+import org.apache.spark.{Logging, SparkConf}
 import org.apache.spark.deploy.worker.Worker
 import org.apache.spark.ui.{JettyUtils, UIUtils}
 import org.apache.spark.ui.JettyUtils._
@@ -34,10 +34,10 @@ import org.apache.spark.util.{AkkaUtils, Utils}
 private[spark]
 class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None)
   extends Logging {
-  val timeout = AkkaUtils.askTimeout
+  val timeout = AkkaUtils.askTimeout(worker.conf)
   val host = Utils.localHostName()
   val port = requestedPort.getOrElse(
-    System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt)
+    worker.conf.get("worker.ui.port",  WorkerWebUI.DEFAULT_PORT).toInt)
 
   var server: Option[Server] = None
   var boundPort: Option[Int] = None
@@ -157,12 +157,12 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
         <body>
           {linkToMaster}
           <div>
-            <div style="float:left;width:40%">{backButton}</div>
+            <div style="float:left; margin-right:10px">{backButton}</div>
             <div style="float:left;">{range}</div>
-            <div style="float:right;">{nextButton}</div>
+            <div style="float:right; margin-left:10px">{nextButton}</div>
           </div>
           <br />
-          <div style="height:500px;overflow:auto;padding:5px;">
+          <div style="height:500px; overflow:auto; padding:5px;">
             <pre>{logText}</pre>
           </div>
         </body>
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index eb1199ed5726374bcb3db5b8e293244adaf9b752..f9e43e0e94404b0039532bb922b1a569ed5c9bb9 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer
 import akka.actor._
 import akka.remote._
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.TaskState.TaskState
 import org.apache.spark.deploy.worker.WorkerWatcher
 import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
@@ -100,10 +100,10 @@ private[spark] object CoarseGrainedExecutorBackend {
     // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor
     // before getting started with all our system properties, etc
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0,
-      indestructible = true)
+      indestructible = true, conf = new SparkConf)
     // set it
     val sparkHostPort = hostname + ":" + boundPort
-    System.setProperty("spark.hostPort", sparkHostPort)
+//    conf.set("spark.hostPort",  sparkHostPort)
     actorSystem.actorOf(
       Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
       name = "Executor")
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index 0f19d7a96b0858a21b2aab65480f7616680543e2..e51d274d338748cd48a162d39458326012b921cb 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -48,8 +48,6 @@ private[spark] class Executor(
 
   private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0))
 
-  initLogging()
-
   // No ip or host:port - just hostname
   Utils.checkHost(slaveHostname, "Expected executed slave to be a hostname")
   // must not have port specified.
@@ -58,16 +56,17 @@ private[spark] class Executor(
   // Make sure the local hostname we report matches the cluster scheduler's name for this host
   Utils.setCustomHostname(slaveHostname)
 
-  // Set spark.* system properties from executor arg
-  for ((key, value) <- properties) {
-    System.setProperty(key, value)
-  }
+  // Set spark.* properties from executor arg
+  val conf = new SparkConf(false)
+  conf.setAll(properties)
 
   // If we are in yarn mode, systems can have different disk layouts so we must set it
   // to what Yarn on this system said was available. This will be used later when SparkEnv
   // created.
-  if (java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))) {
-    System.setProperty("spark.local.dir", getYarnLocalDirs())
+  if (java.lang.Boolean.valueOf(
+      System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))))
+  {
+    conf.set("spark.local.dir", getYarnLocalDirs())
   }
 
   // Create our ClassLoader and set it on this thread
@@ -108,7 +107,7 @@ private[spark] class Executor(
   // Initialize Spark environment (using system properties read above)
   private val env = {
     if (!isLocal) {
-      val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0,
+      val _env = SparkEnv.create(conf, executorId, slaveHostname, 0,
         isDriver = false, isLocal = false)
       SparkEnv.set(_env)
       _env.metricsSystem.registerSource(executorSource)
@@ -142,11 +141,6 @@ private[spark] class Executor(
     val tr = runningTasks.get(taskId)
     if (tr != null) {
       tr.kill()
-      // We remove the task also in the finally block in TaskRunner.run.
-      // The reason we need to remove it here is because killTask might be called before the task
-      // is even launched, and never reaching that finally block. ConcurrentHashMap's remove is
-      // idempotent.
-      runningTasks.remove(taskId)
     }
   }
 
@@ -168,6 +162,8 @@ private[spark] class Executor(
   class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer)
     extends Runnable {
 
+    object TaskKilledException extends Exception
+
     @volatile private var killed = false
     @volatile private var task: Task[Any] = _
 
@@ -201,9 +197,11 @@ private[spark] class Executor(
         // If this task has been killed before we deserialized it, let's quit now. Otherwise,
         // continue executing the task.
         if (killed) {
-          logInfo("Executor killed task " + taskId)
-          execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
-          return
+          // Throw an exception rather than returning, because returning within a try{} block
+          // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl
+          // exception will be caught by the catch block, leading to an incorrect ExceptionFailure
+          // for the task.
+          throw TaskKilledException
         }
 
         attemptedTask = Some(task)
@@ -217,9 +215,7 @@ private[spark] class Executor(
 
         // If the task has been killed, let's fail it.
         if (task.killed) {
-          logInfo("Executor killed task " + taskId)
-          execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
-          return
+          throw TaskKilledException
         }
 
         val resultSer = SparkEnv.get.serializer.newInstance()
@@ -261,6 +257,11 @@ private[spark] class Executor(
           execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
         }
 
+        case TaskKilledException => {
+          logInfo("Executor killed task " + taskId)
+          execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
+        }
+
         case t: Throwable => {
           val serviceTime = (System.currentTimeMillis() - taskStart).toInt
           val metrics = attemptedTask.flatMap(t => t.metrics)
@@ -303,7 +304,7 @@ private[spark] class Executor(
    * new classes defined by the REPL as the user types code
    */
   private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = {
-    val classUri = System.getProperty("spark.repl.class.uri")
+    val classUri = conf.get("spark.repl.class.uri", null)
     if (classUri != null) {
       logInfo("Using REPL class URI: " + classUri)
       try {
@@ -331,12 +332,12 @@ private[spark] class Executor(
       // Fetch missing dependencies
       for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
         logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf)
         currentFiles(name) = timestamp
       }
       for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
         logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf)
         currentJars(name) = timestamp
         // Add it to our class loader
         val localName = name.split("/").last
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 570a979b56879effacd4d93ccdccc3f52f076e78..a1e98845f6a848b8ab0651aab5e3b0d484d32a02 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -22,6 +22,7 @@ import java.io.{InputStream, OutputStream}
 import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
 
 import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream}
+import org.apache.spark.{SparkEnv, SparkConf}
 
 
 /**
@@ -37,15 +38,15 @@ trait CompressionCodec {
 
 
 private[spark] object CompressionCodec {
-
-  def createCodec(): CompressionCodec = {
-    createCodec(System.getProperty(
+  def createCodec(conf: SparkConf): CompressionCodec = {
+    createCodec(conf, conf.get(
       "spark.io.compression.codec", classOf[LZFCompressionCodec].getName))
   }
 
-  def createCodec(codecName: String): CompressionCodec = {
-    Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
-      .newInstance().asInstanceOf[CompressionCodec]
+  def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
+    val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
+      .getConstructor(classOf[SparkConf])
+    ctor.newInstance(conf).asInstanceOf[CompressionCodec]
   }
 }
 
@@ -53,7 +54,7 @@ private[spark] object CompressionCodec {
 /**
  * LZF implementation of [[org.apache.spark.io.CompressionCodec]].
  */
-class LZFCompressionCodec extends CompressionCodec {
+class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
 
   override def compressedOutputStream(s: OutputStream): OutputStream = {
     new LZFOutputStream(s).setFinishBlockOnFlush(true)
@@ -67,10 +68,10 @@ class LZFCompressionCodec extends CompressionCodec {
  * Snappy implementation of [[org.apache.spark.io.CompressionCodec]].
  * Block size can be configured by spark.io.compression.snappy.block.size.
  */
-class SnappyCompressionCodec extends CompressionCodec {
+class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
 
   override def compressedOutputStream(s: OutputStream): OutputStream = {
-    val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt
+    val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt
     new SnappyOutputStream(s, blockSize)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
index caab748d602ee452703091c9f0663ed46a7328f5..6f9f29969eaec85d315c964092c6c815516e75e8 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
@@ -26,7 +26,6 @@ import scala.util.matching.Regex
 import org.apache.spark.Logging
 
 private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging {
-  initLogging()
 
   val DEFAULT_PREFIX = "*"
   val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index bec0c83be8bea24c4c69bc14ad07c72cbf685329..9930537b34db01ac0e619cdf482bbf4bd9ab7fdb 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit
 
 import scala.collection.mutable
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 import org.apache.spark.metrics.sink.{MetricsServlet, Sink}
 import org.apache.spark.metrics.source.Source
 
@@ -62,10 +62,10 @@ import org.apache.spark.metrics.source.Source
  *
  * [options] is the specific property of this source or sink.
  */
-private[spark] class MetricsSystem private (val instance: String) extends Logging {
-  initLogging()
+private[spark] class MetricsSystem private (val instance: String,
+    conf: SparkConf) extends Logging {
 
-  val confFile = System.getProperty("spark.metrics.conf")
+  val confFile = conf.get("spark.metrics.conf", null)
   val metricsConfig = new MetricsConfig(Option(confFile))
 
   val sinks = new mutable.ArrayBuffer[Sink]
@@ -159,5 +159,6 @@ private[spark] object MetricsSystem {
     }
   }
 
-  def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance)
+  def createMetricsSystem(instance: String, conf: SparkConf): MetricsSystem =
+    new MetricsSystem(instance, conf)
 }
diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
index 703bc6a9ca9fa4a539644c6350fefc775808478a..46c40d0a2a02959ffb50ea7541c7092ac0a9bab2 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
@@ -37,7 +37,7 @@ import scala.concurrent.duration._
 
 import org.apache.spark.util.Utils
 
-private[spark] class ConnectionManager(port: Int) extends Logging {
+private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Logging {
 
   class MessageStatus(
       val message: Message,
@@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
   private val selector = SelectorProvider.provider.openSelector()
 
   private val handleMessageExecutor = new ThreadPoolExecutor(
-    System.getProperty("spark.core.connection.handler.threads.min","20").toInt,
-    System.getProperty("spark.core.connection.handler.threads.max","60").toInt,
-    System.getProperty("spark.core.connection.handler.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+    conf.get("spark.core.connection.handler.threads.min", "20").toInt,
+    conf.get("spark.core.connection.handler.threads.max", "60").toInt,
+    conf.get("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
     new LinkedBlockingDeque[Runnable]())
 
   private val handleReadWriteExecutor = new ThreadPoolExecutor(
-    System.getProperty("spark.core.connection.io.threads.min","4").toInt,
-    System.getProperty("spark.core.connection.io.threads.max","32").toInt,
-    System.getProperty("spark.core.connection.io.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+    conf.get("spark.core.connection.io.threads.min", "4").toInt,
+    conf.get("spark.core.connection.io.threads.max", "32").toInt,
+    conf.get("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
     new LinkedBlockingDeque[Runnable]())
 
   // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap
   private val handleConnectExecutor = new ThreadPoolExecutor(
-    System.getProperty("spark.core.connection.connect.threads.min","1").toInt,
-    System.getProperty("spark.core.connection.connect.threads.max","8").toInt,
-    System.getProperty("spark.core.connection.connect.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+    conf.get("spark.core.connection.connect.threads.min", "1").toInt,
+    conf.get("spark.core.connection.connect.threads.max", "8").toInt,
+    conf.get("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
     new LinkedBlockingDeque[Runnable]())
 
   private val serverChannel = ServerSocketChannel.open()
@@ -594,7 +594,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
 private[spark] object ConnectionManager {
 
   def main(args: Array[String]) {
-    val manager = new ConnectionManager(9999)
+    val manager = new ConnectionManager(9999, new SparkConf)
     manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
       println("Received [" + msg + "] from [" + id + "]")
       None
diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
index 781715108be6361197eed6e946d3a6664c5161b7..1c9d6030d68d76f886ad212e34c864b2b244d5b2 100644
--- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
@@ -19,19 +19,19 @@ package org.apache.spark.network
 
 import java.nio.ByteBuffer
 import java.net.InetAddress
+import org.apache.spark.SparkConf
 
 private[spark] object ReceiverTest {
-
   def main(args: Array[String]) {
-    val manager = new ConnectionManager(9999)
+    val manager = new ConnectionManager(9999, new SparkConf)
     println("Started connection manager with id = " + manager.id)
-    
-    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { 
+
+    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
       /*println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/
-      val buffer = ByteBuffer.wrap("response".getBytes())
+      val buffer = ByteBuffer.wrap("response".getBytes)
       Some(Message.createBufferMessage(buffer, msg.id))
     })
-    Thread.currentThread.join()  
+    Thread.currentThread.join()
   }
 }
 
diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala
index 777574980fbc63a82ebc51b60ab02fa7a37c5c6e..dcbd183c88d09b4aad12b85674207adf8fce60ac 100644
--- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala
@@ -19,29 +19,29 @@ package org.apache.spark.network
 
 import java.nio.ByteBuffer
 import java.net.InetAddress
+import org.apache.spark.SparkConf
 
 private[spark] object SenderTest {
-
   def main(args: Array[String]) {
-    
+
     if (args.length < 2) {
       println("Usage: SenderTest <target host> <target port>")
       System.exit(1)
     }
-   
+
     val targetHost = args(0)
     val targetPort = args(1).toInt
     val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort)
 
-    val manager = new ConnectionManager(0)
+    val manager = new ConnectionManager(0, new SparkConf)
     println("Started connection manager with id = " + manager.id)
 
-    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { 
+    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
       println("Received [" + msg + "] from [" + id + "]")
       None
     })
-  
-    val size =  100 * 1024  * 1024 
+
+    val size =  100 * 1024  * 1024
     val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte))
     buffer.flip
 
@@ -50,7 +50,7 @@ private[spark] object SenderTest {
     val count = 100
     (0 until count).foreach(i => {
       val dataMessage = Message.createBufferMessage(buffer.duplicate)
-      val startTime = System.currentTimeMillis  
+      val startTime = System.currentTimeMillis
       /*println("Started timer at " + startTime)*/
       val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match {
         case Some(response) =>
diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
index b1e1576dadc1a88c9b6de8949b0d41255c2a4055..b729eb11c514268798ecd21a679b6abbb3202e90 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
@@ -23,20 +23,20 @@ import io.netty.buffer.ByteBuf
 import io.netty.channel.ChannelHandlerContext
 import io.netty.util.CharsetUtil
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkContext, SparkConf, Logging}
 import org.apache.spark.network.ConnectionManagerId
 
 import scala.collection.JavaConverters._
 import org.apache.spark.storage.BlockId
 
 
-private[spark] class ShuffleCopier extends Logging {
+private[spark] class ShuffleCopier(conf: SparkConf) extends Logging {
 
   def getBlock(host: String, port: Int, blockId: BlockId,
       resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
 
     val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback)
-    val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt
+    val connectTimeout = conf.get("spark.shuffle.netty.connect.timeout", "60000").toInt
     val fc = new FileClient(handler, connectTimeout)
 
     try {
@@ -104,10 +104,10 @@ private[spark] object ShuffleCopier extends Logging {
     val threads = if (args.length > 3) args(3).toInt else 10
 
     val copiers = Executors.newFixedThreadPool(80)
-    val tasks = (for (i <- Range(0, threads)) yield { 
+    val tasks = (for (i <- Range(0, threads)) yield {
       Executors.callable(new Runnable() {
         def run() {
-          val copier = new ShuffleCopier()
+          val copier = new ShuffleCopier(new SparkConf)
           copier.getBlock(host, port, blockId, echoResultCollectCallBack)
         }
       })
diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
index 293a7d1f6841552c1570ead689bcf5e22a65f6a3..6d4f46125f1a60bf872dfe27baad4cf1d07070ad 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -76,7 +76,6 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String)
 }
 
 private[spark] object CheckpointRDD extends Logging {
-
   def splitIdToFile(splitId: Int): String = {
     "part-%05d".format(splitId)
   }
@@ -98,7 +97,7 @@ private[spark] object CheckpointRDD extends Logging {
       throw new IOException("Checkpoint failed: temporary path " +
         tempOutputPath + " already exists")
     }
-    val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+    val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
 
     val fileOutputStream = if (blockSize < 0) {
       fs.create(tempOutputPath, false, bufferSize)
@@ -132,7 +131,7 @@ private[spark] object CheckpointRDD extends Logging {
     ): Iterator[T] = {
     val env = SparkEnv.get
     val fs = path.getFileSystem(broadcastedConf.value.value)
-    val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+    val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
     val fileInputStream = fs.open(path, bufferSize)
     val serializer = env.serializer.newInstance()
     val deserializeStream = serializer.deserializeStream(fileInputStream)
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 911a002884d65db61efb9ebc040ec6324e66ee3e..4ba4696fef52159c8fd6c7141e8cce1ac5fd3056 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -114,7 +114,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
       map.changeValue(k, update)
     }
 
-    val ser = SparkEnv.get.serializerManager.get(serializerClass)
+    val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)
     for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
       case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
         // Read them from the parent
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 48168e152e954ceb548caf3f04fb672c23755479..04a8d05988f168a2a294c939aa70e9c8ef78824c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -40,12 +40,15 @@ import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil
 import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob}
 import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter}
 
+import com.clearspring.analytics.stream.cardinality.HyperLogLog
+
 import org.apache.spark._
 import org.apache.spark.SparkContext._
 import org.apache.spark.partial.{BoundedDouble, PartialResult}
 import org.apache.spark.Aggregator
 import org.apache.spark.Partitioner
 import org.apache.spark.Partitioner.defaultPartitioner
+import org.apache.spark.util.SerializableHyperLogLog
 
 /**
  * Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
@@ -207,6 +210,45 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
     self.map(_._1).countByValueApprox(timeout, confidence)
   }
 
+  /**
+   * Return approximate number of distinct values for each key in this RDD.
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. Uses the provided
+   * Partitioner to partition the output RDD.
+   */
+  def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = {
+    val createHLL = (v: V) => new SerializableHyperLogLog(new HyperLogLog(relativeSD)).add(v)
+    val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => hll.add(v)
+    val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2)
+
+    combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.value.cardinality())
+  }
+
+  /**
+   * Return approximate number of distinct values for each key in this RDD. 
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. HashPartitions the
+   * output RDD into numPartitions.
+   *
+   */
+  def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = {
+    countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions))
+  }
+
+  /**
+   * Return approximate number of distinct values for each key this RDD.
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. The default value of
+   * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism
+   * level.
+   */
+  def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = {
+    countApproxDistinctByKey(relativeSD, defaultPartitioner(self))
+  }
+
   /**
    * Merge the values for each key using an associative reduce function. This will also perform
    * the merging locally on each mapper before sending results to a reducer, similarly to a
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..4c625d062eb9bef5abaf71a180af13cafaf7f18c
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala
@@ -0,0 +1,110 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.rdd
+
+import scala.reflect.ClassTag
+import java.io.{ObjectOutputStream, IOException}
+import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition}
+
+
+/**
+ * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of corresponding partitions
+ * of parent RDDs.
+ */
+private[spark]
+class PartitionerAwareUnionRDDPartition(
+    @transient val rdds: Seq[RDD[_]],
+    val idx: Int
+  ) extends Partition {
+  var parents = rdds.map(_.partitions(idx)).toArray
+  
+  override val index = idx
+  override def hashCode(): Int = idx
+
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    // Update the reference to parent partition at the time of task serialization
+    parents = rdds.map(_.partitions(index)).toArray
+    oos.defaultWriteObject()
+  }
+}
+
+/**
+ * Class representing an RDD that can take multiple RDDs partitioned by the same partitioner and
+ * unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each
+ * will be unified to a single RDD with p partitions and the same partitioner. The preferred
+ * location for each partition of the unified RDD will be the most common preferred location
+ * of the corresponding partitions of the parent RDDs. For example, location of partition 0
+ * of the unified RDD will be where most of partition 0 of the parent RDDs are located.
+ */
+private[spark]
+class PartitionerAwareUnionRDD[T: ClassTag](
+    sc: SparkContext,
+    var rdds: Seq[RDD[T]]
+  ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) {
+  require(rdds.length > 0)
+  require(rdds.flatMap(_.partitioner).toSet.size == 1,
+    "Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner))
+
+  override val partitioner = rdds.head.partitioner
+
+  override def getPartitions: Array[Partition] = {
+    val numPartitions = partitioner.get.numPartitions
+    (0 until numPartitions).map(index => {
+      new PartitionerAwareUnionRDDPartition(rdds, index)
+    }).toArray
+  }
+
+  // Get the location where most of the partitions of parent RDDs are located
+  override def getPreferredLocations(s: Partition): Seq[String] = {
+    logDebug("Finding preferred location for " + this + ", partition " + s.index)
+    val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents
+    val locations = rdds.zip(parentPartitions).flatMap {
+      case (rdd, part) => {
+        val parentLocations = currPrefLocs(rdd, part)
+        logDebug("Location of " + rdd + " partition " + part.index + " = " + parentLocations)
+        parentLocations
+      }
+    }
+    val location = if (locations.isEmpty) {
+      None
+    } else  {
+      // Find the location that maximum number of parent partitions prefer
+      Some(locations.groupBy(x => x).maxBy(_._2.length)._1)
+    }
+    logDebug("Selected location for " + this + ", partition " + s.index + " = " + location)
+    location.toSeq
+  }
+
+  override def compute(s: Partition, context: TaskContext): Iterator[T] = {
+    val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents
+    rdds.zip(parentPartitions).iterator.flatMap {
+      case (rdd, p) => rdd.iterator(p, context)
+    }
+  }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdds = null
+  }
+
+  // Get the *current* preferred locations from the DAGScheduler (as opposed to the static ones)
+  private def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = {
+    rdd.context.getPreferredLocs(rdd, part.index).map(tl => tl.host)
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index ea45566ad1da96610f13d3250e6c66a5630040c5..3f41b662799873694958b2b699563c0627cab499 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -33,6 +33,7 @@ import org.apache.hadoop.io.Text
 import org.apache.hadoop.mapred.TextOutputFormat
 
 import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
+import com.clearspring.analytics.stream.cardinality.HyperLogLog
 
 import org.apache.spark.Partitioner._
 import org.apache.spark.api.java.JavaRDD
@@ -41,7 +42,7 @@ import org.apache.spark.partial.CountEvaluator
 import org.apache.spark.partial.GroupedCountEvaluator
 import org.apache.spark.partial.PartialResult
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.{Utils, BoundedPriorityQueue}
+import org.apache.spark.util.{Utils, BoundedPriorityQueue, SerializableHyperLogLog}
 
 import org.apache.spark.SparkContext._
 import org.apache.spark._
@@ -81,6 +82,7 @@ abstract class RDD[T: ClassTag](
   def this(@transient oneParent: RDD[_]) =
     this(oneParent.context , List(new OneToOneDependency(oneParent)))
 
+  private[spark] def conf = sc.conf
   // =======================================================================
   // Methods that should be implemented by subclasses of RDD
   // =======================================================================
@@ -788,6 +790,19 @@ abstract class RDD[T: ClassTag](
     sc.runApproximateJob(this, countPartition, evaluator, timeout)
   }
 
+  /**
+   * Return approximate number of distinct elements in the RDD.
+   *
+   * The accuracy of approximation can be controlled through the relative standard deviation
+   * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+   * more accurate counts but increase the memory footprint and vise versa. The default value of
+   * relativeSD is 0.05.
+   */
+  def countApproxDistinct(relativeSD: Double = 0.05): Long = {
+    val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD))
+    aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality()
+  }
+
   /**
    * Take the first num elements of the RDD. It works by first scanning one partition, and use the
    * results from that partition to estimate the number of additional partitions needed to satisfy
@@ -938,7 +953,7 @@ abstract class RDD[T: ClassTag](
   private var storageLevel: StorageLevel = StorageLevel.NONE
 
   /** Record user function generating this RDD. */
-  @transient private[spark] val origin = Utils.formatSparkCallSite
+  @transient private[spark] val origin = sc.getCallSite
 
   private[spark] def elementClassTag: ClassTag[T] = classTag[T]
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
index 642dabaad560e1d2a2f68f243e721530d5882efb..bc688110f47362cef8cdb5bf9e92d3e71b22f9c9 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
@@ -40,7 +40,7 @@ private[spark] object CheckpointState extends Enumeration {
  * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations
  * of the checkpointed RDD.
  */
-private[spark] class RDDCheckpointData[T: ClassTag](rdd: RDD[T])
+private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T])
   extends Logging with Serializable {
 
   import CheckpointState._
diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index 3682c84598e730a659f6fd1c639f53afb7403892..0ccb309d0d9d56a8fd5a7f762632ca218b488b3d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -59,7 +59,7 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag](
   override def compute(split: Partition, context: TaskContext): Iterator[P] = {
     val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
     SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context,
-      SparkEnv.get.serializerManager.get(serializerClass))
+      SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf))
   }
 
   override def clearDependencies() {
diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
index aab30b1bb49048ee90a513adece417e28ed64372..4f90c7d3d68ab0dbd55c335756e861a92ae16170 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -93,7 +93,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag](
 
   override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = {
     val partition = p.asInstanceOf[CoGroupPartition]
-    val serializer = SparkEnv.get.serializerManager.get(serializerClass)
+    val serializer = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)
     val map = new JHashMap[K, ArrayBuffer[V]]
     def getSeq(k: K): ArrayBuffer[V] = {
       val seq = map.get(k)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 963d15b76d4c4d7495eed8e7fd01d86588c67334..043e01dbfbf28d930eb08ad6e03a12be5ab5a0f2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -152,13 +152,15 @@ class DAGScheduler(
   val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done
   val running = new HashSet[Stage] // Stages we are running right now
   val failed = new HashSet[Stage]  // Stages that must be resubmitted due to fetch failures
-  val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
+  // Missing tasks from each stage
+  val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]]
   var lastFetchFailureTime: Long = 0  // Used to wait a bit to avoid repeated resubmits
 
   val activeJobs = new HashSet[ActiveJob]
   val resultStageToJob = new HashMap[Stage, ActiveJob]
 
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup)
+  val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf)
 
   /**
    * Starts the event processing actor.  The actor has two responsibilities:
@@ -239,7 +241,8 @@ class DAGScheduler(
     shuffleToMapStage.get(shuffleDep.shuffleId) match {
       case Some(stage) => stage
       case None =>
-        val stage = newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId)
+        val stage =
+          newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId)
         shuffleToMapStage(shuffleDep.shuffleId) = stage
         stage
     }
@@ -248,7 +251,8 @@ class DAGScheduler(
   /**
    * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation
    * of a shuffle map stage in newOrUsedStage.  The stage will be associated with the provided
-   * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage directly.
+   * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage
+   * directly.
    */
   private def newStage(
       rdd: RDD[_],
@@ -358,7 +362,8 @@ class DAGScheduler(
         stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId
         jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id
         val parents = getParentStages(s.rdd, jobId)
-        val parentsWithoutThisJobId = parents.filter(p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId)))
+        val parentsWithoutThisJobId = parents.filter(p =>
+          !stageIdToJobIds.get(p.id).exists(_.contains(jobId)))
         updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail)
       }
     }
@@ -366,8 +371,9 @@ class DAGScheduler(
   }
 
   /**
-   * Removes job and any stages that are not needed by any other job.  Returns the set of ids for stages that
-   * were removed.  The associated tasks for those stages need to be cancelled if we got here via job cancellation.
+   * Removes job and any stages that are not needed by any other job.  Returns the set of ids for
+   * stages that were removed.  The associated tasks for those stages need to be cancelled if we
+   * got here via job cancellation.
    */
   private def removeJobAndIndependentStages(jobId: Int): Set[Int] = {
     val registeredStages = jobIdToStageIds(jobId)
@@ -378,7 +384,8 @@ class DAGScheduler(
       stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach {
         case (stageId, jobSet) =>
           if (!jobSet.contains(jobId)) {
-            logError("Job %d not registered for stage %d even though that stage was registered for the job"
+            logError(
+              "Job %d not registered for stage %d even though that stage was registered for the job"
               .format(jobId, stageId))
           } else {
             def removeStage(stageId: Int) {
@@ -389,7 +396,8 @@ class DAGScheduler(
                   running -= s
                 }
                 stageToInfos -= s
-                shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove)
+                shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleId =>
+                  shuffleToMapStage.remove(shuffleId))
                 if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) {
                   logDebug("Removing pending status for stage %d".format(stageId))
                 }
@@ -407,7 +415,8 @@ class DAGScheduler(
               stageIdToStage -= stageId
               stageIdToJobIds -= stageId
 
-              logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size))
+              logDebug("After removal of stage %d, remaining stages = %d"
+                .format(stageId, stageIdToStage.size))
             }
 
             jobSet -= jobId
@@ -459,7 +468,8 @@ class DAGScheduler(
     assert(partitions.size > 0)
     val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
     val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
-    eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)
+    eventProcessActor ! JobSubmitted(
+      jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)
     waiter
   }
 
@@ -494,7 +504,8 @@ class DAGScheduler(
     val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
     val partitions = (0 until rdd.partitions.size).toArray
     val jobId = nextJobId.getAndIncrement()
-    eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties)
+    eventProcessActor ! JobSubmitted(
+      jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties)
     listener.awaitResult()    // Will throw an exception if the job fails
   }
 
@@ -529,8 +540,8 @@ class DAGScheduler(
       case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
         var finalStage: Stage = null
         try {
-          // New stage creation at times and if its not protected, the scheduler thread is killed. 
-          // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted
+          // New stage creation may throw an exception if, for example, jobs are run on a HadoopRDD
+          // whose underlying HDFS files have been deleted.
           finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite))
         } catch {
           case e: Exception =>
@@ -563,7 +574,8 @@ class DAGScheduler(
       case JobGroupCancelled(groupId) =>
         // Cancel all jobs belonging to this job group.
         // First finds all active jobs with this group id, and then kill stages for them.
-        val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID))
+        val activeInGroup = activeJobs.filter(activeJob =>
+          groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID))
         val jobIds = activeInGroup.map(_.jobId)
         jobIds.foreach { handleJobCancellation }
 
@@ -585,7 +597,8 @@ class DAGScheduler(
           stage <- stageIdToStage.get(task.stageId);
           stageInfo <- stageToInfos.get(stage)
         ) {
-          if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && !stageInfo.emittedTaskSizeWarning) {
+          if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 &&
+              !stageInfo.emittedTaskSizeWarning) {
             stageInfo.emittedTaskSizeWarning = true
             logWarning(("Stage %d (%s) contains a task of very large " +
               "size (%d KB). The maximum recommended task size is %d KB.").format(
@@ -815,7 +828,7 @@ class DAGScheduler(
       }
       logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime))
       stageToInfos(stage).completionTime = Some(System.currentTimeMillis())
-      listenerBus.post(StageCompleted(stageToInfos(stage)))
+      listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage)))
       running -= stage
     }
     event.reason match {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 1791ee660db0212ca7afc4dbc99cb18639087009..90eb8a747f91c88436aa9f6ffd205eade6380f8e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -32,7 +32,7 @@ import scala.collection.JavaConversions._
 /**
  * Parses and holds information about inputFormat (and files) specified as a parameter.
  */
-class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], 
+class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_],
                       val path: String) extends Logging {
 
   var mapreduceInputFormat: Boolean = false
@@ -40,7 +40,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
 
   validate()
 
-  override def toString(): String = {
+  override def toString: String = {
     "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path
   }
 
@@ -125,7 +125,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
    }
 
   private def findPreferredLocations(): Set[SplitInfo] = {
-    logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + 
+    logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat +
       ", inputFormatClazz : " + inputFormatClazz)
     if (mapreduceInputFormat) {
       return prefLocsFromMapreduceInputFormat()
@@ -143,14 +143,14 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
 object InputFormatInfo {
   /**
     Computes the preferred locations based on input(s) and returned a location to block map.
-    Typical use of this method for allocation would follow some algo like this 
-    (which is what we currently do in YARN branch) :
+    Typical use of this method for allocation would follow some algo like this:
+
     a) For each host, count number of splits hosted on that host.
     b) Decrement the currently allocated containers on that host.
     c) Compute rack info for each host and update rack -> count map based on (b).
     d) Allocate nodes based on (c)
-    e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node 
-       (even if data locality on that is very high) : this is to prevent fragility of job if a single 
+    e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node
+       (even if data locality on that is very high) : this is to prevent fragility of job if a single
        (or small set of) hosts go down.
 
     go to (a) until required nodes are allocated.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
index 60927831a159a7d4e2b092b97775ac3005ed1c1e..f8fa5a9f7a5903384c4514883701d37c82809827 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -297,7 +297,7 @@ class JobLogger(val user: String, val logDirName: String)
    * When stage is completed, record stage completion status
    * @param stageCompleted Stage completed event
    */
-  override def onStageCompleted(stageCompleted: StageCompleted) {
+  override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) {
     stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format(
         stageCompleted.stage.stageId))
   }
@@ -328,10 +328,6 @@ class JobLogger(val user: String, val logDirName: String)
                       task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" +
                       mapId + " REDUCE_ID=" + reduceId
         stageLogInfo(task.stageId, taskStatus)
-      case OtherFailure(message) =>
-        taskStatus += " STATUS=FAILURE TID=" + taskInfo.taskId +
-                      " STAGE_ID=" + task.stageId + " INFO=" + message
-        stageLogInfo(task.stageId, taskStatus)
       case _ =>
     }
   }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
index 596f9adde94938ad6a9c092aeb0d34cb937b665e..17912422150782a72252385ee696044355397f87 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
@@ -117,8 +117,4 @@ private[spark] class Pool(
       parent.decreaseRunningTasks(taskNum)
     }
   }
-
-  override def hasPendingTasks(): Boolean = {
-    schedulableQueue.exists(_.hasPendingTasks())
-  }
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index 310ec62ca8afe3a25264bb1d3368c06e86ea4f1e..28f3ba53b84253e3b5029c686d3ef9c7936eadf9 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -32,7 +32,9 @@ private[spark] object ResultTask {
   // expensive on the master node if it needs to launch thousands of tasks.
   val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
 
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues)
+  // TODO: This object shouldn't have global variables
+  val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf)
 
   def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
     synchronized {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
index 1c7ea2dccc7d9b60b735fa37c4cc604db8fe6158..d573e125a33d1af5eec279b8d84c5c74c63d1f73 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
@@ -42,5 +42,4 @@ private[spark] trait Schedulable {
   def executorLost(executorId: String, host: String): Unit
   def checkSpeculatableTasks(): Boolean
   def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager]
-  def hasPendingTasks(): Boolean
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
index 356fe56bf32c73c70f2f67ad2c5502957577fd80..3cf995ea74244c2440d101b5a247fa6896454bc3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
@@ -20,7 +20,7 @@ package org.apache.spark.scheduler
 import java.io.{FileInputStream, InputStream}
 import java.util.{NoSuchElementException, Properties}
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 import scala.xml.XML
 
@@ -49,10 +49,10 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool)
   }
 }
 
-private[spark] class FairSchedulableBuilder(val rootPool: Pool)
+private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf)
   extends SchedulableBuilder with Logging {
 
-  val schedulerAllocFile = Option(System.getProperty("spark.scheduler.allocation.file"))
+  val schedulerAllocFile = conf.getOption("spark.scheduler.allocation.file")
   val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml"
   val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool"
   val DEFAULT_POOL_NAME = "default"
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
index 89aa098664dae46e227ba833ff5264deae083660..02bdbba825781968fe672ca969e9919be7c3a319 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
@@ -31,7 +31,4 @@ private[spark] trait SchedulerBackend {
   def defaultParallelism(): Int
 
   def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException
-
-  // Memory used by each executor (in megabytes)
-  protected val executorMemory: Int = SparkContext.executorMemoryRequested
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 0f2deb4bcbbb207b7757cd183896ceb3dc7b2ff9..a37ead563271a9b2171b1095eb7589151abe1777 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -37,7 +37,9 @@ private[spark] object ShuffleMapTask {
   // expensive on the master node if it needs to launch thousands of tasks.
   val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
 
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues)
+  // TODO: This object shouldn't have global variables
+  val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf)
 
   def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
     synchronized {
@@ -152,7 +154,7 @@ private[spark] class ShuffleMapTask(
 
     try {
       // Obtain all the block writers for shuffle blocks.
-      val ser = SparkEnv.get.serializerManager.get(dep.serializerClass)
+      val ser = SparkEnv.get.serializerManager.get(dep.serializerClass, SparkEnv.get.conf)
       shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser)
 
       // Write the map output to its associated buckets.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index ee63b3c4a15a26aad6c061cd65c29779ae7ab00c..627995c826e2b844cdc6a9656ee2d45c51a204df 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -27,7 +27,7 @@ sealed trait SparkListenerEvents
 case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties)
      extends SparkListenerEvents
 
-case class StageCompleted(val stage: StageInfo) extends SparkListenerEvents
+case class SparkListenerStageCompleted(val stage: StageInfo) extends SparkListenerEvents
 
 case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents
 
@@ -47,7 +47,7 @@ trait SparkListener {
   /**
    * Called when a stage is completed, with information on the completed stage
    */
-  def onStageCompleted(stageCompleted: StageCompleted) { }
+  def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { }
 
   /**
    * Called when a stage is submitted
@@ -86,7 +86,7 @@ trait SparkListener {
  * Simple SparkListener that logs a few summary statistics when each stage completes
  */
 class StatsReportListener extends SparkListener with Logging {
-  override def onStageCompleted(stageCompleted: StageCompleted) {
+  override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) {
     import org.apache.spark.scheduler.StatsReportListener._
     implicit val sc = stageCompleted
     this.logInfo("Finished stage: " + stageCompleted.stage)
@@ -119,13 +119,17 @@ object StatsReportListener extends Logging {
   val probabilities = percentiles.map{_ / 100.0}
   val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%"
 
-  def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = {
+  def extractDoubleDistribution(stage: SparkListenerStageCompleted,
+      getMetric: (TaskInfo,TaskMetrics) => Option[Double])
+    : Option[Distribution] = {
     Distribution(stage.stage.taskInfos.flatMap {
       case ((info,metric)) => getMetric(info, metric)})
   }
 
   //is there some way to setup the types that I can get rid of this completely?
-  def extractLongDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Long]): Option[Distribution] = {
+  def extractLongDistribution(stage: SparkListenerStageCompleted,
+      getMetric: (TaskInfo,TaskMetrics) => Option[Long])
+    : Option[Distribution] = {
     extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble})
   }
 
@@ -147,12 +151,12 @@ object StatsReportListener extends Logging {
   }
 
   def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double])
-    (implicit stage: StageCompleted) {
+    (implicit stage: SparkListenerStageCompleted) {
     showDistribution(heading, extractDoubleDistribution(stage, getMetric), format)
   }
 
   def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long])
-    (implicit stage: StageCompleted) {
+    (implicit stage: SparkListenerStageCompleted) {
     showBytesDistribution(heading, extractLongDistribution(stage, getMetric))
   }
 
@@ -169,7 +173,7 @@ object StatsReportListener extends Logging {
   }
 
   def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long])
-    (implicit stage: StageCompleted) {
+    (implicit stage: SparkListenerStageCompleted) {
     showMillisDistribution(heading, extractLongDistribution(stage, getMetric))
   }
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index 85687ea330660533c2fb95c1f5016c3db0ffb152..e7defd768b2c3857e03b8747b6026dfcb9428dd6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -41,7 +41,7 @@ private[spark] class SparkListenerBus() extends Logging {
         event match {
           case stageSubmitted: SparkListenerStageSubmitted =>
             sparkListeners.foreach(_.onStageSubmitted(stageSubmitted))
-          case stageCompleted: StageCompleted =>
+          case stageCompleted: SparkListenerStageCompleted =>
             sparkListeners.foreach(_.onStageCompleted(stageCompleted))
           case jobStart: SparkListenerJobStart =>
             sparkListeners.foreach(_.onJobStart(jobStart))
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index 89102720fa0fddb00e1dfdf2297ebe4a6ca1950a..e22b1e53e80482149fc19d7c65d5930a71a352d8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -30,7 +30,8 @@ import org.apache.spark.util.Utils
  */
 private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl)
   extends Logging {
-  private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt
+
+  private val THREADS = sparkEnv.conf.get("spark.resultGetter.threads", "4").toInt
   private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool(
     THREADS, "Result resolver thread")
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index dbac6b96ac7de8303e93666d4073563ad35f9b30..0c8ed6275991a4cd7f8def626244c1f73ed8cd28 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -35,7 +35,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
  * It can also work with a local setup by using a LocalBackend and setting isLocal to true.
  * It handles common logic, like determining a scheduling order across jobs, waking up to launch
  * speculative tasks, etc.
- * 
+ *
  * Clients should first call initialize() and start(), then submit task sets through the
  * runTasks method.
  *
@@ -47,15 +47,19 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
  */
 private[spark] class TaskSchedulerImpl(
     val sc: SparkContext,
-    val maxTaskFailures : Int = System.getProperty("spark.task.maxFailures", "4").toInt,
+    val maxTaskFailures: Int,
     isLocal: Boolean = false)
-  extends TaskScheduler with Logging {
+  extends TaskScheduler with Logging
+{
+  def this(sc: SparkContext) = this(sc, sc.conf.get("spark.task.maxFailures", "4").toInt)
+
+  val conf = sc.conf
 
   // How often to check for speculative tasks
-  val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong
+  val SPECULATION_INTERVAL = conf.get("spark.speculation.interval", "100").toLong
 
   // Threshold above which we warn user initial TaskSet may be starved
-  val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong
+  val STARVATION_TIMEOUT = conf.get("spark.starvation.timeout", "15000").toLong
 
   // TaskSetManagers are not thread safe, so any access to one should be synchronized
   // on this class.
@@ -92,7 +96,7 @@ private[spark] class TaskSchedulerImpl(
   var rootPool: Pool = null
   // default scheduler is FIFO
   val schedulingMode: SchedulingMode = SchedulingMode.withName(
-    System.getProperty("spark.scheduler.mode", "FIFO"))
+    conf.get("spark.scheduler.mode", "FIFO"))
 
   // This is a var so that we can reset it for testing purposes.
   private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this)
@@ -110,7 +114,7 @@ private[spark] class TaskSchedulerImpl(
         case SchedulingMode.FIFO =>
           new FIFOSchedulableBuilder(rootPool)
         case SchedulingMode.FAIR =>
-          new FairSchedulableBuilder(rootPool)
+          new FairSchedulableBuilder(rootPool, conf)
       }
     }
     schedulableBuilder.buildPools()
@@ -121,7 +125,7 @@ private[spark] class TaskSchedulerImpl(
   override def start() {
     backend.start()
 
-    if (!isLocal && System.getProperty("spark.speculation", "false").toBoolean) {
+    if (!isLocal && conf.get("spark.speculation", "false").toBoolean) {
       logInfo("Starting speculative execution thread")
       import sc.env.actorSystem.dispatcher
       sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
@@ -281,7 +285,8 @@ private[spark] class TaskSchedulerImpl(
               }
             }
           case None =>
-            logInfo("Ignoring update from TID " + tid + " because its task set is gone")
+            logInfo("Ignoring update with state %s from TID %s because its task set is gone"
+              .format(state, tid))
         }
       } catch {
         case e: Exception => logError("Exception in statusUpdate", e)
@@ -324,7 +329,7 @@ private[spark] class TaskSchedulerImpl(
         // Have each task set throw a SparkException with the error
         for ((taskSetId, manager) <- activeTaskSets) {
           try {
-            manager.error(message)
+            manager.abort(message)
           } catch {
             case e: Exception => logError("Exception in error callback", e)
           }
@@ -365,13 +370,6 @@ private[spark] class TaskSchedulerImpl(
     }
   }
 
-  // Check for pending tasks in all our active jobs.
-  def hasPendingTasks: Boolean = {
-    synchronized {
-      rootPool.hasPendingTasks()
-    }
-  }
-
   def executorLost(executorId: String, reason: ExecutorLossReason) {
     var failedExecutor: Option[String] = None
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index c676e73e033d6f2fb2b65ab5602a6b043e1db634..6dd1469d8f801b829a739f878bec88c1b8746b97 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -54,12 +54,14 @@ private[spark] class TaskSetManager(
     clock: Clock = SystemClock)
   extends Schedulable with Logging
 {
+  val conf = sched.sc.conf
+
   // CPUs to request per task
-  val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt
+  val CPUS_PER_TASK = conf.get("spark.task.cpus", "1").toInt
 
   // Quantile of tasks at which to start speculation
-  val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble
-  val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble
+  val SPECULATION_QUANTILE = conf.get("spark.speculation.quantile", "0.75").toDouble
+  val SPECULATION_MULTIPLIER = conf.get("spark.speculation.multiplier", "1.5").toDouble
 
   // Serializer for closures and tasks.
   val env = SparkEnv.get
@@ -112,13 +114,9 @@ private[spark] class TaskSetManager(
   // Task index, start and finish time for each task attempt (indexed by task ID)
   val taskInfos = new HashMap[Long, TaskInfo]
 
-  // Did the TaskSet fail?
-  var failed = false
-  var causeOfFailure = ""
-
   // How frequently to reprint duplicate exceptions in full, in milliseconds
   val EXCEPTION_PRINT_INTERVAL =
-    System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong
+    conf.get("spark.logging.exceptionPrintInterval", "10000").toLong
 
   // Map of recent exceptions (identified by string representation and top stack frame) to
   // duplicate count (how many times the same exception has appeared) and time the full exception
@@ -550,14 +548,7 @@ private[spark] class TaskSetManager(
     }
   }
 
-  def error(message: String) {
-    // Save the error message
-    abort("Error: " + message)
-  }
-
   def abort(message: String) {
-    failed = true
-    causeOfFailure = message
     // TODO: Kill running tasks if we were not terminated due to a Mesos error
     sched.dagScheduler.taskSetFailed(taskSet, message)
     removeAllRunningTasks()
@@ -681,19 +672,15 @@ private[spark] class TaskSetManager(
     return foundTasks
   }
 
-  override def hasPendingTasks(): Boolean = {
-    numTasks > 0 && tasksSuccessful < numTasks
-  }
-
   private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
-    val defaultWait = System.getProperty("spark.locality.wait", "3000")
+    val defaultWait = conf.get("spark.locality.wait", "3000")
     level match {
       case TaskLocality.PROCESS_LOCAL =>
-        System.getProperty("spark.locality.wait.process", defaultWait).toLong
+        conf.get("spark.locality.wait.process", defaultWait).toLong
       case TaskLocality.NODE_LOCAL =>
-        System.getProperty("spark.locality.wait.node", defaultWait).toLong
+        conf.get("spark.locality.wait.node", defaultWait).toLong
       case TaskLocality.RACK_LOCAL =>
-        System.getProperty("spark.locality.wait.rack", defaultWait).toLong
+        conf.get("spark.locality.wait.rack", defaultWait).toLong
       case TaskLocality.ANY =>
         0L
     }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 5c534a6f436f87de32cb4511e807e3ee6aa2ae4f..2f5bcafe40394d24a28bfd66915bb98436652f2f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -48,8 +48,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
 {
   // Use an atomic variable to track total number of cores in the cluster for simplicity and speed
   var totalCoreCount = new AtomicInteger(0)
-
-  private val timeout = AkkaUtils.askTimeout
+  val conf = scheduler.sc.conf
+  private val timeout = AkkaUtils.askTimeout(conf)
 
   class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor {
     private val executorActor = new HashMap[String, ActorRef]
@@ -63,7 +63,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
       context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
 
       // Periodically revive offers to allow delay scheduling to work
-      val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong
+      val reviveInterval = conf.get("spark.scheduler.revive.interval", "1000").toLong
       import context.dispatcher
       context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers)
     }
@@ -119,7 +119,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
         removeExecutor(executorId, reason)
         sender ! true
 
-      case DisassociatedEvent(_, address, _) => 
+      case DisassociatedEvent(_, address, _) =>
         addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated"))
 
     }
@@ -164,14 +164,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
 
   override def start() {
     val properties = new ArrayBuffer[(String, String)]
-    val iterator = System.getProperties.entrySet.iterator
-    while (iterator.hasNext) {
-      val entry = iterator.next
-      val (key, value) = (entry.getKey.toString, entry.getValue.toString)
+    for ((key, value) <- scheduler.sc.conf.getAll) {
       if (key.startsWith("spark.") && !key.equals("spark.hostPort")) {
         properties += ((key, value))
       }
     }
+    //TODO (prashant) send conf instead of properties
     driverActor = actorSystem.actorOf(
       Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME)
   }
@@ -210,8 +208,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
     driverActor ! KillTask(taskId, executorId)
   }
 
-  override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism"))
-      .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2))
+  override def defaultParallelism(): Int = {
+    conf.getOption("spark.default.parallelism").map(_.toInt).getOrElse(
+      math.max(totalCoreCount.get(), 2))
+  }
 
   // Called by subclasses when notified of a lost worker
   def removeExecutor(executorId: String, reason: String) {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
index ec3e68e9709378fd4ce5f45c668f90250aebff31..b44d1e43c85c770b5f7d3d76c5d29d556be2e93d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -33,13 +33,13 @@ private[spark] class SimrSchedulerBackend(
   val tmpPath = new Path(driverFilePath + "_tmp")
   val filePath = new Path(driverFilePath)
 
-  val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt
+  val maxCores = conf.get("spark.simr.executor.cores", "1").toInt
 
   override def start() {
     super.start()
 
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
-      System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+      sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"),
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
 
     val conf = new Configuration()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 0615f7b565e404fc331982ad9a6ee4fa165d6a3c..faa6e1ebe886f46b030f2114844a8229cee357c5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -38,23 +38,23 @@ private[spark] class SparkDeploySchedulerBackend(
   var stopping = false
   var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
 
-  val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt
+  val maxCores = conf.getOption("spark.cores.max").map(_.toInt)
 
   override def start() {
     super.start()
 
     // The endpoint for executors to talk to us
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
-      System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+      conf.get("spark.driver.host"),  conf.get("spark.driver.port"),
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
     val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}")
     val command = Command(
       "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
     val sparkHome = sc.getSparkHome().getOrElse(null)
-    val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
+    val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome,
         "http://" + sc.ui.appUIAddress)
 
-    client = new AppClient(sc.env.actorSystem, masters, appDesc, this)
+    client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf)
     client.start()
   }
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 39573fc8c9b5138c7d72154527bf3d4c0fea772f..d46fceba8918a1cc76deea05b938315072f86ff3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -62,7 +62,7 @@ private[spark] class CoarseMesosSchedulerBackend(
   var driver: SchedulerDriver = null
 
   // Maximum number of cores to acquire (TODO: we'll need more flexible controls here)
-  val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt
+  val maxCores = conf.get("spark.cores.max",  Int.MaxValue.toString).toInt
 
   // Cores we have acquired with each Mesos task ID
   val coresByTaskId = new HashMap[Int, Int]
@@ -77,7 +77,7 @@ private[spark] class CoarseMesosSchedulerBackend(
     "Spark home is not set; set it through the spark.home system " +
     "property, the SPARK_HOME environment variable or the SparkContext constructor"))
 
-  val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt
+  val extraCoresPerSlave = conf.get("spark.mesos.extra.cores", "0").toInt
 
   var nextMesosTaskId = 0
 
@@ -122,12 +122,12 @@ private[spark] class CoarseMesosSchedulerBackend(
     val command = CommandInfo.newBuilder()
       .setEnvironment(environment)
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
-      System.getProperty("spark.driver.host"),
-      System.getProperty("spark.driver.port"),
+      conf.get("spark.driver.host"),
+      conf.get("spark.driver.port"),
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
-    val uri = System.getProperty("spark.executor.uri")
+    val uri = conf.get("spark.executor.uri", null)
     if (uri == null) {
-      val runScript = new File(sparkHome, "spark-class").getCanonicalPath
+      val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath
       command.setValue(
         "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format(
           runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
@@ -136,7 +136,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
       command.setValue(
-        "cd %s*; ./spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
+        "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
           .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }
@@ -177,7 +177,7 @@ private[spark] class CoarseMesosSchedulerBackend(
         val slaveId = offer.getSlaveId.toString
         val mem = getResource(offer.getResourcesList, "mem")
         val cpus = getResource(offer.getResourcesList, "cpus").toInt
-        if (totalCoresAcquired < maxCores && mem >= executorMemory && cpus >= 1 &&
+        if (totalCoresAcquired < maxCores && mem >= sc.executorMemory && cpus >= 1 &&
             failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES &&
             !slaveIdsWithExecutors.contains(slaveId)) {
           // Launch an executor on the slave
@@ -193,7 +193,7 @@ private[spark] class CoarseMesosSchedulerBackend(
             .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave))
             .setName("Task " + taskId)
             .addResources(createResource("cpus", cpusToUse))
-            .addResources(createResource("mem", executorMemory))
+            .addResources(createResource("mem", sc.executorMemory))
             .build()
           d.launchTasks(offer.getId, Collections.singletonList(task), filters)
         } else {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 6aa788c4609cdfceb1fec08c412e38cd507cf0f6..ae8d527352f733b5f8e002372b0a422e0b33c8c6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -100,20 +100,20 @@ private[spark] class MesosSchedulerBackend(
     }
     val command = CommandInfo.newBuilder()
       .setEnvironment(environment)
-    val uri = System.getProperty("spark.executor.uri")
+    val uri = sc.conf.get("spark.executor.uri", null)
     if (uri == null) {
-      command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath)
+      command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath)
     } else {
       // 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*; ./spark-executor".format(basename))
+      command.setValue("cd %s*; ./sbin/spark-executor".format(basename))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }
     val memory = Resource.newBuilder()
       .setName("mem")
       .setType(Value.Type.SCALAR)
-      .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build())
+      .setScalar(Value.Scalar.newBuilder().setValue(sc.executorMemory).build())
       .build()
     ExecutorInfo.newBuilder()
       .setExecutorId(ExecutorID.newBuilder().setValue(execId).build())
@@ -198,7 +198,7 @@ private[spark] class MesosSchedulerBackend(
         def enoughMemory(o: Offer) = {
           val mem = getResource(o.getResourcesList, "mem")
           val slaveId = o.getSlaveId.getValue
-          mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId)
+          mem >= sc.executorMemory || slaveIdsWithExecutors.contains(slaveId)
         }
 
         for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) {
@@ -340,5 +340,5 @@ private[spark] class MesosSchedulerBackend(
   }
 
   // TODO: query Mesos for number of cores
-  override def defaultParallelism() = System.getProperty("spark.default.parallelism", "8").toInt
+  override def defaultParallelism() = sc.conf.get("spark.default.parallelism", "8").toInt
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
index 4edc6a0d3f2a06a17a0768e90b3a69bfd7d15348..897d47a9ad981951e756d58f46545d9c8f42ef4c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
@@ -47,7 +47,8 @@ private[spark] class LocalActor(
   private val localExecutorId = "localhost"
   private val localExecutorHostname = "localhost"
 
-  val executor = new Executor(localExecutorId, localExecutorHostname, Seq.empty, isLocal = true)
+  val executor = new Executor(
+    localExecutorId, localExecutorHostname, scheduler.conf.getAll, isLocal = true)
 
   def receive = {
     case ReviveOffers =>
diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index 4de81617b1dd8bf38b617b86e1e67d1158729463..5d3d43623d9d2368b7527c12dcf165f012f0943c 100644
--- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
@@ -21,6 +21,7 @@ import java.io._
 import java.nio.ByteBuffer
 
 import org.apache.spark.util.ByteBufferInputStream
+import org.apache.spark.SparkConf
 
 private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream {
   val objOut = new ObjectOutputStream(out)
@@ -77,6 +78,6 @@ private[spark] class JavaSerializerInstance extends SerializerInstance {
 /**
  * A Spark serializer that uses Java's built-in serialization.
  */
-class JavaSerializer extends Serializer {
+class JavaSerializer(conf: SparkConf) extends Serializer {
   def newInstance(): SerializerInstance = new JavaSerializerInstance
 }
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index e748c2275d589c6702576b72ac49038e86033c93..a24a3b04b87ccf57f365143781b98226b4c256a0 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -25,18 +25,18 @@ import com.esotericsoftware.kryo.{KryoException, Kryo}
 import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
 import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar}
 
-import org.apache.spark.{SerializableWritable, Logging}
+import org.apache.spark._
 import org.apache.spark.broadcast.HttpBroadcast
 import org.apache.spark.scheduler.MapStatus
 import org.apache.spark.storage._
+import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock}
 
 /**
  * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]].
  */
-class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging {
-
+class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging {
   private val bufferSize = {
-    System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
+    conf.get("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
   }
 
   def newKryoOutput() = new KryoOutput(bufferSize)
@@ -48,7 +48,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
 
     // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops.
     // Do this before we invoke the user registrator so the user registrator can override this.
-    kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean)
+    kryo.setReferences(conf.get("spark.kryo.referenceTracking", "true").toBoolean)
 
     for (cls <- KryoSerializer.toRegister) kryo.register(cls)
 
@@ -58,13 +58,13 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
 
     // Allow the user to register their own classes by setting spark.kryo.registrator
     try {
-      Option(System.getProperty("spark.kryo.registrator")).foreach { regCls =>
+      for (regCls <- conf.getOption("spark.kryo.registrator")) {
         logDebug("Running user registrator: " + regCls)
         val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]
         reg.registerClasses(kryo)
       }
     } catch {
-      case _: Exception => println("Failed to register spark.kryo.registrator")
+      case e: Exception => logError("Failed to run spark.kryo.registrator", e)
     }
 
     // Register Chill's classes; we do this after our ranges and the user's own classes to let
diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
index 160cca4d6c54a69dca1232a38b5cbf94ad9e54d7..9a5e3cb77e1d53a501797e4698bf623738fff09c 100644
--- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
@@ -29,6 +29,9 @@ import org.apache.spark.util.{NextIterator, ByteBufferInputStream}
  * A serializer. Because some serialization libraries are not thread safe, this class is used to
  * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual serialization and are
  * guaranteed to only be called from one thread at a time.
+ *
+ * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a
+ * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes precedence.
  */
 trait Serializer {
   def newInstance(): SerializerInstance
diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
index 2955986feced55d1a2f3ec00f174b5b793560f8c..36a37af4f821daddfcf0ac84db02a266f3ee09c5 100644
--- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.serializer
 
 import java.util.concurrent.ConcurrentHashMap
+import org.apache.spark.SparkConf
 
 
 /**
@@ -26,18 +27,19 @@ import java.util.concurrent.ConcurrentHashMap
  * creating a new one.
  */
 private[spark] class SerializerManager {
+  // TODO: Consider moving this into SparkConf itself to remove the global singleton.
 
   private val serializers = new ConcurrentHashMap[String, Serializer]
   private var _default: Serializer = _
 
   def default = _default
 
-  def setDefault(clsName: String): Serializer = {
-    _default = get(clsName)
+  def setDefault(clsName: String, conf: SparkConf): Serializer = {
+    _default = get(clsName, conf)
     _default
   }
 
-  def get(clsName: String): Serializer = {
+  def get(clsName: String, conf: SparkConf): Serializer = {
     if (clsName == null) {
       default
     } else {
@@ -51,8 +53,19 @@ private[spark] class SerializerManager {
         serializer = serializers.get(clsName)
         if (serializer == null) {
           val clsLoader = Thread.currentThread.getContextClassLoader
-          serializer =
-            Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer]
+          val cls = Class.forName(clsName, true, clsLoader)
+
+          // First try with the constructor that takes SparkConf. If we can't find one,
+          // use a no-arg constructor instead.
+          try {
+            val constructor = cls.getConstructor(classOf[SparkConf])
+            serializer = constructor.newInstance(conf).asInstanceOf[Serializer]
+          } catch {
+            case _: NoSuchMethodException =>
+              val constructor = cls.getConstructor()
+              serializer = constructor.newInstance().asInstanceOf[Serializer]
+          }
+
           serializers.put(clsName, serializer)
         }
         serializer
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
index e51c5b30a3daba19b3407c6a7ec1191f486747c6..47478631a11f0289282250fe9ba4d00622479236 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
@@ -312,7 +312,7 @@ object BlockFetcherIterator {
       logDebug("Sending request for %d blocks (%s) from %s".format(
         req.blocks.size, Utils.bytesToString(req.size), req.address.host))
       val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort)
-      val cpier = new ShuffleCopier
+      val cpier = new ShuffleCopier(blockManager.conf)
       cpier.getBlocks(cmId, req.blocks, putResult)
       logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host )
     }
@@ -327,7 +327,7 @@ object BlockFetcherIterator {
         fetchRequestsSync.put(request)
       }
 
-      copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt)
+      copiers = startCopiers(conf.get("spark.shuffle.copier.threads", "6").toInt)
       logInfo("Started " + fetchRequestsSync.size + " remote gets in " +
         Utils.getUsedTimeMs(startTime))
 
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 19a025a32973aceada0d25abbf5a5dd4d390ec70..6d2cda97b04ebb5c9e08b2ee33c6450a2499a449 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -30,7 +30,7 @@ import scala.concurrent.duration._
 
 import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream}
 
-import org.apache.spark.{Logging, SparkEnv, SparkException}
+import org.apache.spark.{SparkConf, Logging, SparkEnv, SparkException}
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.network._
 import org.apache.spark.serializer.Serializer
@@ -43,12 +43,13 @@ private[spark] class BlockManager(
     actorSystem: ActorSystem,
     val master: BlockManagerMaster,
     val defaultSerializer: Serializer,
-    maxMemory: Long)
+    maxMemory: Long,
+    val conf: SparkConf)
   extends Logging {
 
   val shuffleBlockManager = new ShuffleBlockManager(this)
   val diskBlockManager = new DiskBlockManager(shuffleBlockManager,
-    System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
+    conf.get("spark.local.dir",  System.getProperty("java.io.tmpdir")))
 
   private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo]
 
@@ -57,12 +58,12 @@ private[spark] class BlockManager(
 
   // If we use Netty for shuffle, start a new Netty-based shuffle sender service.
   private val nettyPort: Int = {
-    val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean
-    val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt
+    val useNetty = conf.get("spark.shuffle.use.netty", "false").toBoolean
+    val nettyPortConfig = conf.get("spark.shuffle.sender.port", "0").toInt
     if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0
   }
 
-  val connectionManager = new ConnectionManager(0)
+  val connectionManager = new ConnectionManager(0, conf)
   implicit val futureExecContext = connectionManager.futureExecContext
 
   val blockManagerId = BlockManagerId(
@@ -71,18 +72,18 @@ private[spark] class BlockManager(
   // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory
   // for receiving shuffle outputs)
   val maxBytesInFlight =
-    System.getProperty("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024
+    conf.get("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024
 
   // Whether to compress broadcast variables that are stored
-  val compressBroadcast = System.getProperty("spark.broadcast.compress", "true").toBoolean
+  val compressBroadcast = conf.get("spark.broadcast.compress", "true").toBoolean
   // Whether to compress shuffle output that are stored
-  val compressShuffle = System.getProperty("spark.shuffle.compress", "true").toBoolean
+  val compressShuffle = conf.get("spark.shuffle.compress", "true").toBoolean
   // Whether to compress RDD partitions that are stored serialized
-  val compressRdds = System.getProperty("spark.rdd.compress", "false").toBoolean
+  val compressRdds = conf.get("spark.rdd.compress", "false").toBoolean
 
-  val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties
+  val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf)
 
-  val hostPort = Utils.localHostPort()
+  val hostPort = Utils.localHostPort(conf)
 
   val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)),
     name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next)
@@ -100,8 +101,11 @@ private[spark] class BlockManager(
 
   var heartBeatTask: Cancellable = null
 
-  private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks)
-  private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks)
+  private val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf)
+  private val broadcastCleaner = new MetadataCleaner(
+    MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf)
+
   initialize()
 
   // The compression codec to use. Note that the "lazy" val is necessary because we want to delay
@@ -109,14 +113,14 @@ private[spark] class BlockManager(
   // program could be using a user-defined codec in a third party jar, which is loaded in
   // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been
   // loaded yet.
-  private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec()
+  private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf)
 
   /**
    * Construct a BlockManager with a memory limit set based on system properties.
    */
   def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster,
-           serializer: Serializer) = {
-    this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties)
+           serializer: Serializer, conf: SparkConf) = {
+    this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf)
   }
 
   /**
@@ -126,7 +130,7 @@ private[spark] class BlockManager(
   private def initialize() {
     master.registerBlockManager(blockManagerId, maxMemory, slaveActor)
     BlockManagerWorker.startBlockManagerWorker(this)
-    if (!BlockManager.getDisableHeartBeatsForTesting) {
+    if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
       heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) {
         heartBeat()
       }
@@ -439,7 +443,7 @@ private[spark] class BlockManager(
       : BlockFetcherIterator = {
 
     val iter =
-      if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) {
+      if (conf.get("spark.shuffle.use.netty", "false").toBoolean) {
         new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer)
       } else {
         new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer)
@@ -465,7 +469,8 @@ private[spark] class BlockManager(
   def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int)
     : BlockObjectWriter = {
     val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _)
-    new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream)
+    val syncWrites = conf.get("spark.shuffle.sync", "false").toBoolean
+    new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites)
   }
 
   /**
@@ -856,19 +861,18 @@ private[spark] class BlockManager(
 
 
 private[spark] object BlockManager extends Logging {
-
   val ID_GENERATOR = new IdGenerator
 
-  def getMaxMemoryFromSystemProperties: Long = {
-    val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble
+  def getMaxMemory(conf: SparkConf): Long = {
+    val memoryFraction = conf.get("spark.storage.memoryFraction", "0.66").toDouble
     (Runtime.getRuntime.maxMemory * memoryFraction).toLong
   }
 
-  def getHeartBeatFrequencyFromSystemProperties: Long =
-    System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
+  def getHeartBeatFrequency(conf: SparkConf): Long =
+    conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
 
-  def getDisableHeartBeatsForTesting: Boolean =
-    System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean
+  def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean =
+    conf.get("spark.test.disableBlockManagerHeartBeat", "false").toBoolean
 
   /**
    * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
index e1d68ef59296af77c6d6e1b09bbd2690ce6e7444..51a29ed8ef81ad2818b86f5fad6bbc6a874c2378 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -23,19 +23,19 @@ import scala.concurrent.ExecutionContext.Implicits.global
 import akka.actor._
 import akka.pattern.ask
 
-import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.{SparkConf, Logging, SparkException}
 import org.apache.spark.storage.BlockManagerMessages._
 import org.apache.spark.util.AkkaUtils
 
 private[spark]
-class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging {
+class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging {
 
-  val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
-  val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt
+  val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt
+  val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt
 
   val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"
 
-  val timeout = AkkaUtils.askTimeout
+  val timeout = AkkaUtils.askTimeout(conf)
 
   /** Remove a dead executor from the driver actor. This is only called on the driver side. */
   def removeExecutor(execId: String) {
@@ -158,10 +158,7 @@ class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) ext
     while (attempts < AKKA_RETRY_ATTEMPTS) {
       attempts += 1
       try {
-        val future = driverActor match {
-          case Left(a: ActorRef) => a.ask(message)(timeout)
-          case Right(b: ActorSelection) => b.ask(message)(timeout)
-        }
+        val future = driverActor.ask(message)(timeout)
         val result = Await.result(future, timeout)
         if (result == null) {
           throw new SparkException("BlockManagerMaster returned null")
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index 21022e1cfb0866e30514b1d1db11888a356b3179..58452d96574c9b2d78dc148b14d91e0a2b50769e 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -27,7 +27,7 @@ import scala.concurrent.duration._
 import akka.actor.{Actor, ActorRef, Cancellable}
 import akka.pattern.ask
 
-import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.{SparkConf, Logging, SparkException}
 import org.apache.spark.storage.BlockManagerMessages._
 import org.apache.spark.util.{AkkaUtils, Utils}
 
@@ -36,7 +36,7 @@ import org.apache.spark.util.{AkkaUtils, Utils}
  * all slaves' block managers.
  */
 private[spark]
-class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
+class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging {
 
   // Mapping from block manager id to the block manager's information.
   private val blockManagerInfo =
@@ -48,20 +48,18 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
   // Mapping from block id to the set of block managers that have the block.
   private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]]
 
-  private val akkaTimeout = AkkaUtils.askTimeout
+  private val akkaTimeout = AkkaUtils.askTimeout(conf)
 
-  initLogging()
+  val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs",
+    "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong
 
-  val slaveTimeout = System.getProperty("spark.storage.blockManagerSlaveTimeoutMs",
-    "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong
-
-  val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs",
+  val checkTimeoutInterval = conf.get("spark.storage.blockManagerTimeoutIntervalMs",
     "60000").toLong
 
   var timeoutCheckingTask: Cancellable = null
 
   override def preStart() {
-    if (!BlockManager.getDisableHeartBeatsForTesting) {
+    if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
       import context.dispatcher
       timeoutCheckingTask = context.system.scheduler.schedule(
         0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
index 0c66addf9def6f78628bfff2cec9b7c9e9a8a3b4..21f003609b14de5f511ebca2006afe72faf53d0b 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
@@ -30,7 +30,6 @@ import org.apache.spark.util.Utils
  * TODO: Use event model.
  */
 private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
-  initLogging()
 
   blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive)
 
@@ -101,8 +100,6 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
 private[spark] object BlockManagerWorker extends Logging {
   private var blockManagerWorker: BlockManagerWorker = null
 
-  initLogging()
-
   def startBlockManagerWorker(manager: BlockManager) {
     blockManagerWorker = new BlockManagerWorker(manager)
   }
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
index 6ce9127c7450175d61ed66871678ea7b7fc62dc8..a06f50a0ac89cf6f3ada7fad21d8beb59245ab82 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
@@ -37,8 +37,6 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM
 
   def length = blockMessages.length 
 
-  initLogging()
-  
   def set(bufferMessage: BufferMessage) {
     val startTime = System.currentTimeMillis
     val newBlockMessages = new ArrayBuffer[BlockMessage]()
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
index b4451fc7b8e5666a9c93c8e89d27ebb08b224543..61e63c60d56e3b8e46bfa6fd1599863cfd0143be 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -74,7 +74,8 @@ class DiskBlockObjectWriter(
     file: File,
     serializer: Serializer,
     bufferSize: Int,
-    compressStream: OutputStream => OutputStream)
+    compressStream: OutputStream => OutputStream,
+    syncWrites: Boolean)
   extends BlockObjectWriter(blockId)
   with Logging
 {
@@ -97,8 +98,6 @@ class DiskBlockObjectWriter(
     override def flush() = out.flush()
   }
 
-  private val syncWrites = System.getProperty("spark.shuffle.sync", "false").toBoolean
-
   /** The file channel, used for repositioning / truncating the file. */
   private var channel: FileChannel = null
   private var bs: OutputStream = null
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
index fcd2e9798295596b48966c4b4b2529f4730a7ea3..55dcb3742c9677829a900779686142a4f3e6559d 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD
   extends PathResolver with Logging {
 
   private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
-  private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt
+  private val subDirsPerLocalDir = shuffleManager.conf.get("spark.diskStore.subDirectories", "64").toInt
 
   // Create one local directory for each path mentioned in spark.local.dir; then, inside this
   // directory, create multiple subdirectories that we will hash files into, in order to avoid
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
index 212ef6506f6f5092307be9797a091c1bff7bbed3..39dc7bb19afeed7bd87a230f7a9b9ac23221675a 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -59,12 +59,14 @@ private[spark] trait ShuffleWriterGroup {
  */
 private[spark]
 class ShuffleBlockManager(blockManager: BlockManager) {
+  def conf = blockManager.conf
+
   // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file.
   // TODO: Remove this once the shuffle file consolidation feature is stable.
   val consolidateShuffleFiles =
-    System.getProperty("spark.shuffle.consolidateFiles", "false").toBoolean
+    conf.get("spark.shuffle.consolidateFiles", "false").toBoolean
 
-  private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024
+  private val bufferSize = conf.get("spark.shuffle.file.buffer.kb", "100").toInt * 1024
 
   /**
    * Contains all the state related to a particular shuffle. This includes a pool of unused
@@ -85,8 +87,8 @@ class ShuffleBlockManager(blockManager: BlockManager) {
   type ShuffleId = Int
   private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState]
 
-  private
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup)
+  private val metadataCleaner =
+    new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf)
 
   def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = {
     new ShuffleWriterGroup {
diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
index d52b3d8284d1510b1ba6d2b3c7adf00865a6e740..40734aab49f93107b512f1affe324baaefb08509 100644
--- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
@@ -56,7 +56,7 @@ object StoragePerfTester {
 
     def writeOutputBytes(mapId: Int, total: AtomicLong) = {
       val shuffle = blockManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits,
-        new KryoSerializer())
+        new KryoSerializer(sc.conf))
       val writers = shuffle.writers
       for (i <- 1 to recordsPerMap) {
         writers(i % numOutputSplits).write(writeData)
diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
index a8db37ded14331b2b2b45d3d7e8fb90be364a4d1..729ba2c550a20bc949803aeff589e9bdd1e2b39c 100644
--- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
@@ -22,6 +22,7 @@ import akka.actor._
 import java.util.concurrent.ArrayBlockingQueue
 import util.Random
 import org.apache.spark.serializer.KryoSerializer
+import org.apache.spark.{SparkConf, SparkContext}
 
 /**
  * This class tests the BlockManager and MemoryStore for thread safety and
@@ -91,11 +92,12 @@ private[spark] object ThreadingTest {
   def main(args: Array[String]) {
     System.setProperty("spark.kryoserializer.buffer.mb", "1")
     val actorSystem = ActorSystem("test")
-    val serializer = new KryoSerializer
+    val conf = new SparkConf()
+    val serializer = new KryoSerializer(conf)
     val blockManagerMaster = new BlockManagerMaster(
-      Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))))
+      actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf)
     val blockManager = new BlockManager(
-      "<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024)
+      "<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf)
     val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i))
     val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue))
     producers.foreach(_.start)
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index f1d86c0221fea897133068e17e8bf37e1048e5c2..50dfdbdf5ae9b5aac467c533da11b587c5f8f637 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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.
@@ -32,7 +32,7 @@ import org.apache.spark.util.Utils
 /** Top level user interface for Spark */
 private[spark] class SparkUI(sc: SparkContext) extends Logging {
   val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName())
-  val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt
+  val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt
   var boundPort: Option[Int] = None
   var server: Option[Server] = None
 
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index fcd1b518d099b366e2532f700282fd1c658aee1a..6ba15187d9f637a496513f58bbc0a9b276e60bf8 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -19,7 +19,7 @@ package org.apache.spark.ui
 
 import scala.util.Random
 
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.SparkContext._
 import org.apache.spark.scheduler.SchedulingMode
 
@@ -27,25 +27,26 @@ import org.apache.spark.scheduler.SchedulingMode
 /**
  * Continuously generates jobs that expose various features of the WebUI (internal testing tool).
  *
- * Usage: ./run spark.ui.UIWorkloadGenerator [master]
+ * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]
  */
 private[spark] object UIWorkloadGenerator {
+
   val NUM_PARTITIONS = 100
   val INTER_JOB_WAIT_MS = 5000
 
   def main(args: Array[String]) {
     if (args.length < 2) {
-      println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+      println("usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
       System.exit(1)
     }
-    val master = args(0)
-    val schedulingMode = SchedulingMode.withName(args(1))
-    val appName = "Spark UI Tester"
 
+    val conf = new SparkConf().setMaster(args(0)).setAppName("Spark UI tester")
+
+    val schedulingMode = SchedulingMode.withName(args(1))
     if (schedulingMode == SchedulingMode.FAIR) {
-      System.setProperty("spark.scheduler.mode", "FAIR")
+      conf.set("spark.scheduler.mode", "FAIR")
     }
-    val sc = new SparkContext(master, appName)
+    val sc = new SparkContext(conf)
 
     def setProperties(s: String) = {
       if(schedulingMode == SchedulingMode.FAIR) {
@@ -55,11 +56,11 @@ private[spark] object UIWorkloadGenerator {
     }
 
     val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS)
-    def nextFloat() = (new Random()).nextFloat()
+    def nextFloat() = new Random().nextFloat()
 
     val jobs = Seq[(String, () => Long)](
       ("Count", baseData.count),
-      ("Cache and Count", baseData.map(x => x).cache.count),
+      ("Cache and Count", baseData.map(x => x).cache().count),
       ("Single Shuffle", baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count),
       ("Entirely failed phase", baseData.map(x => throw new Exception).count),
       ("Partially failed phase", {
diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
index c5bf2acc9ec95d1ee11ed85d7ff2cff68f5ac405..88f41be8d3dd2df802076b9c642ed6e8cf00a471 100644
--- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
@@ -48,12 +48,15 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
     def jvmTable =
       UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true)
 
-    val properties = System.getProperties.iterator.toSeq
-    val classPathProperty = properties.find { case (k, v) =>
-      k.contains("java.class.path")
+    val sparkProperties = sc.conf.getAll.sorted
+
+    val systemProperties = System.getProperties.iterator.toSeq
+    val classPathProperty = systemProperties.find { case (k, v) =>
+      k == "java.class.path"
     }.getOrElse(("", ""))
-    val sparkProperties = properties.filter(_._1.startsWith("spark")).sorted
-    val otherProperties = properties.diff(sparkProperties :+ classPathProperty).sorted
+    val otherProperties = systemProperties.filter { case (k, v) =>
+      k != "java.class.path" && !k.startsWith("spark.")
+    }.sorted
 
     val propertyHeaders = Seq("Name", "Value")
     def propertyRow(kv: (String, String)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
@@ -63,7 +66,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
       UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true)
 
     val classPathEntries = classPathProperty._2
-        .split(System.getProperty("path.separator", ":"))
+        .split(sc.conf.get("path.separator", ":"))
         .filterNot(e => e.isEmpty)
         .map(e => (e, "System Classpath"))
     val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index 07a42f05035b30d65c16df1272a93949f23cca07..b7b87250b98ee86adf9adc3abe2b6cac733f9bf9 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -33,7 +33,7 @@ import org.apache.spark.scheduler._
  */
 private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener {
   // How many stages to remember
-  val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt
+  val RETAINED_STAGES = sc.conf.get("spark.ui.retained_stages", "1000").toInt
   val DEFAULT_POOL_NAME = "default"
 
   val stageIdToPool = new HashMap[Int, String]()
@@ -61,7 +61,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
 
   override def onJobStart(jobStart: SparkListenerJobStart) {}
 
-  override def onStageCompleted(stageCompleted: StageCompleted) = synchronized {
+  override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized {
     val stage = stageCompleted.stage
     poolToActiveStages(stageIdToPool(stage.stageId)) -= stage
     activeStages -= stage
@@ -106,7 +106,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
     val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]())
     stages += stage
   }
-  
+
   override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
     val sid = taskStart.task.stageId
     val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())
@@ -146,12 +146,9 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
         // update duration
         y.taskTime += taskEnd.taskInfo.duration
 
-        taskEnd.taskMetrics.shuffleReadMetrics.foreach { shuffleRead =>
-          y.shuffleRead += shuffleRead.remoteBytesRead
-        }
-
-        taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite =>
-          y.shuffleWrite += shuffleWrite.shuffleBytesWritten
+        Option(taskEnd.taskMetrics).foreach { taskMetrics =>
+          taskMetrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead }
+          taskMetrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten }
         }
       }
       case _ => {}
diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index 1c8b51b8bc1e15f3f4e79c457beb814639a04bf2..3f009a8998cbd7ed1afdecbba4b408b3abd3cee4 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -17,11 +17,15 @@
 
 package org.apache.spark.util
 
+import scala.collection.JavaConversions.mapAsJavaMap
 import scala.concurrent.duration.{Duration, FiniteDuration}
 
 import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem}
 import com.typesafe.config.ConfigFactory
 
+import org.apache.log4j.{Level, Logger}
+import org.apache.spark.SparkConf
+
 /**
  * Various utility classes for working with Akka.
  */
@@ -37,24 +41,32 @@ private[spark] object AkkaUtils {
    * If indestructible is set to true, the Actor System will continue running in the event
    * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]].
    */
-  def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false)
-    : (ActorSystem, Int) = {
+  def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false,
+    conf: SparkConf): (ActorSystem, Int) = {
+
+    val akkaThreads   = conf.get("spark.akka.threads", "4").toInt
+    val akkaBatchSize = conf.get("spark.akka.batchSize", "15").toInt
 
-    val akkaThreads   = System.getProperty("spark.akka.threads", "4").toInt
-    val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
+    val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt
 
-    val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt
+    val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt
+    val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean
+    val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off"
+    if (!akkaLogLifecycleEvents) {
+      // As a workaround for Akka issue #3787, we coerce the "EndpointWriter" log to be silent.
+      // See: https://www.assembla.com/spaces/akka/tickets/3787#/
+      Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL))
+    }
 
-    val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
-    val lifecycleEvents =
-      if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
+    val logAkkaConfig = if (conf.get("spark.akka.logAkkaConfig", "false").toBoolean) "on" else "off"
 
-    val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt
+    val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt
     val akkaFailureDetector =
-      System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble
-    val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt
+      conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble
+    val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt
 
-    val akkaConf = ConfigFactory.parseString(
+    val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback(
+      ConfigFactory.parseString(
       s"""
       |akka.daemonic = on
       |akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
@@ -72,8 +84,11 @@ private[spark] object AkkaUtils {
       |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB
       |akka.remote.netty.tcp.execution-pool-size = $akkaThreads
       |akka.actor.default-dispatcher.throughput = $akkaBatchSize
+      |akka.log-config-on-start = $logAkkaConfig
       |akka.remote.log-remote-lifecycle-events = $lifecycleEvents
-      """.stripMargin)
+      |akka.log-dead-letters = $lifecycleEvents
+      |akka.log-dead-letters-during-shutdown = $lifecycleEvents
+      """.stripMargin))
 
     val actorSystem = if (indestructible) {
       IndestructibleActorSystem(name, akkaConf)
@@ -87,7 +102,12 @@ private[spark] object AkkaUtils {
   }
 
   /** Returns the default Spark timeout to use for Akka ask operations. */
-  def askTimeout: FiniteDuration = {
-    Duration.create(System.getProperty("spark.akka.askTimeout", "30").toLong, "seconds")
+  def askTimeout(conf: SparkConf): FiniteDuration = {
+    Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds")
+  }
+
+  /** Returns the default Spark timeout to use for Akka remote actor lookup. */
+  def lookupTimeout(conf: SparkConf): FiniteDuration = {
+    Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds")
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
index fe56960cbf952c4d7ac47969cec31fcbd0ae66ed..aa7f52cafbf37f5f3e45a286652b8308fee82216 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -18,16 +18,21 @@
 package org.apache.spark.util
 
 import java.util.{TimerTask, Timer}
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 
 
 /**
  * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries)
  */
-class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging {
+class MetadataCleaner(
+    cleanerType: MetadataCleanerType.MetadataCleanerType,
+    cleanupFunc: (Long) => Unit,
+    conf: SparkConf)
+  extends Logging
+{
   val name = cleanerType.toString
 
-  private val delaySeconds = MetadataCleaner.getDelaySeconds(cleanerType)
+  private val delaySeconds = MetadataCleaner.getDelaySeconds(conf, cleanerType)
   private val periodSeconds = math.max(10, delaySeconds / 10)
   private val timer = new Timer(name + " cleanup timer", true)
 
@@ -65,22 +70,28 @@ object MetadataCleanerType extends Enumeration {
   def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString
 }
 
+// TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the
+// initialization of StreamingContext. It's okay for users trying to configure stuff themselves.
 object MetadataCleaner {
+  def getDelaySeconds(conf: SparkConf) = {
+    conf.get("spark.cleaner.ttl", "3500").toInt
+  }
 
-  // using only sys props for now : so that workers can also get to it while preserving earlier behavior.
-  def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt
-
-  def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = {
-    System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt
+  def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int =
+  {
+    conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString)
+      .toInt
   }
 
-  def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) {
-    System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString)
+  def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType,
+      delay: Int)
+  {
+    conf.set(MetadataCleanerType.systemProperty(cleanerType),  delay.toString)
   }
 
-  def setDelaySeconds(delay: Int, resetAll: Boolean = true) {
+  def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) {
     // override for all ?
-    System.setProperty("spark.cleaner.ttl", delay.toString)
+    conf.set("spark.cleaner.ttl", delay.toString)
     if (resetAll) {
       for (cleanerType <- MetadataCleanerType.values) {
         System.clearProperty(MetadataCleanerType.systemProperty(cleanerType))
diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala
new file mode 100644
index 0000000000000000000000000000000000000000..8b4e7c104cb19424f1868c4e48fc5baa3880c34b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.util
+
+import java.io.{Externalizable, ObjectOutput, ObjectInput}
+import com.clearspring.analytics.stream.cardinality.{ICardinality, HyperLogLog}
+
+/**
+ * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is serializable.
+ */
+private[spark]
+class SerializableHyperLogLog(var value: ICardinality) extends Externalizable {
+
+  def this() = this(null)  // For deserialization
+
+  def merge(other: SerializableHyperLogLog) = new SerializableHyperLogLog(value.merge(other.value))
+
+  def add[T](elem: T) = {
+    this.value.offer(elem)
+    this
+  }
+
+  def readExternal(in: ObjectInput) {
+    val byteLength = in.readInt()
+    val bytes = new Array[Byte](byteLength)
+    in.readFully(bytes)
+    value = HyperLogLog.Builder.build(bytes)
+  }
+
+  def writeExternal(out: ObjectOutput) {
+    val bytes = value.getBytes()
+    out.writeInt(bytes.length)
+    out.write(bytes)
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
index a25b37a2a96a9eb2eb567c2cfadad22e7e062582..bddb3bb7350bc43af2b958f15ddc5f523de3eb4a 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -30,10 +30,10 @@ import java.lang.management.ManagementFactory
 import scala.collection.mutable.ArrayBuffer
 
 import it.unimi.dsi.fastutil.ints.IntOpenHashSet
-import org.apache.spark.Logging
+import org.apache.spark.{SparkEnv, SparkConf, SparkContext, Logging}
 
 /**
- * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in 
+ * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in
  * memory-aware caches.
  *
  * Based on the following JavaWorld article:
@@ -89,9 +89,11 @@ private[spark] object SizeEstimator extends Logging {
     classInfos.put(classOf[Object], new ClassInfo(objectSize, Nil))
   }
 
-  private def getIsCompressedOops : Boolean = {
+  private def getIsCompressedOops: Boolean = {
+    // This is only used by tests to override the detection of compressed oops. The test
+    // actually uses a system property instead of a SparkConf, so we'll stick with that.
     if (System.getProperty("spark.test.useCompressedOops") != null) {
-      return System.getProperty("spark.test.useCompressedOops").toBoolean 
+      return System.getProperty("spark.test.useCompressedOops").toBoolean
     }
 
     try {
@@ -103,7 +105,7 @@ private[spark] object SizeEstimator extends Logging {
       val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption",
           Class.forName("java.lang.String"))
 
-      val bean = ManagementFactory.newPlatformMXBeanProxy(server, 
+      val bean = ManagementFactory.newPlatformMXBeanProxy(server,
         hotSpotMBeanName, hotSpotMBeanClass)
       // TODO: We could use reflection on the VMOption returned ?
       return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true")
@@ -251,7 +253,7 @@ private[spark] object SizeEstimator extends Logging {
     if (info != null) {
       return info
     }
-    
+
     val parent = getClassInfo(cls.getSuperclass)
     var shellSize = parent.shellSize
     var pointerFields = parent.pointerFields
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 3f7858d2dedc311cdbf70017d10d0c17cd173dbc..5f1253100b33888834b65f2d6f238412b65dded8 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -36,14 +36,13 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
 import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
 import org.apache.spark.deploy.SparkHadoopUtil
 import java.nio.ByteBuffer
-import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.{SparkConf, SparkContext, SparkException, Logging}
 
 
 /**
  * Various utility methods used by Spark.
  */
 private[spark] object Utils extends Logging {
-
   /** Serialize an object using Java serialization */
   def serialize[T](o: T): Array[Byte] = {
     val bos = new ByteArrayOutputStream()
@@ -239,9 +238,9 @@ private[spark] object Utils extends Logging {
    * Throws SparkException if the target file already exists and has different contents than
    * the requested file.
    */
-  def fetchFile(url: String, targetDir: File) {
+  def fetchFile(url: String, targetDir: File, conf: SparkConf) {
     val filename = url.split("/").last
-    val tempDir = getLocalDir
+    val tempDir = getLocalDir(conf)
     val tempFile =  File.createTempFile("fetchFileTemp", null, new File(tempDir))
     val targetFile = new File(targetDir, filename)
     val uri = new URI(url)
@@ -311,8 +310,8 @@ private[spark] object Utils extends Logging {
    * return a single directory, even though the spark.local.dir property might be a list of
    * multiple paths.
    */
-  def getLocalDir: String = {
-    System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0)
+  def getLocalDir(conf: SparkConf): String = {
+    conf.get("spark.local.dir",  System.getProperty("java.io.tmpdir")).split(',')(0)
   }
 
   /**
@@ -397,13 +396,12 @@ private[spark] object Utils extends Logging {
     InetAddress.getByName(address).getHostName
   }
 
-  def localHostPort(): String = {
-    val retval = System.getProperty("spark.hostPort", null)
+  def localHostPort(conf: SparkConf): String = {
+    val retval = conf.get("spark.hostPort", null)
     if (retval == null) {
       logErrorWithStack("spark.hostPort not set but invoking localHostPort")
       return localHostName()
     }
-
     retval
   }
 
@@ -415,9 +413,12 @@ private[spark] object Utils extends Logging {
     assert(hostPort.indexOf(':') != -1, message)
   }
 
-  // Used by DEBUG code : remove when all testing done
   def logErrorWithStack(msg: String) {
-    try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
+    try {
+      throw new Exception
+    } catch {
+      case ex: Exception => logError(msg, ex)
+    }
   }
 
   // Typically, this will be of order of number of nodes in cluster
@@ -837,7 +838,7 @@ private[spark] object Utils extends Logging {
     }
   }
 
-  /** 
+  /**
    * Timing method based on iterations that permit JVM JIT optimization.
    * @param numIters number of iterations
    * @param f function to be executed
diff --git a/core/src/test/resources/spark.conf b/core/src/test/resources/spark.conf
new file mode 100644
index 0000000000000000000000000000000000000000..aa4e7512354d30506a80c5674056732620d8ad34
--- /dev/null
+++ b/core/src/test/resources/spark.conf
@@ -0,0 +1,8 @@
+# A simple spark.conf file used only in our unit tests
+
+spark.test.intTestProperty = 1
+
+spark.test {
+  stringTestProperty = "hi"
+  listTestProperty = ["a", "b"]
+}
diff --git a/core/src/test/resources/uncommons-maths-1.2.2.jar b/core/src/test/resources/uncommons-maths-1.2.2.jar
deleted file mode 100644
index e126001c1c270aa1b149970d07c53dbe0d13514e..0000000000000000000000000000000000000000
Binary files a/core/src/test/resources/uncommons-maths-1.2.2.jar and /dev/null differ
diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
index f25d921d3f87faa004975500e56c8b96416c3a13..ec13b329b25a8534078e5a55da8fe25801f3e0ca 100644
--- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
@@ -26,8 +26,6 @@ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId}
 import org.apache.spark.util.Utils
 
 class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
-  initLogging()
-
   var checkpointDir: File = _
   val partitioner = new HashPartitioner(2)
 
@@ -57,15 +55,15 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
   }
 
   test("RDDs with one-to-one dependencies") {
-    testCheckpointing(_.map(x => x.toString))
-    testCheckpointing(_.flatMap(x => 1 to x))
-    testCheckpointing(_.filter(_ % 2 == 0))
-    testCheckpointing(_.sample(false, 0.5, 0))
-    testCheckpointing(_.glom())
-    testCheckpointing(_.mapPartitions(_.map(_.toString)))
-    testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
-    testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
-    testCheckpointing(_.pipe(Seq("cat")))
+    testRDD(_.map(x => x.toString))
+    testRDD(_.flatMap(x => 1 to x))
+    testRDD(_.filter(_ % 2 == 0))
+    testRDD(_.sample(false, 0.5, 0))
+    testRDD(_.glom())
+    testRDD(_.mapPartitions(_.map(_.toString)))
+    testRDD(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
+    testRDD(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
+    testRDD(_.pipe(Seq("cat")))
   }
 
   test("ParallelCollection") {
@@ -97,7 +95,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
   }
 
   test("ShuffledRDD") {
-    testCheckpointing(rdd => {
+    testRDD(rdd => {
       // Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD
       new ShuffledRDD[Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner)
     })
@@ -105,25 +103,17 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
 
   test("UnionRDD") {
     def otherRDD = sc.makeRDD(1 to 10, 1)
-
-    // Test whether the size of UnionRDDPartitions reduce in size after parent RDD is checkpointed.
-    // Current implementation of UnionRDD has transient reference to parent RDDs,
-    // so only the partitions will reduce in serialized size, not the RDD.
-    testCheckpointing(_.union(otherRDD), false, true)
-    testParentCheckpointing(_.union(otherRDD), false, true)
+    testRDD(_.union(otherRDD))
+    testRDDPartitions(_.union(otherRDD))
   }
 
   test("CartesianRDD") {
     def otherRDD = sc.makeRDD(1 to 10, 1)
-    testCheckpointing(new CartesianRDD(sc, _, otherRDD))
-
-    // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
-    // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
-    // so only the RDD will reduce in serialized size, not the partitions.
-    testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false)
+    testRDD(new CartesianRDD(sc, _, otherRDD))
+    testRDDPartitions(new CartesianRDD(sc, _, otherRDD))
 
     // Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after
-    // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
+    // the parent RDD has been checkpointed and parent partitions have been changed.
     // Note that this test is very specific to the current implementation of CartesianRDD.
     val ones = sc.makeRDD(1 to 100, 10).map(x => x)
     ones.checkpoint() // checkpoint that MappedRDD
@@ -134,23 +124,20 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
     val splitAfterCheckpoint =
       serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition])
     assert(
-      (splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) &&
-        (splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2),
-      "CartesianRDD.parents not updated after parent RDD checkpointed"
+      (splitAfterCheckpoint.s1.getClass != splitBeforeCheckpoint.s1.getClass) &&
+        (splitAfterCheckpoint.s2.getClass != splitBeforeCheckpoint.s2.getClass),
+      "CartesianRDD.s1 and CartesianRDD.s2 not updated after parent RDD is checkpointed"
     )
   }
 
   test("CoalescedRDD") {
-    testCheckpointing(_.coalesce(2))
-
-    // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
-    // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
-    // so only the RDD will reduce in serialized size, not the partitions.
-    testParentCheckpointing(_.coalesce(2), true, false)
+    testRDD(_.coalesce(2))
+    testRDDPartitions(_.coalesce(2))
 
-    // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents) after
-    // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
-    // Note that this test is very specific to the current implementation of CoalescedRDDPartitions
+    // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents)
+    // after the parent RDD has been checkpointed and parent partitions have been changed.
+    // Note that this test is very specific to the current implementation of
+    // CoalescedRDDPartitions.
     val ones = sc.makeRDD(1 to 100, 10).map(x => x)
     ones.checkpoint() // checkpoint that MappedRDD
     val coalesced = new CoalescedRDD(ones, 2)
@@ -160,33 +147,78 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
     val splitAfterCheckpoint =
       serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition])
     assert(
-      splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head,
-      "CoalescedRDDPartition.parents not updated after parent RDD checkpointed"
+      splitAfterCheckpoint.parents.head.getClass != splitBeforeCheckpoint.parents.head.getClass,
+      "CoalescedRDDPartition.parents not updated after parent RDD is checkpointed"
     )
   }
 
   test("CoGroupedRDD") {
-    val longLineageRDD1 = generateLongLineageRDDForCoGroupedRDD()
-    testCheckpointing(rdd => {
+    val longLineageRDD1 = generateFatPairRDD()
+    testRDD(rdd => {
       CheckpointSuite.cogroup(longLineageRDD1, rdd.map(x => (x % 2, 1)), partitioner)
-    }, false, true)
+    })
 
-    val longLineageRDD2 = generateLongLineageRDDForCoGroupedRDD()
-    testParentCheckpointing(rdd => {
+    val longLineageRDD2 = generateFatPairRDD()
+    testRDDPartitions(rdd => {
       CheckpointSuite.cogroup(
         longLineageRDD2, sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)), partitioner)
-    }, false, true)
+    })
   }
 
   test("ZippedRDD") {
-    testCheckpointing(
-      rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
-
-    // Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed
-    // Current implementation of ZippedRDDPartitions has transient references to parent RDDs,
-    // so only the RDD will reduce in serialized size, not the partitions.
-    testParentCheckpointing(
-      rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
+    testRDD(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)))
+    testRDDPartitions(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)))
+
+    // Test that the ZippedPartition updates parent partitions
+    // after the parent RDD has been checkpointed and parent partitions have been changed.
+    // Note that this test is very specific to the current implementation of ZippedRDD.
+    val rdd = generateFatRDD()
+    val zippedRDD = new ZippedRDD(sc, rdd, rdd.map(x => x))
+    zippedRDD.rdd1.checkpoint()
+    zippedRDD.rdd2.checkpoint()
+    val partitionBeforeCheckpoint =
+      serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]])
+    zippedRDD.count()
+    val partitionAfterCheckpoint =
+      serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]])
+    assert(
+      partitionAfterCheckpoint.partition1.getClass != partitionBeforeCheckpoint.partition1.getClass &&
+        partitionAfterCheckpoint.partition2.getClass != partitionBeforeCheckpoint.partition2.getClass,
+      "ZippedRDD.partition1 and ZippedRDD.partition2 not updated after parent RDD is checkpointed"
+    )
+  }
+
+  test("PartitionerAwareUnionRDD") {
+    testRDD(rdd => {
+      new PartitionerAwareUnionRDD[(Int, Int)](sc, Array(
+        generateFatPairRDD(),
+        rdd.map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
+      ))
+    })
+
+    testRDDPartitions(rdd => {
+      new PartitionerAwareUnionRDD[(Int, Int)](sc, Array(
+        generateFatPairRDD(),
+        rdd.map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
+      ))
+    })
+
+    // Test that the PartitionerAwareUnionRDD updates parent partitions
+    // (PartitionerAwareUnionRDD.parents) after the parent RDD has been checkpointed and parent
+    // partitions have been changed. Note that this test is very specific to the current
+    // implementation of PartitionerAwareUnionRDD.
+    val pairRDD = generateFatPairRDD()
+    pairRDD.checkpoint()
+    val unionRDD = new PartitionerAwareUnionRDD(sc, Array(pairRDD))
+    val partitionBeforeCheckpoint =  serializeDeserialize(
+      unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition])
+    pairRDD.count()
+    val partitionAfterCheckpoint =  serializeDeserialize(
+      unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition])
+    assert(
+      partitionBeforeCheckpoint.parents.head.getClass != partitionAfterCheckpoint.parents.head.getClass,
+      "PartitionerAwareUnionRDDPartition.parents not updated after parent RDD is checkpointed"
+    )
   }
 
   test("CheckpointRDD with zero partitions") {
@@ -200,29 +232,32 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
   }
 
   /**
-   * Test checkpointing of the final RDD generated by the given operation. By default,
-   * this method tests whether the size of serialized RDD has reduced after checkpointing or not.
-   * It can also test whether the size of serialized RDD partitions has reduced after checkpointing or
-   * not, but this is not done by default as usually the partitions do not refer to any RDD and
-   * therefore never store the lineage.
+   * Test checkpointing of the RDD generated by the given operation. It tests whether the
+   * serialized size of the RDD is reduce after checkpointing or not. This function should be called
+   * on all RDDs that have a parent RDD (i.e., do not call on ParallelCollection, BlockRDD, etc.).
    */
-  def testCheckpointing[U: ClassTag](
-      op: (RDD[Int]) => RDD[U],
-      testRDDSize: Boolean = true,
-      testRDDPartitionSize: Boolean = false
-    ) {
+  def testRDD[U: ClassTag](op: (RDD[Int]) => RDD[U]) {
     // Generate the final RDD using given RDD operation
-    val baseRDD = generateLongLineageRDD()
+    val baseRDD = generateFatRDD()
     val operatedRDD = op(baseRDD)
     val parentRDD = operatedRDD.dependencies.headOption.orNull
     val rddType = operatedRDD.getClass.getSimpleName
     val numPartitions = operatedRDD.partitions.length
 
+    // Force initialization of all the data structures in RDDs
+    // Without this, serializing the RDD will give a wrong estimate of the size of the RDD
+    initializeRdd(operatedRDD)
+
+    val partitionsBeforeCheckpoint = operatedRDD.partitions
+
     // Find serialized sizes before and after the checkpoint
-    val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+    logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
+    val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
     operatedRDD.checkpoint()
     val result = operatedRDD.collect()
-    val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+    operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables
+    val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+    logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
 
     // Test whether the checkpoint file has been created
     assert(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get).collect() === result)
@@ -230,6 +265,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
     // Test whether dependencies have been changed from its earlier parent RDD
     assert(operatedRDD.dependencies.head.rdd != parentRDD)
 
+    // Test whether the partitions have been changed from its earlier partitions
+    assert(operatedRDD.partitions.toList != partitionsBeforeCheckpoint.toList)
+
     // Test whether the partitions have been changed to the new Hadoop partitions
     assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList)
 
@@ -239,122 +277,72 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
     // Test whether the data in the checkpointed RDD is same as original
     assert(operatedRDD.collect() === result)
 
-    // Test whether serialized size of the RDD has reduced. If the RDD
-    // does not have any dependency to another RDD (e.g., ParallelCollection,
-    // ShuffleRDD with ShuffleDependency), it may not reduce in size after checkpointing.
-    if (testRDDSize) {
-      logInfo("Size of " + rddType +
-        "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]")
-      assert(
-        rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
-        "Size of " + rddType + " did not reduce after checkpointing " +
-          "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
-      )
-    }
+    // Test whether serialized size of the RDD has reduced.
+    logInfo("Size of " + rddType +
+      " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]")
+    assert(
+      rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
+      "Size of " + rddType + " did not reduce after checkpointing " +
+        " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
+    )
 
-    // Test whether serialized size of the partitions has reduced. If the partitions
-    // do not have any non-transient reference to another RDD or another RDD's partitions, it
-    // does not refer to a lineage and therefore may not reduce in size after checkpointing.
-    // However, if the original partitions before checkpointing do refer to a parent RDD, the partitions
-    // must be forgotten after checkpointing (to remove all reference to parent RDDs) and
-    // replaced with the HadooPartitions of the checkpointed RDD.
-    if (testRDDPartitionSize) {
-      logInfo("Size of " + rddType + " partitions "
-        + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]")
-      assert(
-        splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
-        "Size of " + rddType + " partitions did not reduce after checkpointing " +
-          "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
-      )
-    }
   }
 
   /**
    * Test whether checkpointing of the parent of the generated RDD also
    * truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent
    * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed,
-   * this RDD will remember the partitions and therefore potentially the whole lineage.
+   * the generated RDD will remember the partitions and therefore potentially the whole lineage.
+   * This function should be called only those RDD whose partitions refer to parent RDD's
+   * partitions (i.e., do not call it on simple RDD like MappedRDD).
+   *
    */
-  def testParentCheckpointing[U: ClassTag](
-      op: (RDD[Int]) => RDD[U],
-      testRDDSize: Boolean,
-      testRDDPartitionSize: Boolean
-    ) {
+  def testRDDPartitions[U: ClassTag](op: (RDD[Int]) => RDD[U]) {
     // Generate the final RDD using given RDD operation
-    val baseRDD = generateLongLineageRDD()
+    val baseRDD = generateFatRDD()
     val operatedRDD = op(baseRDD)
-    val parentRDD = operatedRDD.dependencies.head.rdd
+    val parentRDDs = operatedRDD.dependencies.map(_.rdd)
     val rddType = operatedRDD.getClass.getSimpleName
-    val parentRDDType = parentRDD.getClass.getSimpleName
 
-    // Get the partitions and dependencies of the parent in case they're lazily computed
-    parentRDD.dependencies
-    parentRDD.partitions
+    // Force initialization of all the data structures in RDDs
+    // Without this, serializing the RDD will give a wrong estimate of the size of the RDD
+    initializeRdd(operatedRDD)
 
     // Find serialized sizes before and after the checkpoint
-    val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
-    parentRDD.checkpoint()  // checkpoint the parent RDD, not the generated one
-    val result = operatedRDD.collect()
-    val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+    logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
+    val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+    parentRDDs.foreach(_.checkpoint())  // checkpoint the parent RDD, not the generated one
+    val result = operatedRDD.collect()  // force checkpointing
+    operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables
+    val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+    logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
 
     // Test whether the data in the checkpointed RDD is same as original
     assert(operatedRDD.collect() === result)
 
-    // Test whether serialized size of the RDD has reduced because of its parent being
-    // checkpointed. If this RDD or its parent RDD do not have any dependency
-    // to another RDD (e.g., ParallelCollection, ShuffleRDD with ShuffleDependency), it may
-    // not reduce in size after checkpointing.
-    if (testRDDSize) {
-      assert(
-        rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
-        "Size of " + rddType + " did not reduce after checkpointing parent " + parentRDDType +
-          "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
-      )
-    }
-
-    // Test whether serialized size of the partitions has reduced because of its parent being
-    // checkpointed. If the partitions do not have any non-transient reference to another RDD
-    // or another RDD's partitions, it does not refer to a lineage and therefore may not reduce
-    // in size after checkpointing. However, if the partitions do refer to the *partitions* of a parent
-    // RDD, then these partitions must update reference to the parent RDD partitions as the parent RDD's
-    // partitions must have changed after checkpointing.
-    if (testRDDPartitionSize) {
-      assert(
-        splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
-        "Size of " + rddType + " partitions did not reduce after checkpointing parent " + parentRDDType +
-          "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
-      )
-    }
-
+    // Test whether serialized size of the partitions has reduced
+    logInfo("Size of partitions of " + rddType +
+      " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]")
+    assert(
+      partitionSizeAfterCheckpoint < partitionSizeBeforeCheckpoint,
+      "Size of " + rddType + " partitions did not reduce after checkpointing parent RDDs" +
+        " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]"
+    )
   }
 
   /**
-   * Generate an RDD with a long lineage of one-to-one dependencies.
+   * Generate an RDD such that both the RDD and its partitions have large size.
    */
-  def generateLongLineageRDD(): RDD[Int] = {
-    var rdd = sc.makeRDD(1 to 100, 4)
-    for (i <- 1 to 50) {
-      rdd = rdd.map(x => x + 1)
-    }
-    rdd
+  def generateFatRDD(): RDD[Int] = {
+    new FatRDD(sc.makeRDD(1 to 100, 4)).map(x => x)
   }
 
   /**
-   * Generate an RDD with a long lineage specifically for CoGroupedRDD.
-   * A CoGroupedRDD can have a long lineage only one of its parents have a long lineage
-   * and narrow dependency with this RDD. This method generate such an RDD by a sequence
-   * of cogroups and mapValues which creates a long lineage of narrow dependencies.
+   * Generate an pair RDD (with partitioner) such that both the RDD and its partitions
+   * have large size.
    */
-  def generateLongLineageRDDForCoGroupedRDD() = {
-    val add = (x: (Seq[Int], Seq[Int])) => (x._1 ++ x._2).reduce(_ + _)
-
-    def ones: RDD[(Int, Int)] = sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
-
-    var cogrouped: RDD[(Int, (Seq[Int], Seq[Int]))] = ones.cogroup(ones)
-    for(i <- 1 to 10) {
-      cogrouped = cogrouped.mapValues(add).cogroup(ones)
-    }
-    cogrouped.mapValues(add)
+  def generateFatPairRDD() = {
+    new FatPairRDD(sc.makeRDD(1 to 100, 4), partitioner).mapValues(x => x)
   }
 
   /**
@@ -362,8 +350,26 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
    * upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint.
    */
   def getSerializedSizes(rdd: RDD[_]): (Int, Int) = {
-    (Utils.serialize(rdd).length - Utils.serialize(rdd.checkpointData).length,
-     Utils.serialize(rdd.partitions).length)
+    val rddSize = Utils.serialize(rdd).size
+    val rddCpDataSize = Utils.serialize(rdd.checkpointData).size
+    val rddPartitionSize = Utils.serialize(rdd.partitions).size
+    val rddDependenciesSize = Utils.serialize(rdd.dependencies).size
+
+    // Print detailed size, helps in debugging
+    logInfo("Serialized sizes of " + rdd +
+      ": RDD = " + rddSize +
+      ", RDD checkpoint data = " + rddCpDataSize +
+      ", RDD partitions = " + rddPartitionSize +
+      ", RDD dependencies = " + rddDependenciesSize
+    )
+    // this makes sure that serializing the RDD's checkpoint data does not
+    // serialize the whole RDD as well
+    assert(
+      rddSize > rddCpDataSize,
+      "RDD's checkpoint data (" + rddCpDataSize  + ") is equal or larger than the " +
+        "whole RDD with checkpoint data (" + rddSize + ")"
+    )
+    (rddSize - rddCpDataSize, rddPartitionSize)
   }
 
   /**
@@ -375,8 +381,49 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
     val bytes = Utils.serialize(obj)
     Utils.deserialize[T](bytes)
   }
+
+  /**
+   * Recursively force the initialization of the all members of an RDD and it parents.
+   */
+  def initializeRdd(rdd: RDD[_]) {
+    rdd.partitions // forces the
+    rdd.dependencies.map(_.rdd).foreach(initializeRdd(_))
+  }
 }
 
+/** RDD partition that has large serialized size. */
+class FatPartition(val partition: Partition) extends Partition {
+  val bigData = new Array[Byte](10000)
+  def index: Int = partition.index
+}
+
+/** RDD that has large serialized size. */
+class FatRDD(parent: RDD[Int]) extends RDD[Int](parent) {
+  val bigData = new Array[Byte](100000)
+
+  protected def getPartitions: Array[Partition] = {
+    parent.partitions.map(p => new FatPartition(p))
+  }
+
+  def compute(split: Partition, context: TaskContext): Iterator[Int] = {
+    parent.compute(split.asInstanceOf[FatPartition].partition, context)
+  }
+}
+
+/** Pair RDD that has large serialized size. */
+class FatPairRDD(parent: RDD[Int], _partitioner: Partitioner) extends RDD[(Int, Int)](parent) {
+  val bigData = new Array[Byte](100000)
+
+  protected def getPartitions: Array[Partition] = {
+    parent.partitions.map(p => new FatPartition(p))
+  }
+
+  @transient override val partitioner = Some(_partitioner)
+
+  def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = {
+    parent.compute(split.asInstanceOf[FatPartition].partition, context).map(x => (x, x))
+  }
+}
 
 object CheckpointSuite {
   // This is a custom cogroup function that does not use mapValues like
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index 6d1695eae74befd8a652e7b5f26d4dc3e61bbacd..fb89537258542245c0d0584f3d69c33f9204a875 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -30,13 +30,15 @@ import org.apache.spark.util.Utils
 
 class DriverSuite extends FunSuite with Timeouts {
   test("driver should exit after finishing") {
-    assert(System.getenv("SPARK_HOME") != null)
+    val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get
     // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing"
     val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
     forAll(masters) { (master: String) =>
       failAfter(60 seconds) {
-        Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master),
-          new File(System.getenv("SPARK_HOME")))
+        Utils.executeAndGetOutput(
+          Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
+          new File(sparkHome), 
+          Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
       }
     }
   }
diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
index c210dd5c3b4e2093aba8b1fcb522dc173041b750..a2eb9a4e84696c9326c5523e673add86b9d620d8 100644
--- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
@@ -17,33 +17,49 @@
 
 package org.apache.spark
 
+import java.io._
+import java.util.jar.{JarEntry, JarOutputStream}
+
+import SparkContext._
 import com.google.common.io.Files
 import org.scalatest.FunSuite
-import java.io.{File, PrintWriter, FileReader, BufferedReader}
-import SparkContext._
 
 class FileServerSuite extends FunSuite with LocalSparkContext {
 
   @transient var tmpFile: File = _
-  @transient var testJarFile: File = _
-
-  override def beforeEach() {
-    super.beforeEach()
-    // Create a sample text file
-    val tmpdir = new File(Files.createTempDir(), "test")
-    tmpdir.mkdir()
-    tmpFile = new File(tmpdir, "FileServerSuite.txt")
-    val pw = new PrintWriter(tmpFile)
+  @transient var tmpJarUrl: String = _
+
+  override def beforeAll() {
+    super.beforeAll()
+    val tmpDir = new File(Files.createTempDir(), "test")
+    tmpDir.mkdir()
+
+    val textFile = new File(tmpDir, "FileServerSuite.txt")
+    val pw = new PrintWriter(textFile)
     pw.println("100")
     pw.close()
-  }
+    
+    val jarFile = new File(tmpDir, "test.jar")
+    val jarStream = new FileOutputStream(jarFile)
+    val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest())
 
-  override def afterEach() {
-    super.afterEach()
-    // Clean up downloaded file
-    if (tmpFile.exists) {
-      tmpFile.delete()
+    val jarEntry = new JarEntry(textFile.getName)
+    jar.putNextEntry(jarEntry)
+    
+    val in = new FileInputStream(textFile)
+    val buffer = new Array[Byte](10240)
+    var nRead = 0
+    while (nRead <= 0) {
+      nRead = in.read(buffer, 0, buffer.length)
+      jar.write(buffer, 0, nRead)
     }
+
+    in.close()
+    jar.close()
+    jarStream.close()
+
+    tmpFile = textFile
+    tmpJarUrl = jarFile.toURI.toURL.toString
   }
 
   test("Distributing files locally") {
@@ -77,18 +93,13 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
 
   test ("Dynamically adding JARS locally") {
     sc = new SparkContext("local[4]", "test")
-    val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile()
-    sc.addJar(sampleJarFile)
-    val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0))
-    val result = sc.parallelize(testData).reduceByKey { (x,y) =>
-      val fac = Thread.currentThread.getContextClassLoader()
-                                    .loadClass("org.uncommons.maths.Maths")
-                                    .getDeclaredMethod("factorial", classOf[Int])
-      val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
-      val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
-      a + b
-    }.collect()
-    assert(result.toSet === Set((1,2), (2,7), (3,121)))
+    sc.addJar(tmpJarUrl)
+    val testData = Array((1, 1))
+    sc.parallelize(testData).foreach { x =>
+      if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) {
+        throw new SparkException("jar not added")
+      }
+    }
   }
 
   test("Distributing files on a standalone cluster") {
@@ -107,33 +118,24 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
 
   test ("Dynamically adding JARS on a standalone cluster") {
     sc = new SparkContext("local-cluster[1,1,512]", "test")
-    val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile()
-    sc.addJar(sampleJarFile)
-    val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0))
-    val result = sc.parallelize(testData).reduceByKey { (x,y) =>
-      val fac = Thread.currentThread.getContextClassLoader()
-                                    .loadClass("org.uncommons.maths.Maths")
-                                    .getDeclaredMethod("factorial", classOf[Int])
-      val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
-      val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
-      a + b
-    }.collect()
-    assert(result.toSet === Set((1,2), (2,7), (3,121)))
+    sc.addJar(tmpJarUrl)
+    val testData = Array((1,1))
+    sc.parallelize(testData).foreach { x =>
+      if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) {
+        throw new SparkException("jar not added")
+      }
+    }
   }
 
   test ("Dynamically adding JARS on a standalone cluster using local: URL") {
     sc = new SparkContext("local-cluster[1,1,512]", "test")
-    val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile()
-    sc.addJar(sampleJarFile.replace("file", "local"))
-    val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0))
-    val result = sc.parallelize(testData).reduceByKey { (x,y) =>
-      val fac = Thread.currentThread.getContextClassLoader()
-                                    .loadClass("org.uncommons.maths.Maths")
-                                    .getDeclaredMethod("factorial", classOf[Int])
-      val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
-      val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
-      a + b
-    }.collect()
-    assert(result.toSet === Set((1,2), (2,7), (3,121)))
+    sc.addJar(tmpJarUrl.replace("file", "local"))
+    val testData = Array((1,1))
+    sc.parallelize(testData).foreach { x =>
+      if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) {
+        throw new SparkException("jar not added")
+      }
+    }
   }
+
 }
diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
index 5e2899c97b3f81d5d3d20105420b9a95c33af983..23ec6c3b311f0a78e58ed7f2b0a1387418a53f0f 100644
--- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
@@ -930,4 +930,36 @@ public class JavaAPISuite implements Serializable {
                         parts[1]);
   }
 
+  @Test
+  public void countApproxDistinct() {
+    List<Integer> arrayData = new ArrayList<Integer>();
+    int size = 100;
+    for (int i = 0; i < 100000; i++) {
+      arrayData.add(i % size);
+    }
+    JavaRDD<Integer> simpleRdd = sc.parallelize(arrayData, 10);
+    Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.2) - size) / (size * 1.0)) < 0.2);
+    Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.05);
+    Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.01) - size) / (size * 1.0)) <= 0.01);
+  }
+
+  @Test
+  public void countApproxDistinctByKey() {
+    double relativeSD = 0.001;
+
+    List<Tuple2<Integer, Integer>> arrayData = new ArrayList<Tuple2<Integer, Integer>>();
+    for (int i = 10; i < 100; i++)
+      for (int j = 0; j < i; j++)
+        arrayData.add(new Tuple2<Integer, Integer>(i, j));
+
+    JavaPairRDD<Integer, Integer> pairRdd = sc.parallelizePairs(arrayData);
+    List<Tuple2<Integer, Object>> res =  pairRdd.countApproxDistinctByKey(relativeSD).collect();
+    for (Tuple2<Integer, Object> resItem : res) {
+      double count = (double)resItem._1();
+      Long resCount = (Long)resItem._2();
+      Double error = Math.abs((resCount - count) / count);
+      Assert.assertTrue(error < relativeSD);
+    }
+
+  }
 }
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index 271dc905bc386d6dcc2313c759d934bdf6f5ab2e..afc1beff989c4d47a5181e8b22b6fe4385f14d3d 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -23,9 +23,10 @@ import akka.actor._
 import org.apache.spark.scheduler.MapStatus
 import org.apache.spark.storage.BlockManagerId
 import org.apache.spark.util.AkkaUtils
+import scala.concurrent.Await
 
 class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
-
+  private val conf = new SparkConf
   test("compressSize") {
     assert(MapOutputTracker.compressSize(0L) === 0)
     assert(MapOutputTracker.compressSize(1L) === 1)
@@ -48,15 +49,15 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
 
   test("master start and stop") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTrackerMaster()
-    tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))))
+    val tracker = new MapOutputTrackerMaster(conf)
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
     tracker.stop()
   }
 
   test("master register and fetch") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTrackerMaster()
-    tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))))
+    val tracker = new MapOutputTrackerMaster(conf)
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
     tracker.registerShuffle(10, 2)
     val compressedSize1000 = MapOutputTracker.compressSize(1000L)
     val compressedSize10000 = MapOutputTracker.compressSize(10000L)
@@ -74,8 +75,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
 
   test("master register and unregister and fetch") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTrackerMaster()
-    tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))))
+    val tracker = new MapOutputTrackerMaster(conf)
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
     tracker.registerShuffle(10, 2)
     val compressedSize1000 = MapOutputTracker.compressSize(1000L)
     val compressedSize10000 = MapOutputTracker.compressSize(10000L)
@@ -96,18 +97,20 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
 
   test("remote fetch") {
     val hostname = "localhost"
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0)
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf)
     System.setProperty("spark.driver.port", boundPort.toString)    // Will be cleared by LocalSparkContext
     System.setProperty("spark.hostPort", hostname + ":" + boundPort)
 
-    val masterTracker = new MapOutputTrackerMaster()
-    masterTracker.trackerActor = Left(actorSystem.actorOf(
-        Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker"))
+    val masterTracker = new MapOutputTrackerMaster(conf)
+    masterTracker.trackerActor = actorSystem.actorOf(
+        Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")
 
-    val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0)
-    val slaveTracker = new MapOutputTracker()
-    slaveTracker.trackerActor = Right(slaveSystem.actorSelection(
-        "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker"))
+    val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf)
+    val slaveTracker = new MapOutputTracker(conf)
+    val selection = slaveSystem.actorSelection(
+      s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
+    val timeout = AkkaUtils.lookupTimeout(conf)
+    slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
 
     masterTracker.registerShuffle(10, 1)
     masterTracker.incrementEpoch()
diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
index 288aa14eeb03b2bcc7be42bd751716260bb90a11..c650ef4ed58cb3b015accb3951c90f7bc88d5c41 100644
--- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
@@ -27,8 +27,10 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite =>
 
   def sc: SparkContext = _sc
 
+  var conf = new SparkConf(false)
+
   override def beforeAll() {
-    _sc = new SparkContext("local", "test")
+    _sc = new SparkContext("local", "test", conf)
     super.beforeAll()
   }
 
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ef5936dd2f588101491b078fa9f83d104a0c75e3
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -0,0 +1,110 @@
+package org.apache.spark
+
+import org.scalatest.FunSuite
+
+class SparkConfSuite extends FunSuite with LocalSparkContext {
+  // This test uses the spark.conf in core/src/test/resources, which has a few test properties
+  test("loading from spark.conf") {
+    val conf = new SparkConf()
+    assert(conf.get("spark.test.intTestProperty") === "1")
+    assert(conf.get("spark.test.stringTestProperty") === "hi")
+    // NOTE: we don't use list properties yet, but when we do, we'll have to deal with this syntax
+    assert(conf.get("spark.test.listTestProperty") === "[a, b]")
+  }
+
+  // This test uses the spark.conf in core/src/test/resources, which has a few test properties
+  test("system properties override spark.conf") {
+    try {
+      System.setProperty("spark.test.intTestProperty", "2")
+      val conf = new SparkConf()
+      assert(conf.get("spark.test.intTestProperty") === "2")
+      assert(conf.get("spark.test.stringTestProperty") === "hi")
+    } finally {
+      System.clearProperty("spark.test.intTestProperty")
+    }
+  }
+
+  test("initializing without loading defaults") {
+    try {
+      System.setProperty("spark.test.intTestProperty", "2")
+      val conf = new SparkConf(false)
+      assert(!conf.contains("spark.test.intTestProperty"))
+      assert(!conf.contains("spark.test.stringTestProperty"))
+    } finally {
+      System.clearProperty("spark.test.intTestProperty")
+    }
+  }
+
+  test("named set methods") {
+    val conf = new SparkConf(false)
+
+    conf.setMaster("local[3]")
+    conf.setAppName("My app")
+    conf.setSparkHome("/path")
+    conf.setJars(Seq("a.jar", "b.jar"))
+    conf.setExecutorEnv("VAR1", "value1")
+    conf.setExecutorEnv(Seq(("VAR2", "value2"), ("VAR3", "value3")))
+
+    assert(conf.get("spark.master") === "local[3]")
+    assert(conf.get("spark.app.name") === "My app")
+    assert(conf.get("spark.home") === "/path")
+    assert(conf.get("spark.jars") === "a.jar,b.jar")
+    assert(conf.get("spark.executorEnv.VAR1") === "value1")
+    assert(conf.get("spark.executorEnv.VAR2") === "value2")
+    assert(conf.get("spark.executorEnv.VAR3") === "value3")
+
+    // Test the Java-friendly versions of these too
+    conf.setJars(Array("c.jar", "d.jar"))
+    conf.setExecutorEnv(Array(("VAR4", "value4"), ("VAR5", "value5")))
+    assert(conf.get("spark.jars") === "c.jar,d.jar")
+    assert(conf.get("spark.executorEnv.VAR4") === "value4")
+    assert(conf.get("spark.executorEnv.VAR5") === "value5")
+  }
+
+  test("basic get and set") {
+    val conf = new SparkConf(false)
+    assert(conf.getAll.toSet === Set())
+    conf.set("k1", "v1")
+    conf.setAll(Seq(("k2", "v2"), ("k3", "v3")))
+    assert(conf.getAll.toSet === Set(("k1", "v1"), ("k2", "v2"), ("k3", "v3")))
+    conf.set("k1", "v4")
+    conf.setAll(Seq(("k2", "v5"), ("k3", "v6")))
+    assert(conf.getAll.toSet === Set(("k1", "v4"), ("k2", "v5"), ("k3", "v6")))
+    assert(conf.contains("k1"), "conf did not contain k1")
+    assert(!conf.contains("k4"), "conf contained k4")
+    assert(conf.get("k1") === "v4")
+    intercept[Exception] { conf.get("k4") }
+    assert(conf.get("k4", "not found") === "not found")
+    assert(conf.getOption("k1") === Some("v4"))
+    assert(conf.getOption("k4") === None)
+  }
+
+  test("creating SparkContext without master and app name") {
+    val conf = new SparkConf(false)
+    intercept[SparkException] { sc = new SparkContext(conf) }
+  }
+
+  test("creating SparkContext without master") {
+    val conf = new SparkConf(false).setAppName("My app")
+    intercept[SparkException] { sc = new SparkContext(conf) }
+  }
+
+  test("creating SparkContext without app name") {
+    val conf = new SparkConf(false).setMaster("local")
+    intercept[SparkException] { sc = new SparkContext(conf) }
+  }
+
+  test("creating SparkContext with both master and app name") {
+    val conf = new SparkConf(false).setMaster("local").setAppName("My app")
+    sc = new SparkContext(conf)
+    assert(sc.master === "local")
+    assert(sc.appName === "My app")
+  }
+
+  test("SparkContext property overriding") {
+    val conf = new SparkConf(false).setMaster("local").setAppName("My app")
+    sc = new SparkContext("local[2]", "My other app", conf)
+    assert(sc.master === "local[2]")
+    assert(sc.appName === "My other app")
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 372c9f4378988fc3b6f541c888cacb0f3d8c16ff..d05bbd6ff7e6f0a84a47a3e1c10f834ef5e8b658 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -74,10 +74,12 @@ class JsonProtocolSuite extends FunSuite {
 
   def createAppDesc(): ApplicationDescription = {
     val cmd = new Command("mainClass", List("arg1", "arg2"), Map())
-    new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl")
+    new ApplicationDescription("name", Some(4), 1234, cmd, "sparkHome", "appUiUrl")
   }
-  def createAppInfo(): ApplicationInfo = {
-    new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr")
+
+  def createAppInfo() : ApplicationInfo = {
+    new ApplicationInfo(
+      3, "id", createAppDesc(), new Date(123456789), null, "appUriStr", Int.MaxValue)
   }
 
   def createDriverCommand() = new Command(
@@ -86,7 +88,7 @@ class JsonProtocolSuite extends FunSuite {
   )
 
   def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3,
-    createDriverCommand())
+    false, createDriverCommand())
 
   def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date())
 
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
new file mode 100644
index 0000000000000000000000000000000000000000..45dbcaffae94f335d73ffc5d756b9c92e9a384d7
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
@@ -0,0 +1,131 @@
+package org.apache.spark.deploy.worker
+
+import java.io.File
+
+import scala.collection.JavaConversions._
+
+import org.mockito.Mockito._
+import org.mockito.Matchers._
+import org.scalatest.FunSuite
+
+import org.apache.spark.deploy.{Command, DriverDescription}
+import org.mockito.stubbing.Answer
+import org.mockito.invocation.InvocationOnMock
+
+class DriverRunnerTest extends FunSuite {
+  private def createDriverRunner() = {
+    val command = new Command("mainClass", Seq(), Map())
+    val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command)
+    new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription,
+      null, "akka://1.2.3.4/worker/")
+  }
+
+  private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = {
+    val processBuilder = mock(classOf[ProcessBuilderLike])
+    when(processBuilder.command).thenReturn(Seq("mocked", "command"))
+    val process = mock(classOf[Process])
+    when(processBuilder.start()).thenReturn(process)
+    (processBuilder, process)
+  }
+
+  test("Process succeeds instantly") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    // One failure then a successful run
+    when(process.waitFor()).thenReturn(0)
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(process, times(1)).waitFor()
+    verify(sleeper, times(0)).sleep(anyInt())
+  }
+
+  test("Process failing several times and then succeeding") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    // fail, fail, fail, success
+    when(process.waitFor()).thenReturn(-1).thenReturn(-1).thenReturn(-1).thenReturn(0)
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(process, times(4)).waitFor()
+    verify(sleeper, times(3)).sleep(anyInt())
+    verify(sleeper, times(1)).sleep(1)
+    verify(sleeper, times(1)).sleep(2)
+    verify(sleeper, times(1)).sleep(4)
+  }
+
+  test("Process doesn't restart if not supervised") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    when(process.waitFor()).thenReturn(-1)
+
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = false)
+
+    verify(process, times(1)).waitFor()
+    verify(sleeper, times(0)).sleep(anyInt())
+  }
+
+  test("Process doesn't restart if killed") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    when(process.waitFor()).thenAnswer(new Answer[Int] {
+      def answer(invocation: InvocationOnMock): Int = {
+        runner.kill()
+        -1
+      }
+    })
+
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(process, times(1)).waitFor()
+    verify(sleeper, times(0)).sleep(anyInt())
+  }
+
+  test("Reset of backoff counter") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val clock = mock(classOf[Clock])
+    runner.setClock(clock)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+
+    when(process.waitFor())
+      .thenReturn(-1) // fail 1
+      .thenReturn(-1) // fail 2
+      .thenReturn(-1) // fail 3
+      .thenReturn(-1) // fail 4
+      .thenReturn(0) // success
+    when(clock.currentTimeMillis())
+      .thenReturn(0).thenReturn(1000) // fail 1 (short)
+      .thenReturn(1000).thenReturn(2000) // fail 2 (short)
+      .thenReturn(2000).thenReturn(10000) // fail 3 (long)
+      .thenReturn(10000).thenReturn(11000) // fail 4 (short)
+      .thenReturn(11000).thenReturn(21000) // success (long)
+
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(sleeper, times(4)).sleep(anyInt())
+    // Expected sequence of sleeps is 1,2,1,2
+    verify(sleeper, times(2)).sleep(1)
+    verify(sleeper, times(2)).sleep(2)
+  }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
index 4cb4ddc9cd66ae9385aa981618ebdd4baf23cbf3..a79ee690d39ff37eb6f77100471f1ee9d76040fb 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
@@ -18,19 +18,21 @@
 package org.apache.spark.deploy.worker
 
 import java.io.File
+
 import org.scalatest.FunSuite
+
 import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription}
 
 class ExecutorRunnerTest extends FunSuite {
   test("command includes appId") {
     def f(s:String) = new File(s)
-    val sparkHome = sys.env("SPARK_HOME")
-    val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()),
+    val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get
+    val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()),
       sparkHome, "appUiUrl")
     val appId = "12345-worker321-9876"
     val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome),
-      f("ooga"), ExecutorState.RUNNING)
+      f("ooga"), "blah", ExecutorState.RUNNING)
 
-    assert(er.buildCommandSeq().last === appId)
+    assert(er.getCommandSeq.last === appId)
   }
 }
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..94d88d307a163ee08fd2167ad572deb027a6c28b
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
@@ -0,0 +1,32 @@
+package org.apache.spark.deploy.worker
+
+
+import akka.testkit.TestActorRef
+import org.scalatest.FunSuite
+import akka.remote.DisassociatedEvent
+import akka.actor.{ActorSystem, AddressFromURIString, Props}
+
+class WorkerWatcherSuite extends FunSuite {
+  test("WorkerWatcher shuts down on valid disassociation") {
+    val actorSystem = ActorSystem("test")
+    val targetWorkerUrl = "akka://1.2.3.4/user/Worker"
+    val targetWorkerAddress = AddressFromURIString(targetWorkerUrl)
+    val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem)
+    val workerWatcher = actorRef.underlyingActor
+    workerWatcher.setTesting(testing = true)
+    actorRef.underlyingActor.receive(new DisassociatedEvent(null, targetWorkerAddress, false))
+    assert(actorRef.underlyingActor.isShutDown)
+  }
+
+  test("WorkerWatcher stays alive on invalid disassociation") {
+    val actorSystem = ActorSystem("test")
+    val targetWorkerUrl = "akka://1.2.3.4/user/Worker"
+    val otherAkkaURL = "akka://4.3.2.1/user/OtherActor"
+    val otherAkkaAddress = AddressFromURIString(otherAkkaURL)
+    val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem)
+    val workerWatcher = actorRef.underlyingActor
+    workerWatcher.setTesting(testing = true)
+    actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false))
+    assert(!actorRef.underlyingActor.isShutDown)
+  }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
index ab81bfbe559479ce688fa024944bd982f445730e..8d7546085f489767b313c7f96f83e74b3f0b9ffe 100644
--- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
+++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
@@ -20,9 +20,11 @@ package org.apache.spark.io
 import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
 
 import org.scalatest.FunSuite
+import org.apache.spark.SparkConf
 
 
 class CompressionCodecSuite extends FunSuite {
+  val conf = new SparkConf(false)
 
   def testCodec(codec: CompressionCodec) {
     // Write 1000 integers to the output stream, compressed.
@@ -43,19 +45,19 @@ class CompressionCodecSuite extends FunSuite {
   }
 
   test("default compression codec") {
-    val codec = CompressionCodec.createCodec()
+    val codec = CompressionCodec.createCodec(conf)
     assert(codec.getClass === classOf[LZFCompressionCodec])
     testCodec(codec)
   }
 
   test("lzf compression codec") {
-    val codec = CompressionCodec.createCodec(classOf[LZFCompressionCodec].getName)
+    val codec = CompressionCodec.createCodec(conf, classOf[LZFCompressionCodec].getName)
     assert(codec.getClass === classOf[LZFCompressionCodec])
     testCodec(codec)
   }
 
   test("snappy compression codec") {
-    val codec = CompressionCodec.createCodec(classOf[SnappyCompressionCodec].getName)
+    val codec = CompressionCodec.createCodec(conf, classOf[SnappyCompressionCodec].getName)
     assert(codec.getClass === classOf[SnappyCompressionCodec])
     testCodec(codec)
   }
diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
index 7181333adfe689461d7ac93d131c8dd5dd4f4872..71a2c6c498eef7244b8fab582dc533ec14bd9aa1 100644
--- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
@@ -19,17 +19,19 @@ package org.apache.spark.metrics
 
 import org.scalatest.{BeforeAndAfter, FunSuite}
 import org.apache.spark.deploy.master.MasterSource
+import org.apache.spark.SparkConf
 
 class MetricsSystemSuite extends FunSuite with BeforeAndAfter {
   var filePath: String = _
+  var conf: SparkConf = null
 
   before {
     filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile()
-    System.setProperty("spark.metrics.conf", filePath)
+    conf = new SparkConf(false).set("spark.metrics.conf", filePath)
   }
 
   test("MetricsSystem with default config") {
-    val metricsSystem = MetricsSystem.createMetricsSystem("default")
+    val metricsSystem = MetricsSystem.createMetricsSystem("default", conf)
     val sources = metricsSystem.sources
     val sinks = metricsSystem.sinks
 
@@ -39,7 +41,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter {
   }
 
   test("MetricsSystem with sources add") {
-    val metricsSystem = MetricsSystem.createMetricsSystem("test")
+    val metricsSystem = MetricsSystem.createMetricsSystem("test", conf)
     val sources = metricsSystem.sources
     val sinks = metricsSystem.sinks
 
diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
index 57d3382ed0b3fc30c47335743ecd5cae367ea6f3..5da538a1ddfd5ee05063d6a3e555db1f5305fcb7 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.rdd
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashSet
+import scala.util.Random
 
 import org.scalatest.FunSuite
 
@@ -109,6 +110,39 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext {
     assert(deps.size === 2) // ShuffledRDD, ParallelCollection.
   }
 
+  test("countApproxDistinctByKey") {
+    def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble
+
+    /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is
+     * only a statistical bound, the tests can fail for large values of relativeSD. We will be using
+     * relatively tight error bounds to check correctness of functionality rather than checking
+     * whether the approximation conforms with the requested bound.
+     */
+    val relativeSD = 0.001
+
+    // For each value i, there are i tuples with first element equal to i.
+    // Therefore, the expected count for key i would be i.
+    val stacked = (1 to 100).flatMap(i => (1 to i).map(j => (i, j)))
+    val rdd1 = sc.parallelize(stacked)
+    val counted1 = rdd1.countApproxDistinctByKey(relativeSD).collect()
+    counted1.foreach{
+      case(k, count) => assert(error(count, k) < relativeSD)
+    }
+
+    val rnd = new Random()
+
+    // The expected count for key num would be num
+    val randStacked = (1 to 100).flatMap { i =>
+      val num = rnd.nextInt % 500
+      (1 to num).map(j => (num, j))
+    }
+    val rdd2 = sc.parallelize(randStacked)
+    val counted2 = rdd2.countApproxDistinctByKey(relativeSD, 4).collect()
+    counted2.foreach{
+      case(k, count) => assert(error(count, k) < relativeSD)
+    }
+  }
+
   test("join") {
     val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)))
     val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
index d8dcd6d14c23856e0d79cc519b45e3d14dc84072..559ea051d35335de1dfc53a8ce8a973e86b46d1e 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -63,6 +63,19 @@ class RDDSuite extends FunSuite with SharedSparkContext {
     }
   }
 
+  test("countApproxDistinct") {
+
+    def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble
+
+    val size = 100
+    val uniformDistro = for (i <- 1 to 100000) yield i % size
+    val simpleRdd = sc.makeRDD(uniformDistro)
+    assert(error(simpleRdd.countApproxDistinct(0.2), size) < 0.2)
+    assert(error(simpleRdd.countApproxDistinct(0.05), size) < 0.05)
+    assert(error(simpleRdd.countApproxDistinct(0.01), size) < 0.01)
+    assert(error(simpleRdd.countApproxDistinct(0.001), size) < 0.001)
+  }
+
   test("SparkContext.union") {
     val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
     assert(sc.union(nums).collect().toList === List(1, 2, 3, 4))
@@ -71,6 +84,33 @@ class RDDSuite extends FunSuite with SharedSparkContext {
     assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4))
   }
 
+  test("partitioner aware union") {
+    import SparkContext._
+    def makeRDDWithPartitioner(seq: Seq[Int]) = {
+      sc.makeRDD(seq, 1)
+        .map(x => (x, null))
+        .partitionBy(new HashPartitioner(2))
+        .mapPartitions(_.map(_._1), true)
+    }
+
+    val nums1 = makeRDDWithPartitioner(1 to 4)
+    val nums2 = makeRDDWithPartitioner(5 to 8)
+    assert(nums1.partitioner == nums2.partitioner)
+    assert(new PartitionerAwareUnionRDD(sc, Seq(nums1)).collect().toSet === Set(1, 2, 3, 4))
+
+    val union = new PartitionerAwareUnionRDD(sc, Seq(nums1, nums2))
+    assert(union.collect().toSet === Set(1, 2, 3, 4, 5, 6, 7, 8))
+    val nums1Parts = nums1.collectPartitions()
+    val nums2Parts = nums2.collectPartitions()
+    val unionParts = union.collectPartitions()
+    assert(nums1Parts.length === 2)
+    assert(nums2Parts.length === 2)
+    assert(unionParts.length === 2)
+    assert((nums1Parts(0) ++ nums2Parts(0)).toList === unionParts(0).toList)
+    assert((nums1Parts(1) ++ nums2Parts(1)).toList === unionParts(1).toList)
+    assert(union.partitioner === nums1.partitioner)
+  }
+
   test("aggregate") {
     val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3)))
     type StringMap = HashMap[String, Int]
diff --git a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
index 702edb862fea97e317d6d74a218692b5700d088b..7bf2020fe378eae92a89355d20005efc9c822ec9 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
@@ -167,7 +167,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
     val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
     System.setProperty("spark.scheduler.allocation.file", xmlPath)
     val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
-    val schedulableBuilder = new FairSchedulableBuilder(rootPool)
+    val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf)
     schedulableBuilder.buildPools()
 
     assert(rootPool.getSchedulableByName("default") != null)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 706d84a58b5630a80ea2bf23ea01e197ffa3f43a..2aa259daf38b30b5e8670c70e5a4e95bc0cb6cf9 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -17,21 +17,14 @@
 
 package org.apache.spark.scheduler
 
-import scala.collection.mutable.{Map, HashMap}
-
-import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
-
-import org.apache.spark.LocalSparkContext
-import org.apache.spark.MapOutputTrackerMaster
-import org.apache.spark.SparkContext
-import org.apache.spark.Partition
-import org.apache.spark.TaskContext
-import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency}
-import org.apache.spark.{FetchFailed, Success, TaskEndReason}
+import scala.Tuple2
+import scala.collection.mutable.{HashMap, Map}
+
+import org.apache.spark._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
 import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
+import org.scalatest.{BeforeAndAfter, FunSuite}
 
 /**
  * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler
@@ -46,7 +39,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
  * and capturing the resulting TaskSets from the mock TaskScheduler.
  */
 class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
-
+  val conf = new SparkConf
   /** Set of TaskSets the DAGScheduler has requested executed. */
   val taskSets = scala.collection.mutable.Buffer[TaskSet]()
   val taskScheduler = new TaskScheduler() {
@@ -74,7 +67,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
    */
   val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]]
   // stub out BlockManagerMaster.getLocations to use our cacheLocations
-  val blockManagerMaster = new BlockManagerMaster(null) {
+  val blockManagerMaster = new BlockManagerMaster(null, conf) {
       override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
         blockIds.map {
           _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)).
@@ -99,7 +92,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
     taskSets.clear()
     cacheLocations.clear()
     results.clear()
-    mapOutputTracker = new MapOutputTrackerMaster()
+    mapOutputTracker = new MapOutputTrackerMaster(conf)
     scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) {
       override def runLocally(job: ActiveJob) {
         // don't bother with the thread while unit testing
diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
index 002368ff554f7226cc4f7f6b1643f8c1f61df0c8..5cc48ee00a8990bbe5a97ba7812e2a6c0af9cdfc 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
@@ -95,7 +95,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
 
     assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
 
-    val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER)
+    val user = System.getProperty("user.name",  SparkContext.SPARK_UNKNOWN_USER)
     
     joblogger.getLogDir should be ("/tmp/spark-%s".format(user))
     joblogger.getJobIDtoPrintWriter.size should be (1)
@@ -117,7 +117,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
       override def onTaskEnd(taskEnd: SparkListenerTaskEnd)  = onTaskEndCount += 1
       override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1
       override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1
-      override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1
+      override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = onStageCompletedCount += 1
       override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1
     }
     sc.addSparkListener(joblogger)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index d4320e5e14458afef262316bdf18d4e75257ac18..1a16e438c43d8e543c701829ae2d8ed2ec17d360 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -174,7 +174,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
 
   class SaveStageInfo extends SparkListener {
     val stageInfos = Buffer[StageInfo]()
-    override def onStageCompleted(stage: StageCompleted) {
+    override def onStageCompleted(stage: SparkListenerStageCompleted) {
       stageInfos += stage.stage
     }
   }
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
index ca97f7d2a5be6fa979e2f7f656834ed0bef80ac7..4b52d9651ebe82cbc8ff640e09445d89e0bd4299 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer
 
 import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
 
-import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv}
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv}
 import org.apache.spark.storage.TaskResultBlockId
 
 /**
@@ -43,13 +43,13 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule
         case IndirectTaskResult(blockId) =>
           sparkEnv.blockManager.master.removeBlock(blockId)
         case directResult: DirectTaskResult[_] =>
-          taskSetManager.abort("Internal error: expect only indirect results") 
+          taskSetManager.abort("Internal error: expect only indirect results")
       }
       serializedData.rewind()
       removedResult = true
     }
     super.enqueueSuccessfulTask(taskSetManager, tid, serializedData)
-  } 
+  }
 }
 
 /**
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index 3dcb01ae5e44630d0daef407e883a3992430b63c..1eec6726f48bc0900c64c97b0bb9edaaa641705f 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -81,7 +81,9 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /*
 class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
   import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL}
 
-  val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong
+  private val conf = new SparkConf
+
+  val LOCALITY_WAIT = conf.get("spark.locality.wait", "3000").toLong
   val MAX_TASK_FAILURES = 4
 
   test("TaskSet with no preferences") {
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index c016c5117149fabcf03e71f6fdd755c7bb60b3cd..3898583275557ac0089b69b06b3056605d34e0cf 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -22,12 +22,15 @@ import scala.collection.mutable
 import com.esotericsoftware.kryo.Kryo
 
 import org.scalatest.FunSuite
-import org.apache.spark.SharedSparkContext
+import org.apache.spark.{SparkConf, SharedSparkContext}
 import org.apache.spark.serializer.KryoTest._
 
 class KryoSerializerSuite extends FunSuite with SharedSparkContext {
+  conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+  conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName)
+
   test("basic types") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }
@@ -57,7 +60,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("pairs") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }
@@ -81,7 +84,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("Scala data structures") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }
@@ -104,7 +107,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("ranges") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
       // Check that very long ranges don't get written one element at a time
@@ -125,9 +128,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("custom registrator") {
-    System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
-
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }
@@ -172,6 +173,10 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
     assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11))
   }
 
+  test("kryo with SerializableHyperLogLog") {
+    assert(sc.parallelize( Array(1, 2, 3, 2, 3, 3, 2, 3, 1) ).countApproxDistinct(0.01) === 3)
+  }
+
   test("kryo with reduce") {
     val control = 1 :: 2 :: Nil
     val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_))
@@ -186,18 +191,6 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
         .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x
     assert(10 + control.sum === result)
   }
-
-  override def beforeAll() {
-    System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-    System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
-    super.beforeAll()
-  }
-
-  override def afterAll() {
-    super.afterAll()
-    System.clearProperty("spark.kryo.registrator")
-    System.clearProperty("spark.serializer")
-  }
 }
 
 object KryoTest {
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index 5b4d63b954cc850ff9b0bd8c2f08e6612edd2607..f60ce270c7387f06b08afcd1e68168ca53037ce8 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -31,41 +31,42 @@ import org.scalatest.time.SpanSugar._
 
 import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream}
 import org.apache.spark.serializer.{JavaSerializer, KryoSerializer}
+import org.apache.spark.{SparkConf, SparkContext}
 
 class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester {
+  private val conf = new SparkConf(false)
   var store: BlockManager = null
   var store2: BlockManager = null
   var actorSystem: ActorSystem = null
   var master: BlockManagerMaster = null
   var oldArch: String = null
-  var oldOops: String = null
-  var oldHeartBeat: String = null
 
   // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
-  System.setProperty("spark.kryoserializer.buffer.mb", "1")
-  val serializer = new KryoSerializer
+  conf.set("spark.kryoserializer.buffer.mb", "1")
+  val serializer = new KryoSerializer(conf)
 
   // Implicitly convert strings to BlockIds for test clarity.
   implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value)
   def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId)
 
   before {
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0)
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf)
     this.actorSystem = actorSystem
-    System.setProperty("spark.driver.port", boundPort.toString)
-    System.setProperty("spark.hostPort", "localhost:" + boundPort)
+    conf.set("spark.driver.port", boundPort.toString)
+    conf.set("spark.hostPort", "localhost:" + boundPort)
 
     master = new BlockManagerMaster(
-      Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))))
+      actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf)
 
     // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
     oldArch = System.setProperty("os.arch", "amd64")
-    oldOops = System.setProperty("spark.test.useCompressedOops", "true")
-    oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true")
+    conf.set("os.arch", "amd64")
+    conf.set("spark.test.useCompressedOops", "true")
+    conf.set("spark.storage.disableBlockManagerHeartBeat", "true")
     val initialize = PrivateMethod[Unit]('initialize)
     SizeEstimator invokePrivate initialize()
     // Set some value ...
-    System.setProperty("spark.hostPort", Utils.localHostName() + ":" + 1111)
+    conf.set("spark.hostPort", Utils.localHostName() + ":" + 1111)
   }
 
   after {
@@ -86,16 +87,12 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     master = null
 
     if (oldArch != null) {
-      System.setProperty("os.arch", oldArch)
+      conf.set("os.arch", oldArch)
     } else {
       System.clearProperty("os.arch")
     }
 
-    if (oldOops != null) {
-      System.setProperty("spark.test.useCompressedOops", oldOops)
-    } else {
-      System.clearProperty("spark.test.useCompressedOops")
-    }
+    System.clearProperty("spark.test.useCompressedOops")
   }
 
   test("StorageLevel object caching") {
@@ -133,7 +130,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("master + 1 manager interaction") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -163,8 +160,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("master + 2 managers interaction") {
-    store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
-    store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000)
+    store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf)
+    store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf)
 
     val peers = master.getPeers(store.blockManagerId, 1)
     assert(peers.size === 1, "master did not return the other manager as a peer")
@@ -179,7 +176,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("removing block") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -227,7 +224,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("removing rdd") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -261,7 +258,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   test("reregistration on heart beat") {
     val heartBeat = PrivateMethod[Unit]('heartBeat)
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
     val a1 = new Array[Byte](400)
 
     store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
@@ -277,7 +274,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("reregistration on block update") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
 
@@ -296,7 +293,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   test("reregistration doesn't dead lock") {
     val heartBeat = PrivateMethod[Unit]('heartBeat)
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
     val a1 = new Array[Byte](400)
     val a2 = List(new Array[Byte](400))
 
@@ -333,7 +330,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU storage") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -352,7 +349,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU storage with serialization") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -371,7 +368,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU for partitions of same RDD") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -390,7 +387,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU for partitions of multiple RDDs") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
     store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
     store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
@@ -413,7 +410,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("on-disk storage") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -426,7 +423,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -441,7 +438,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage with getLocalBytes") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -456,7 +453,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage with serialization") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -471,7 +468,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage with serialization and getLocalBytes") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -486,7 +483,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("LRU with mixed storage levels") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -511,7 +508,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU with streams") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val list1 = List(new Array[Byte](200), new Array[Byte](200))
     val list2 = List(new Array[Byte](200), new Array[Byte](200))
     val list3 = List(new Array[Byte](200), new Array[Byte](200))
@@ -535,7 +532,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("LRU with mixed storage levels and streams") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
     val list1 = List(new Array[Byte](200), new Array[Byte](200))
     val list2 = List(new Array[Byte](200), new Array[Byte](200))
     val list3 = List(new Array[Byte](200), new Array[Byte](200))
@@ -581,7 +578,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("overly large block") {
-    store = new BlockManager("<driver>", actorSystem, master, serializer, 500)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 500, conf)
     store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY)
     assert(store.getSingle("a1") === None, "a1 was in store")
     store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK)
@@ -591,53 +588,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   test("block compression") {
     try {
-      System.setProperty("spark.shuffle.compress", "true")
-      store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
+      conf.set("spark.shuffle.compress", "true")
+      store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf)
       store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100,
         "shuffle_0_0_0 was not compressed")
       store.stop()
       store = null
 
-      System.setProperty("spark.shuffle.compress", "false")
-      store = new BlockManager("exec2", actorSystem, master, serializer, 2000)
+      conf.set("spark.shuffle.compress", "false")
+      store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf)
       store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000,
         "shuffle_0_0_0 was compressed")
       store.stop()
       store = null
 
-      System.setProperty("spark.broadcast.compress", "true")
-      store = new BlockManager("exec3", actorSystem, master, serializer, 2000)
+      conf.set("spark.broadcast.compress", "true")
+      store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf)
       store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100,
         "broadcast_0 was not compressed")
       store.stop()
       store = null
 
-      System.setProperty("spark.broadcast.compress", "false")
-      store = new BlockManager("exec4", actorSystem, master, serializer, 2000)
+      conf.set("spark.broadcast.compress", "false")
+      store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf)
       store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed")
       store.stop()
       store = null
 
-      System.setProperty("spark.rdd.compress", "true")
-      store = new BlockManager("exec5", actorSystem, master, serializer, 2000)
+      conf.set("spark.rdd.compress", "true")
+      store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf)
       store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed")
       store.stop()
       store = null
 
-      System.setProperty("spark.rdd.compress", "false")
-      store = new BlockManager("exec6", actorSystem, master, serializer, 2000)
+      conf.set("spark.rdd.compress", "false")
+      store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf)
       store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed")
       store.stop()
       store = null
 
       // Check that any other block types are also kept uncompressed
-      store = new BlockManager("exec7", actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf)
       store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY)
       assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed")
       store.stop()
@@ -651,7 +648,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   test("block store put failure") {
     // Use Java serializer so we can create an unserializable error.
-    store = new BlockManager("<driver>", actorSystem, master, new JavaSerializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, new JavaSerializer(conf), 1200, conf)
 
     // The put should fail since a1 is not serializable.
     class UnserializableClass
diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
index 070982e798f343b2bdbdeaf6c663b0e0b2575873..829f389460f3bdca7480db59fab2e2d0b0455c89 100644
--- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
@@ -17,15 +17,16 @@
 
 package org.apache.spark.storage
 
-import java.io.{FileWriter, File}
+import java.io.{File, FileWriter}
 
 import scala.collection.mutable
 
 import com.google.common.io.Files
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll {
+import org.apache.spark.SparkConf
+import org.scalatest.{BeforeAndAfterEach, FunSuite}
 
+class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach {
+  private val testConf = new SparkConf(false)
   val rootDir0 = Files.createTempDir()
   rootDir0.deleteOnExit()
   val rootDir1 = Files.createTempDir()
@@ -35,21 +36,16 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before
 
   // This suite focuses primarily on consolidation features,
   // so we coerce consolidation if not already enabled.
-  val consolidateProp = "spark.shuffle.consolidateFiles"
-  val oldConsolidate = Option(System.getProperty(consolidateProp))
-  System.setProperty(consolidateProp, "true")
+  testConf.set("spark.shuffle.consolidateFiles", "true")
 
   val shuffleBlockManager = new ShuffleBlockManager(null) {
+    override def conf = testConf.clone
     var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]()
     override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id)
   }
 
   var diskBlockManager: DiskBlockManager = _
 
-  override def afterAll() {
-    oldConsolidate.map(c => System.setProperty(consolidateProp, c))
-  }
-
   override def beforeEach() {
     diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs)
     shuffleBlockManager.idToSegmentMap.clear()
diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
index 5aff26f9fcb29a55bcde8c92a09f4139624a2af7..11ebdc352bd74ae2f85bf300bdf9316d044406dc 100644
--- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.util
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.PrivateMethodTester
+import org.apache.spark.SparkContext
 
 class DummyClass1 {}
 
@@ -139,7 +140,6 @@ class SizeEstimatorSuite
   test("64-bit arch with no compressed oops") {
     val arch = System.setProperty("os.arch", "amd64")
     val oops = System.setProperty("spark.test.useCompressedOops", "false")
-
     val initialize = PrivateMethod[Unit]('initialize)
     SizeEstimator invokePrivate initialize()
 
diff --git a/kmeans_data.txt b/data/kmeans_data.txt
similarity index 100%
rename from kmeans_data.txt
rename to data/kmeans_data.txt
diff --git a/lr_data.txt b/data/lr_data.txt
similarity index 100%
rename from lr_data.txt
rename to data/lr_data.txt
diff --git a/pagerank_data.txt b/data/pagerank_data.txt
similarity index 100%
rename from pagerank_data.txt
rename to data/pagerank_data.txt
diff --git a/docs/_config.yml b/docs/_config.yml
index 02067f9750f489d43dfb16d33273cdcb9b86e0db..11d18f0ac20939968302e99d071171a2f58d542c 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -4,7 +4,7 @@ markdown: kramdown
 # These allow the documentation to be updated with nerw releases
 # of Spark, Scala, and Mesos.
 SPARK_VERSION: 0.9.0-incubating-SNAPSHOT
-SPARK_VERSION_SHORT: 0.9.0-SNAPSHOT
+SPARK_VERSION_SHORT: 0.9.0
 SCALA_VERSION: 2.10
 MESOS_VERSION: 0.13.0
 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
index de001e6c52f85e39c6ca62afd7e699039221ec35..c4f1f6d6ade8cc0f2a870b4eb6fb5f1735ca9323 100644
--- a/docs/bagel-programming-guide.md
+++ b/docs/bagel-programming-guide.md
@@ -157,8 +157,8 @@ trait Message[K] {
 
 # Where to Go from Here
 
-Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.:
+Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `bin/run-example` script included in Spark; e.g.:
 
-    ./run-example org.apache.spark.examples.bagel.WikipediaPageRank
+    ./bin/run-example org.apache.spark.examples.bagel.WikipediaPageRank
 
 Each example program prints usage help when run without any arguments.
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index c709001632261c1d0443fdcb0ac2d94211d0add3..b9ff0af76f6478f49fe37a0e6a360796eb1d967a 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -37,20 +37,16 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit
     # Cloudera CDH 4.2.0 with MapReduce v1
     $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package
 
-For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "hadoop2-yarn" profile and set the "yarn.version" property:
+For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property:
 
     # Apache Hadoop 2.0.5-alpha
-    $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package
+    $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package
 
     # Cloudera CDH 4.2.0 with MapReduce v2
-    $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
+    $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
 
-Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows:
- 
-    # Apache Hadoop 2.2.X and newer
-    $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
-
-The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. 
+    # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer
+    $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package
 
 ## Spark Tests in Maven ##
 
diff --git a/docs/configuration.md b/docs/configuration.md
index 677d182e50b3d8d4a32bf8672784ce1689236bd0..1d6c3d16333c5ebc86a348113ca98a265bbc94c4 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -3,26 +3,37 @@ layout: global
 title: Spark Configuration
 ---
 
-Spark provides three main locations to configure the system:
+Spark provides three locations to configure the system:
 
-* [Java system properties](#system-properties), which control internal configuration parameters and can be set
-  either programmatically (by calling `System.setProperty` *before* creating a `SparkContext`) or through
-  JVM arguments.
-* [Environment variables](#environment-variables) for configuring per-machine settings such as the IP address,
-  which can be set in the `conf/spark-env.sh` script.
-* [Logging configuration](#configuring-logging), which is done through `log4j.properties`.
+* [Spark properties](#spark-properties) control most application parameters and can be set by passing
+  a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java
+  system properties.
+* [Environment variables](#environment-variables) can be used to set per-machine settings, such as
+  the IP address, through the `conf/spark-env.sh` script on each node.
+* [Logging](#configuring-logging) can be configured through `log4j.properties`.
 
 
-# System Properties
+# Spark Properties
 
-To set a system property for configuring Spark, you need to either pass it with a -D flag to the JVM (for example `java -Dspark.cores.max=5 MyProgram`) or call `System.setProperty` in your code *before* creating your Spark context, as follows:
+Spark properties control most application settings and are configured separately for each application.
+The preferred way to set them is by passing a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+class to your SparkContext constructor.
+Alternatively, Spark will also load them from Java system properties (for compatibility with old versions
+of Spark) and from a [`spark.conf` file](#configuration-files) on your classpath.
+
+SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and
+application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could
+initialize an application as follows:
 
 {% highlight scala %}
-System.setProperty("spark.cores.max", "5")
-val sc = new SparkContext(...)
+val conf = new SparkConf()
+             .setMaster("local")
+             .setAppName("My application")
+             .set("spark.executor.memory", "1g")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
-Most of the configurable system properties control internal settings that have reasonable default values. However,
+Most of the properties control internal settings that have reasonable default values. However,
 there are at least five properties that you will commonly want to control:
 
 <table class="table">
@@ -66,12 +77,14 @@ there are at least five properties that you will commonly want to control:
 </tr>
 <tr>
   <td>spark.cores.max</td>
-  <td>(infinite)</td>
+  <td>(not set)</td>
   <td>
     When running on a <a href="spark-standalone.html">standalone deploy cluster</a> or a
     <a href="running-on-mesos.html#mesos-run-modes">Mesos cluster in "coarse-grained"
-    sharing mode</a>, how many CPU cores to request at most. The default will use all available cores
-    offered by the cluster manager.
+    sharing mode</a>, the maximum amount of CPU cores to request for the application from
+    across the cluster (not from each machine). If not set, the default will be
+    <code>spark.deploy.defaultCores</code> on Spark's standalone cluster manager, or
+    infinite (all available cores) on Mesos.
   </td>
 </tr>
 </table>
@@ -348,6 +361,14 @@ Apart from these, the following properties are also available, and may be useful
     Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, <code>BlockManager</code> might take a performance hit.
   </td>
 </tr>
+<tr>
+  <td>akka.x.y....</td>
+  <td>value</td>
+  <td>
+    An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that SparkContext and its assigned executors as well.
+  </td>
+</tr>
+
 <tr>
   <td>spark.shuffle.consolidateFiles</td>
   <td>false</td>
@@ -383,13 +404,73 @@ Apart from these, the following properties are also available, and may be useful
     How many times slower a task is than the median to be considered for speculation.
   </td>
 </tr>
+<tr>
+  <td>spark.logConf</td>
+  <td>false</td>
+  <td>
+    Log the supplied SparkConf as INFO at start of spark context.
+  </td>
+</tr>
+<tr>
+  <td>spark.deploy.spreadOut</td>
+  <td>true</td>
+  <td>
+    Whether the standalone cluster manager should spread applications out across nodes or try
+    to consolidate them onto as few nodes as possible. Spreading out is usually better for
+    data locality in HDFS, but consolidating is more efficient for compute-intensive workloads. <br/>
+    <b>Note:</b> this setting needs to be configured in the standalone cluster master, not in individual
+    applications; you can set it through <code>SPARK_JAVA_OPTS</code> in <code>spark-env.sh</code>.
+  </td>
+</tr>
+<tr>
+  <td>spark.deploy.defaultCores</td>
+  <td>(infinite)</td>
+  <td>
+    Default number of cores to give to applications in Spark's standalone mode if they don't
+    set <code>spark.cores.max</code>. If not set, applications always get all available
+    cores unless they configure <code>spark.cores.max</code> themselves.
+    Set this lower on a shared cluster to prevent users from grabbing
+    the whole cluster by default. <br/>
+    <b>Note:</b> this setting needs to be configured in the standalone cluster master, not in individual
+    applications; you can set it through <code>SPARK_JAVA_OPTS</code> in <code>spark-env.sh</code>.
+  </td>
+</tr>
 </table>
 
+## Viewing Spark Properties
+
+The application web UI at `http://<driver>:4040` lists Spark properties in the "Environment" tab.
+This is a useful place to check to make sure that your properties have been set correctly.
+
+## Configuration Files
+
+You can also configure Spark properties through a `spark.conf` file on your Java classpath.
+Because these properties are usually application-specific, we recommend putting this fine *only* on your
+application's classpath, and not in a global Spark classpath.
+
+The `spark.conf` file uses Typesafe Config's [HOCON format](https://github.com/typesafehub/config#json-superset),
+which is a superset of Java properties files and JSON. For example, the following is a simple config file:
+
+{% highlight awk %}
+# Comments are allowed
+spark.executor.memory = 512m
+spark.serializer = org.apache.spark.serializer.KryoSerializer
+{% endhighlight %}
+
+The format also allows hierarchical nesting, as follows:
+
+{% highlight awk %}
+spark.akka {
+  threads = 8
+  timeout = 200
+}
+{% endhighlight %}
+
 # Environment Variables
 
-Certain Spark settings can also be configured through environment variables, which are read from the `conf/spark-env.sh`
+Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh`
 script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such
-as library search paths. While Java system properties can also be set here, for application settings, we recommend setting
+as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting
 these properties within the application instead of in `spark-env.sh` so that different applications can use different
 settings.
 
@@ -406,7 +487,8 @@ The following variables can be set in `spark-env.sh`:
    Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend
    doing that when possible.
 * `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system
-   properties that you'd like to pass with `-D` (e.g., `-Dspark.local.dir=/disk1,/disk2`). 
+   properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this
+   machine, e.g., `-Dspark.local.dir=/disk1,/disk2`.
 * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores
   to use on each machine and maximum memory.
 
diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css
index ccb529eababd8ab79be308facbf9c7aa0cf67abf..30ea440c008e34b3b6958d3c732be576b77199ca 100644
--- a/docs/css/bootstrap.min.css
+++ b/docs/css/bootstrap.min.css
@@ -6,4 +6,4 @@
  * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Designed and built with all the love in the world @twitter by @mdo and @fat.
- */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#d4e4f4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#d4e4f4));background-image:-webkit-linear-gradient(top,#fff,#d4e4f4);background-image:-o-linear-gradient(top,#fff,#d4e4f4);background-image:linear-gradient(to bottom,#fff,#d4e4f4);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffd4e4f4',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed}
+ */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#feead8);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#feead8));background-image:-webkit-linear-gradient(top,#fff,#feead8);background-image:-o-linear-gradient(top,#fff,#feead8);background-image:linear-gradient(to bottom,#fff,#feead8);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fffeead8',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed}
diff --git a/docs/index.md b/docs/index.md
index d3ac696d1e818a0dee8587756a6f8974276ac289..86d574daaab4a8fc530f5a7f80a179133c4553a6 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -24,9 +24,9 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VE
 # Running the Examples and Shell
 
 Spark comes with several sample programs in the `examples` directory.
-To run one of the samples, use `./run-example <class> <params>` in the top-level Spark directory
-(the `run-example` script sets up the appropriate paths and launches that program).
-For example, try `./run-example org.apache.spark.examples.SparkPi local`.
+To run one of the samples, use `./bin/run-example <class> <params>` in the top-level Spark directory
+(the `bin/run-example` script sets up the appropriate paths and launches that program).
+For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`.
 Each example prints usage help when run with no parameters.
 
 Note that all of the sample programs take a `<master>` parameter specifying the cluster URL
@@ -34,8 +34,8 @@ to connect to. This can be a [URL for a distributed cluster](scala-programming-g
 or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using
 `local` for testing.
 
-Finally, you can run Spark interactively through modified versions of the Scala shell (`./spark-shell`) or
-Python interpreter (`./pyspark`). These are a great way to learn the framework.
+Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or
+Python interpreter (`./bin/pyspark`). These are a great way to learn the framework.
 
 # Launching on a Cluster
 
diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md
index 53085cc6719b07ea185ab1c8e39a9d987cafa315..07732fa1229f3bfef0ad729121efadd25c1f471e 100644
--- a/docs/java-programming-guide.md
+++ b/docs/java-programming-guide.md
@@ -190,9 +190,9 @@ 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/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples).  You can run them by passing the class name to the
-`run-example` script included in Spark; for example:
+`bin/run-example` script included in Spark; for example:
 
-    ./run-example org.apache.spark.examples.JavaWordCount
+    ./bin/run-example org.apache.spark.examples.JavaWordCount
 
 Each example program prints usage help when run
 without any arguments.
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
index dbcb9ae343357f648637eb3487819725beb304bd..df2faa5e41b182c1f11d117c6f94498d533ee6e0 100644
--- a/docs/job-scheduling.md
+++ b/docs/job-scheduling.md
@@ -32,12 +32,11 @@ Resource allocation can be configured as follows, based on the cluster type:
 
 * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in
   FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit
-  the number of nodes an application uses by setting the `spark.cores.max` system property in it. This
-  will allow multiple users/applications to run concurrently. For example, you might launch a long-running
-  server that uses 10 cores, and allow users to launch shells that use 20 cores each.
+  the number of nodes an application uses by setting the `spark.cores.max` configuration property in it,
+  or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. 
   Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls
   its memory use.
-* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` system property to `true`,
+* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`,
   and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode.
   You should also set `spark.executor.memory` to control the executor memory.
 * **YARN:** The `--num-workers` option to the Spark YARN client controls how many workers it will allocate
@@ -78,11 +77,13 @@ of cluster resources. This means that short jobs submitted while a long job is r
 resources right away and still get good response times, without waiting for the long job to finish. This
 mode is best for multi-user settings.
 
-To enable the fair scheduler, simply set the `spark.scheduler.mode` to `FAIR` before creating
+To enable the fair scheduler, simply set the `spark.scheduler.mode` property to `FAIR` when configuring
 a SparkContext:
 
 {% highlight scala %}
-System.setProperty("spark.scheduler.mode", "FAIR")
+val conf = new SparkConf().setMaster(...).setAppName(...)
+conf.set("spark.scheduler.mode", "FAIR")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
 ## Fair Scheduler Pools
@@ -98,8 +99,8 @@ adding the `spark.scheduler.pool` "local property" to the SparkContext in the th
 This is done as follows:
 
 {% highlight scala %}
-// Assuming context is your SparkContext variable
-context.setLocalProperty("spark.scheduler.pool", "pool1")
+// Assuming sc is your SparkContext variable
+sc.setLocalProperty("spark.scheduler.pool", "pool1")
 {% endhighlight %}
 
 After setting this local property, _all_ jobs submitted within this thread (by calls in this thread
@@ -108,7 +109,7 @@ it easy to have a thread run multiple jobs on behalf of the same user. If you'd
 pool that a thread is associated with, simply call:
 
 {% highlight scala %}
-context.setLocalProperty("spark.scheduler.pool", null)
+sc.setLocalProperty("spark.scheduler.pool", null)
 {% endhighlight %}
 
 ## Default Behavior of Pools
@@ -138,10 +139,11 @@ properties:
   of the cluster. By default, each pool's `minShare` is 0.
 
 The pool properties can be set by creating an XML file, similar to `conf/fairscheduler.xml.template`,
-and setting the `spark.scheduler.allocation.file` property:
+and setting a `spark.scheduler.allocation.file` property in your
+[SparkConf](configuration.html#spark-properties).
 
 {% highlight scala %}
-System.setProperty("spark.scheduler.allocation.file", "/path/to/file")
+conf.set("spark.scheduler.allocation.file", "/path/to/file")
 {% endhighlight %}
 
 The format of the XML file is simply a `<pool>` element for each pool, with different elements
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index c1ff9c417c35396ffc71bf7cf56faaad9c136af9..45ee16668872092b68f0dba43815515cff34fd7e 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -3,6 +3,9 @@ layout: global
 title: Machine Learning Library (MLlib)
 ---
 
+* Table of contents
+{:toc}
+
 MLlib is a Spark implementation of some common machine learning (ML)
 functionality, as well associated tests and data generators.  MLlib
 currently supports four common types of machine learning problem settings,
@@ -39,57 +42,6 @@ underlying gradient descent primitive (described
 parameter (*regParam*) along with various parameters associated with gradient
 descent (*stepSize*, *numIterations*, *miniBatchFraction*). 
 
-The following code snippet illustrates how to load a sample dataset, execute a
-training algorithm on this training data using a static method in the algorithm
-object, and make predictions with the resulting model to compute the training
-error.
-
-{% highlight scala %}
-import org.apache.spark.SparkContext
-import org.apache.spark.mllib.classification.SVMWithSGD
-import org.apache.spark.mllib.regression.LabeledPoint
-
-// Load and parse the data file
-val data = sc.textFile("mllib/data/sample_svm_data.txt")
-val parsedData = data.map { line =>
-  val parts = line.split(' ')
-  LabeledPoint(parts(0).toDouble, parts.tail.map(x => x.toDouble).toArray)
-}
-
-// Run training algorithm
-val numIterations = 20
-val model = SVMWithSGD.train(parsedData, numIterations)
- 
-// Evaluate model on training examples and compute training error
-val labelAndPreds = parsedData.map { point =>
-  val prediction = model.predict(point.features)
-  (point.label, prediction)
-}
-val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count
-println("trainError = " + trainErr)
-{% endhighlight %}
-
-The `SVMWithSGD.train()` method by default performs L2 regularization with the
-regularization parameter set to 1.0. If we want to configure this algorithm, we
-can customize `SVMWithSGD` further by creating a new object directly and
-calling setter methods. All other MLlib algorithms support customization in
-this way as well. For example, the following code produces an L1 regularized
-variant of SVMs with regularization parameter set to 0.1, and runs the training
-algorithm for 200 iterations. 
-
-{% highlight scala %}
-import org.apache.spark.mllib.optimization.L1Updater
-
-val svmAlg = new SVMWithSGD()
-svmAlg.optimizer.setNumIterations(200)
-  .setRegParam(0.1)
-  .setUpdater(new L1Updater)
-val modelL1 = svmAlg.run(parsedData)
-{% endhighlight %}
-
-Both of the code snippets above can be executed in `spark-shell` to generate a
-classifier for the provided dataset.
-
 Available algorithms for binary classification:
 
 * [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD)
@@ -121,14 +73,14 @@ of entities with one another based on some notion of similarity.  Clustering is
 often used for exploratory analysis and/or as a component of a hierarchical
 supervised learning pipeline (in which distinct classifiers or regression
 models are trained for each cluster). MLlib supports
-[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, arguably
-the most commonly used clustering approach that clusters the data points into
-*k* clusters. The MLlib implementation includes a parallelized 
+[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of
+the most commonly used clustering algorithms that clusters the data points into
+predfined number of clusters. The MLlib implementation includes a parallelized
 variant of the [k-means++](http://en.wikipedia.org/wiki/K-means%2B%2B) method
 called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf).
 The implementation in MLlib has the following parameters:  
 
-* *k* is the number of clusters.
+* *k* is the number of desired clusters.
 * *maxIterations* is the maximum number of iterations to run.
 * *initializationMode* specifies either random initialization or
 initialization via k-means\|\|.
@@ -169,7 +121,7 @@ the entries in the user-item matrix as *explicit* preferences given by the user
 It is common in many real-world use cases to only have access to *implicit feedback* 
 (e.g. views, clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with 
 such data is taken from 
-[Collaborative Filtering for Implicit Feedback Datasets](http://research.yahoo.com/pub/2433).
+[Collaborative Filtering for Implicit Feedback Datasets](http://www2.research.att.com/~yifanhu/PUB/cf.pdf).
 Essentially instead of trying to model the matrix of ratings directly, this approach treats the data as 
 a combination of binary preferences and *confidence values*. The ratings are then related 
 to the level of confidence in observed user preferences, rather than explicit ratings given to items. 
@@ -210,3 +162,269 @@ at each iteration.
 Available algorithms for gradient descent:
 
 * [GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent)
+
+# Using MLLib in Scala
+
+Following code snippets can be executed in `spark-shell`.
+
+## Binary Classification
+
+The following code snippet illustrates how to load a sample dataset, execute a
+training algorithm on this training data using a static method in the algorithm
+object, and make predictions with the resulting model to compute the training
+error.
+
+{% highlight scala %}
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.classification.SVMWithSGD
+import org.apache.spark.mllib.regression.LabeledPoint
+
+// Load and parse the data file
+val data = sc.textFile("mllib/data/sample_svm_data.txt")
+val parsedData = data.map { line =>
+  val parts = line.split(' ')
+  LabeledPoint(parts(0).toDouble, parts.tail.map(x => x.toDouble).toArray)
+}
+
+// Run training algorithm to build the model
+val numIterations = 20
+val model = SVMWithSGD.train(parsedData, numIterations)
+
+// Evaluate model on training examples and compute training error
+val labelAndPreds = parsedData.map { point =>
+  val prediction = model.predict(point.features)
+  (point.label, prediction)
+}
+val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count
+println("Training Error = " + trainErr)
+{% endhighlight %}
+
+
+The `SVMWithSGD.train()` method by default performs L2 regularization with the
+regularization parameter set to 1.0. If we want to configure this algorithm, we
+can customize `SVMWithSGD` further by creating a new object directly and
+calling setter methods. All other MLlib algorithms support customization in
+this way as well. For example, the following code produces an L1 regularized
+variant of SVMs with regularization parameter set to 0.1, and runs the training
+algorithm for 200 iterations.
+
+{% highlight scala %}
+import org.apache.spark.mllib.optimization.L1Updater
+
+val svmAlg = new SVMWithSGD()
+svmAlg.optimizer.setNumIterations(200)
+  .setRegParam(0.1)
+  .setUpdater(new L1Updater)
+val modelL1 = svmAlg.run(parsedData)
+{% endhighlight %}
+
+## Linear Regression
+The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The
+example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We
+compute the Mean Squared Error at the end to evaluate
+[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit)
+
+{% highlight scala %}
+import org.apache.spark.mllib.regression.LinearRegressionWithSGD
+import org.apache.spark.mllib.regression.LabeledPoint
+
+// Load and parse the data
+val data = sc.textFile("mllib/data/ridge-data/lpsa.data")
+val parsedData = data.map { line =>
+  val parts = line.split(',')
+  LabeledPoint(parts(0).toDouble, parts(1).split(' ').map(x => x.toDouble).toArray)
+}
+
+// Building the model
+val numIterations = 20
+val model = LinearRegressionWithSGD.train(parsedData, numIterations)
+
+// Evaluate model on training examples and compute training error
+val valuesAndPreds = parsedData.map { point =>
+  val prediction = model.predict(point.features)
+  (point.label, prediction)
+}
+val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count
+println("training Mean Squared Error = " + MSE)
+{% endhighlight %}
+
+
+Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training
+[Mean Squared Errors](http://en.wikipedia.org/wiki/Mean_squared_error).
+
+## Clustering
+In the following example after loading and parsing data, we use the KMeans object to cluster the data
+into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within
+Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the
+optimal *k* is usually one where there is an "elbow" in the WSSSE graph.
+
+{% highlight scala %}
+import org.apache.spark.mllib.clustering.KMeans
+
+// Load and parse the data
+val data = sc.textFile("kmeans_data.txt")
+val parsedData = data.map( _.split(' ').map(_.toDouble))
+
+// Cluster the data into two classes using KMeans
+val numIterations = 20
+val numClusters = 2
+val clusters = KMeans.train(parsedData, numClusters, numIterations)
+
+// Evaluate clustering by computing Within Set Sum of Squared Errors
+val WSSSE = clusters.computeCost(parsedData)
+println("Within Set Sum of Squared Errors = " + WSSSE)
+{% endhighlight %}
+
+
+## Collaborative Filtering
+In the following example we load rating data. Each row consists of a user, a product and a rating.
+We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation
+model by measuring the Mean Squared Error of rating prediction.
+
+{% highlight scala %}
+import org.apache.spark.mllib.recommendation.ALS
+import org.apache.spark.mllib.recommendation.Rating
+
+// Load and parse the data
+val data = sc.textFile("mllib/data/als/test.data")
+val ratings = data.map(_.split(',') match {
+    case Array(user, item, rate) =>  Rating(user.toInt, item.toInt, rate.toDouble)
+})
+
+// Build the recommendation model using ALS
+val numIterations = 20
+val model = ALS.train(ratings, 1, 20, 0.01)
+
+// Evaluate the model on rating data
+val usersProducts = ratings.map{ case Rating(user, product, rate)  => (user, product)}
+val predictions = model.predict(usersProducts).map{
+    case Rating(user, product, rate) => ((user, product), rate)
+}
+val ratesAndPreds = ratings.map{
+    case Rating(user, product, rate) => ((user, product), rate)
+}.join(predictions)
+val MSE = ratesAndPreds.map{
+    case ((user, product), (r1, r2)) =>  math.pow((r1- r2), 2)
+}.reduce(_ + _)/ratesAndPreds.count
+println("Mean Squared Error = " + MSE)
+{% endhighlight %}
+
+If the rating matrix is derived from other source of information (i.e., it is inferred from
+other signals), you can use the trainImplicit method to get better results.
+
+{% highlight scala %}
+val model = ALS.trainImplicit(ratings, 1, 20, 0.01)
+{% endhighlight %}
+
+# Using MLLib in Python
+Following examples can be tested in the PySpark shell.
+
+## Binary Classification
+The following example shows how to load a sample dataset, build Logistic Regression model,
+and make predictions with the resulting model to compute the training error.
+
+{% highlight python %}
+from pyspark.mllib.classification import LogisticRegressionWithSGD
+from numpy import array
+
+# Load and parse the data
+data = sc.textFile("mllib/data/sample_svm_data.txt")
+parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')]))
+model = LogisticRegressionWithSGD.train(sc, parsedData)
+
+# Build the model
+labelsAndPreds = parsedData.map(lambda point: (int(point.item(0)),
+        model.predict(point.take(range(1, point.size)))))
+
+# Evaluating the model on training data
+trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count())
+print("Training Error = " + str(trainErr))
+{% endhighlight %}
+
+## Linear Regression
+The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The
+example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We
+compute the Mean Squared Error at the end to evaluate
+[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit)
+
+{% highlight python %}
+from pyspark.mllib.regression import LinearRegressionWithSGD
+from numpy import array
+
+# Load and parse the data
+data = sc.textFile("mllib/data/ridge-data/lpsa.data")
+parsedData = data.map(lambda line: array([float(x) for x in line.replace(',', ' ').split(' ')]))
+
+# Build the model
+model = LinearRegressionWithSGD.train(sc, parsedData)
+
+# Evaluate the model on training data
+valuesAndPreds = parsedData.map(lambda point: (point.item(0),
+        model.predict(point.take(range(1, point.size)))))
+MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y)/valuesAndPreds.count()
+print("Mean Squared Error = " + str(MSE))
+{% endhighlight %}
+
+
+## Clustering
+In the following example after loading and parsing data, we use the KMeans object to cluster the data
+into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within
+Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the
+optimal *k* is usually one where there is an "elbow" in the WSSSE graph.
+
+{% highlight python %}
+from pyspark.mllib.clustering import KMeans
+from numpy import array
+from math import sqrt
+
+# Load and parse the data
+data = sc.textFile("kmeans_data.txt")
+parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')]))
+
+# Build the model (cluster the data)
+clusters = KMeans.train(sc, parsedData, 2, maxIterations=10,
+        runs=30, initialization_mode="random")
+
+# Evaluate clustering by computing Within Set Sum of Squared Errors
+def error(point):
+    center = clusters.centers[clusters.predict(point)]
+    return sqrt(sum([x**2 for x in (point - center)]))
+
+WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y)
+print("Within Set Sum of Squared Error = " + str(WSSSE))
+{% endhighlight %}
+
+Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training Mean Squared
+Errors.
+
+## Collaborative Filtering
+In the following example we load rating data. Each row consists of a user, a product and a rating.
+We use the default ALS.train() method which assumes ratings are explicit. We evaluate the
+recommendation by measuring the Mean Squared Error of rating prediction.
+
+{% highlight python %}
+from pyspark.mllib.recommendation import ALS
+from numpy import array
+
+# Load and parse the data
+data = sc.textFile("mllib/data/als/test.data")
+ratings = data.map(lambda line: array([float(x) for x in line.split(',')]))
+
+# Build the recommendation model using Alternating Least Squares
+model = ALS.train(sc, ratings, 1, 20)
+
+# Evaluate the model on training data
+testdata = ratings.map(lambda p: (int(p[0]), int(p[1])))
+predictions = model.predictAll(testdata).map(lambda r: ((r[0], r[1]), r[2]))
+ratesAndPreds = ratings.map(lambda r: ((r[0], r[1]), r[2])).join(predictions)
+MSE = ratesAndPreds.map(lambda r: (r[1][0] - r[1][1])**2).reduce(lambda x, y: x + y)/ratesAndPreds.count()
+print("Mean Squared Error = " + str(MSE))
+{% endhighlight %}
+
+If the rating matrix is derived from other source of information (i.e., it is inferred from other
+signals), you can use the trainImplicit method to get better results.
+
+{% highlight python %}
+# Build the recommendation model using Alternating Least Squares based on implicit ratings
+model = ALS.trainImplicit(sc, ratings, 1, 20)
+{% endhighlight %}
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 5ed0474477302b2768bdd59e6a66afea867a126c..0d5eb7065e9f0a9ad7d12d5171b75310836346cb 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -32,7 +32,8 @@ Spark has a configurable metrics system based on the
 This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV 
 files. The metrics system is configured via a configuration file that Spark expects to be present 
 at `$SPARK_HOME/conf/metrics.conf`. A custom file location can be specified via the 
-`spark.metrics.conf` Java system property. Spark's metrics are decoupled into different 
+`spark.metrics.conf` [configuration property](configuration.html#spark-properties).
+Spark's metrics are decoupled into different 
 _instances_ corresponding to Spark components. Within each instance, you can configure a 
 set of sinks to which metrics are reported. The following instances are currently supported:
 
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 55e39b1de17a06c4c4a9e025769c66b0977cffb8..dc187b3efec9b7b7b8f6075275a4faf9da40b6fe 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -47,7 +47,7 @@ PySpark will automatically ship these functions to workers, along with any objec
 Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers.
 The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers.
 
-In addition, PySpark fully supports interactive use---simply run `./pyspark` to launch an interactive shell.
+In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell.
 
 
 # Installing and Configuring PySpark
@@ -60,17 +60,17 @@ By default, PySpark requires `python` to be available on the system `PATH` and u
 
 All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported.
 
-Standalone PySpark applications should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
-The script automatically adds the `pyspark` package to the `PYTHONPATH`.
+Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
+The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`.
 
 
 # Interactive Use
 
-The `pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
+The `bin/pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
 
 {% highlight bash %}
 $ sbt/sbt assembly
-$ ./pyspark
+$ ./bin/pyspark
 {% endhighlight %}
 
 The Python shell can be used explore data interactively and is a simple way to learn the API:
@@ -82,35 +82,35 @@ The Python shell can be used explore data interactively and is a simple way to l
 >>> help(pyspark) # Show all pyspark functions
 {% endhighlight %}
 
-By default, the `pyspark` shell creates SparkContext that runs applications locally on a single core.
+By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on a single core.
 To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable.
-For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
+For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
 
 {% highlight bash %}
-$ MASTER=spark://IP:PORT ./pyspark
+$ MASTER=spark://IP:PORT ./bin/pyspark
 {% endhighlight %}
 
 Or, to use four cores on the local machine:
 
 {% highlight bash %}
-$ MASTER=local[4] ./pyspark
+$ MASTER=local[4] ./bin/pyspark
 {% endhighlight %}
 
 
 ## IPython
 
 It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter.
-To do this, set the `IPYTHON` variable to `1` when running `pyspark`:
+To do this, set the `IPYTHON` variable to `1` when running `bin/pyspark`:
 
 {% highlight bash %}
-$ IPYTHON=1 ./pyspark
+$ IPYTHON=1 ./bin/pyspark
 {% endhighlight %}
 
 Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch
 the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support:
 
 {% highlight bash %}
-$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark
+$ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark
 {% endhighlight %}
 
 IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable.
@@ -118,7 +118,7 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env
 
 # Standalone Programs
 
-PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`.
+PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`.
 The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application.
 
 Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor:
@@ -131,15 +131,16 @@ sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg
 Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines.
 Code dependencies can be added to an existing SparkContext using its `addPyFile()` method.
 
-You can set [system properties](configuration.html#system-properties)
-using `SparkContext.setSystemProperty()` class method *before*
-instantiating SparkContext. For example, to set the amount of memory
-per executor process:
+You can set [configuration properties](configuration.html#spark-properties) by passing a
+[SparkConf](api/pyspark/pyspark.conf.SparkConf-class.html) object to SparkContext:
 
 {% highlight python %}
-from pyspark import SparkContext
-SparkContext.setSystemProperty('spark.executor.memory', '2g')
-sc = SparkContext("local", "App Name")
+from pyspark import SparkConf, SparkContext
+conf = (SparkConf()
+         .setMaster("local")
+         .setAppName("My app")
+         .set("spark.executor.memory", "1g"))
+sc = SparkContext(conf = conf)
 {% endhighlight %}
 
 # API Docs
@@ -152,6 +153,6 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc
 PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples).
 You can run them by passing the files to `pyspark`; e.g.:
 
-    ./pyspark python/examples/wordcount.py
+    ./bin/pyspark python/examples/wordcount.py
 
 Each program prints usage help when run without arguments.
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 8f782db5b822b5a60a22c94be84b01cbb5bf9eeb..153081bdaa2863d732956e28dc6e9ad588769f27 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -20,7 +20,7 @@ $ sbt/sbt assembly
 ## Basics
 
 Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively.
-Start the shell by running `./spark-shell` in the Spark directory.
+Start the shell by running `./bin/spark-shell` in the Spark directory.
 
 Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory:
 
@@ -99,7 +99,7 @@ scala> linesWithSpark.count()
 res9: Long = 15
 {% endhighlight %}
 
-It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
+It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
 
 # A Standalone App in Scala
 Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
@@ -124,7 +124,7 @@ object SimpleApp {
 }
 {% endhighlight %}
 
-This program simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
 
 This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on:
 
@@ -146,7 +146,7 @@ If you also wish to read data from Hadoop's HDFS, you will also need to add a de
 libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "<your-hdfs-version>"
 {% endhighlight %}
 
-Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt run` to execute our program.
+Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt/sbt run` to execute our program.
 
 {% highlight bash %}
 $ find .
@@ -157,8 +157,8 @@ $ find .
 ./src/main/scala
 ./src/main/scala/SimpleApp.scala
 
-$ sbt package
-$ sbt run
+$ sbt/sbt package
+$ sbt/sbt run
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight %}
@@ -193,7 +193,7 @@ public class SimpleApp {
 }
 {% endhighlight %}
 
-This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
 
 To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
 
@@ -224,11 +224,11 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a
 If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS:
 
 {% highlight xml %}
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-      <version>...</version>
-    </dependency>
+<dependency>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-client</artifactId>
+  <version>...</version>
+</dependency>
 {% endhighlight %}
 
 We lay out these files according to the canonical Maven directory structure:
@@ -270,18 +270,18 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs)
 {% endhighlight %}
 
 
-This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a text file.
 Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. 
 As with the Scala and Java examples, we use a SparkContext to create RDDs.
 We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference.
 For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
 `SimpleApp` is simple enough that we do not need to specify any code dependencies.
 
-We can run this application using the `pyspark` script:
+We can run this application using the `bin/pyspark` script:
 
 {% highlight python %}
 $ cd $SPARK_HOME
-$ ./pyspark SimpleApp.py
+$ ./bin/pyspark SimpleApp.py
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight python %}
@@ -301,14 +301,38 @@ assembly jar (or "uber" jar) containing your code and its dependencies. Both
 have assembly plugins. When creating assembly jars, list Spark 
 itself as a `provided` dependency; it need not be bundled since it is 
 already present on the slaves. Once you have an assembled jar, 
-add it to the SparkContext as shown here. It is also possible to submit 
-your dependent jars one-by-one when creating a SparkContext.
+add it to the SparkContext as shown here. It is also possible to add
+your dependent jars one-by-one using the `addJar` method of `SparkContext`.
+
+For Python, you can use the `pyFiles` argument of SparkContext
+or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed.
 
 ### Setting Configuration Options
-Spark includes several configuration options which influence the behavior
-of your application. These should be set as 
-[JVM system properties](configuration.html#system-properties) in your 
-program. The options will be captured and shipped to all slave nodes.
+Spark includes several [configuration options](configuration.html#spark-properties)
+that influence the behavior of your application.
+These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+object and passing it to the SparkContext constructor.
+For example, in Java and Scala, you can do:
+
+{% highlight scala %}
+import org.apache.spark.{SparkConf, SparkContext}
+val conf = new SparkConf()
+             .setMaster("local")
+             .setAppName("My application")
+             .set("spark.executor.memory", "1g")
+val sc = new SparkContext(conf)
+{% endhighlight %}
+
+Or in Python:
+
+{% highlight scala %}
+from pyspark import SparkConf, SparkContext
+conf = SparkConf()
+conf.setMaster("local")
+conf.setAppName("My application")
+conf.set("spark.executor.memory", "1g"))
+sc = SparkContext(conf = conf)
+{% endhighlight %}
 
 ### Accessing Hadoop Filesystems
 
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index 322ff585f18786809faee52542604ad9b3903c78..68259f0cb8ad13a6bcc7e835871434e1982e6dd0 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -15,15 +15,16 @@ Spark can run on clusters managed by [Apache Mesos](http://mesos.apache.org/). F
    * `export MESOS_NATIVE_LIBRARY=<path to libmesos.so>`. This path is usually `<prefix>/lib/libmesos.so` (where the prefix is `/usr/local` by default, see above). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`.
    * `export SPARK_EXECUTOR_URI=<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>`.
    * `export MASTER=mesos://HOST:PORT` where HOST:PORT is the host and port (default: 5050) of your Mesos master (or `zk://...` if using Mesos with ZooKeeper).
-8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the first parameter. In addition, you'll need to set the `spark.executor.uri` property. For example:
+8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the master URL. In addition, you'll need to set the `spark.executor.uri` property. For example:
 
 {% highlight scala %}
-System.setProperty("spark.executor.uri", "<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>")
-val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
+val conf = new SparkConf()
+  .setMaster("mesos://HOST:5050")
+  .setAppName("My app")
+  .set("spark.executor.uri", "<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
-If you want to run Spark on Amazon EC2, you can use the Spark [EC2 launch scripts](ec2-scripts.html), which provide an easy way to launch a cluster with Mesos, Spark, and HDFS pre-configured. This will get you a cluster in about five minutes without any configuration on your part.
-
 # Mesos Run Modes
 
 Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default,
@@ -34,17 +35,15 @@ launch only *one* long-running Spark task on each Mesos machine, and dynamically
 it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration
 of the application.
 
-To run in coarse-grained mode, set the `spark.mesos.coarse` system property to true *before* creating your SparkContext:
+To run in coarse-grained mode, set the `spark.mesos.coarse` property in your [SparkConf](configuration.html#spark-properties):
 
 {% highlight scala %}
-System.setProperty("spark.mesos.coarse", "true")
-val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
+conf.set("spark.mesos.coarse", "true")
 {% endhighlight %}
 
 In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default,
 it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just one
-application at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
-Again, this must be done *before* initializing a SparkContext.
+application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example).
 
 
 # Running Alongside Hadoop
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index aa75ca43241fb5fa831895a8c1871d36bcebbccc..717071d72c9b9ef93eb567a45cda24c82fd22bd1 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -12,7 +12,7 @@ was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0.
 We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster.
 This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows:
 
-    SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly
+    SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly
 
 The assembled JAR will be something like this:
 `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`.
@@ -54,7 +54,7 @@ There are two scheduler mode that can be used to launch spark application on YAR
 
 The command to launch the YARN Client is as follows:
 
-    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
+    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./bin/spark-class org.apache.spark.deploy.yarn.Client \
       --jar <YOUR_APP_JAR_FILE> \
       --class <APP_MAIN_CLASS> \
       --args <APP_MAIN_ARGUMENTS> \
@@ -72,14 +72,14 @@ The command to launch the YARN Client is as follows:
 For example:
 
     # Build the Spark assembly JAR and the Spark examples JAR
-    $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly
+    $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly
 
     # Configure logging
     $ cp conf/log4j.properties.template conf/log4j.properties
 
     # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example
     $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
-        ./spark-class org.apache.spark.deploy.yarn.Client \
+        ./bin/spark-class org.apache.spark.deploy.yarn.Client \
           --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
           --class org.apache.spark.examples.SparkPi \
           --args yarn-standalone \
@@ -107,17 +107,15 @@ For example:
 
     SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
     SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
-    ./run-example org.apache.spark.examples.SparkPi yarn-client
+    ./bin/run-example org.apache.spark.examples.SparkPi yarn-client
 
 
     SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
     SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
-    MASTER=yarn-client ./spark-shell
+    MASTER=yarn-client ./bin/spark-shell
 
 # Building Spark for Hadoop/YARN 2.2.x
 
-Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. Furthermore, the build process then uses the directory ```new-yarn``` (instead of ```yarn```), which supports the new YARN API. The build process should seamlessly work out of the box. 
-
 See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process.
 
 # Important Notes
@@ -126,4 +124,3 @@ See [Building Spark with Maven](building-with-maven.html) for instructions on ho
 - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored.
 - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN.
 - The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files.
-- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally.  
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 56d2a3a4a020282b935b9563dc43455367b6e6b8..c1ef46a1cded743d59f7d1e333d3936e001bd3df 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -13,7 +13,7 @@ At a high level, every Spark application consists of a *driver program* that run
 
 A second abstraction in Spark is *shared variables* that can be used in parallel operations. By default, when Spark runs a function in parallel as a set of tasks on different nodes, it ships a copy of each variable used in the function to each task. Sometimes, a variable needs to be shared across tasks, or between tasks and the driver program. Spark supports two types of shared variables: *broadcast variables*, which can be used to cache a value in memory on all nodes, and *accumulators*, which are variables that are only "added" to, such as counters and sums.
 
-This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `spark-shell` script. We highly recommend doing that to follow along!
+This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `bin/spark-shell` script. We highly recommend doing that to follow along!
 
 # Linking with Spark
 
@@ -49,18 +49,21 @@ This is done through the following constructor:
 new SparkContext(master, appName, [sparkHome], [jars])
 {% endhighlight %}
 
+or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+object for more advanced configuration.
+
 The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
 
-In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use
+In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on four cores, use
 
 {% highlight bash %}
-$ MASTER=local[4] ./spark-shell
+$ MASTER=local[4] ./bin/spark-shell
 {% endhighlight %}
 
 Or, to also add `code.jar` to its classpath, use:
 
 {% highlight bash %}
-$ MASTER=local[4] ADD_JARS=code.jar ./spark-shell
+$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell
 {% endhighlight %}
 
 ### Master URLs
@@ -92,8 +95,7 @@ If you want to run your application on a cluster, you will need to specify the t
 * `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them).
 * `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
 
-If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it.  This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
-
+If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it.  This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
 
 # Resilient Distributed Datasets (RDDs)
 
@@ -364,9 +366,9 @@ res2: Int = 10
 # Where to Go from Here
 
 You can see some [example Spark programs](http://spark.incubator.apache.org/examples.html) on the Spark website.
-In addition, Spark includes several samples 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:
+In addition, Spark includes several samples 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 `bin/run-example` script included in Spark; for example:
 
-    ./run-example org.apache.spark.examples.SparkPi
+    ./bin/run-example org.apache.spark.examples.SparkPi
 
 Each example program prints usage help when run without any arguments.
 
diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md
index d6315d97f4d590d47c6e85cb108bc70368ea2323..11c51d5cde7c929fc44a6d6f8368d5e657737b2a 100644
--- a/docs/spark-debugger.md
+++ b/docs/spark-debugger.md
@@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP
 
 ### Loading the event log into the debugger
 
-1. Run a Spark shell with `MASTER=<i>host</i> ./spark-shell`.
+1. Run a Spark shell with `MASTER=<i>host</i> ./bin/spark-shell`.
 2. Use `EventLogReader` to load the event log as follows:
     {% highlight scala %}
 spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log"))
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 59adbce156d0010dd15d32ca77020d7c63122296..2a186261b754ab8a29841d16913d7d69e7d1dda8 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -16,7 +16,7 @@ To install Spark Standlone mode, you simply place a compiled version of Spark on
 
 You can start a standalone master server by executing:
 
-    ./bin/start-master.sh
+    ./sbin/start-master.sh
 
 Once started, the master will print out a `spark://HOST:PORT` URL for itself, which you can use to connect workers to it,
 or pass as the "master" argument to `SparkContext`. You can also find this URL on
@@ -24,7 +24,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:
 
-    ./spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
+    ./bin/spark-class org.apache.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).
@@ -66,12 +66,12 @@ To launch a Spark standalone cluster with the launch scripts, you need to create
 
 Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
 
-- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
-- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
-- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
-- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
-- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
-- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
+- `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on.
+- `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
+- `sbin/start-all.sh` - Starts both a master and a number of slaves as described above.
+- `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
+- `sbin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
+- `sbin/stop-all.sh` - Stops both the master and the slaves as described above.
 
 Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine.
 
@@ -139,9 +139,9 @@ constructor](scala-programming-guide.html#initializing-spark).
 
 To run an interactive Spark shell against the cluster, run the following command:
 
-    MASTER=spark://IP:PORT ./spark-shell
+    MASTER=spark://IP:PORT ./bin/spark-shell
 
-Note that if you are running spark-shell from one of the spark cluster machines, the `spark-shell` script will
+Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will
 automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`.
 
 You can also pass an option `-c <numCores>` to control the number of cores that spark-shell uses on the cluster.
@@ -151,26 +151,26 @@ You can also pass an option `-c <numCores>` to control the number of cores that
 You may also run your application entirely inside of the cluster by submitting your application driver using the submission client. The syntax for submitting applications is as follows:
 
 
-    ./spark-class org.apache.spark.deploy.client.DriverClient launch 
+    ./spark-class org.apache.spark.deploy.Client launch 
        [client-options] \
        <cluster-url> <application-jar-url> <main-class> \
        [application-options]
 
     cluster-url: The URL of the master node.
-    application-jar-url: Path to a bundled jar including your application and all dependencies.
-                         Accepts hdfs://, file://, and http:// paths.
+    application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. 
     main-class: The entry point for your application.
 
     Client Options:
       --memory <count> (amount of memory, in MB, allocated for your driver program)
       --cores <count> (number of cores allocated for your driver program)
       --supervise (whether to automatically restart your driver on application or node failure)
+      --verbose (prints increased logging output)
 
 Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways:
 
  * _Environment variables_: These will be captured from the environment in which you launch the client and applied when launching the driver program.
  * _Java options_: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in which you launch the submission client.
-  * _Dependencies_: You'll still need to call `sc.addJar` inside of your driver program to add your application jar and any dependencies. If you submit a local application jar to the client (e.g one with a `file://` URL), it will be uploaded into the working directory of your driver program. Then, you can add it using `sc.addJar("jar-name.jar")`.
+ * _Dependencies_: You'll still need to call `sc.addJar` inside of your program to make your bundled application jar visible on all worker nodes.
 
 Once you submit a driver program, it will appear in the cluster management UI at port 8080 and
 be assigned an identifier. If you'd like to prematurely terminate the program, you can do so using
@@ -182,12 +182,29 @@ the same client:
 
 The standalone cluster mode currently only supports a simple FIFO scheduler across applications.
 However, to allow multiple concurrent users, you can control the maximum number of resources each
-application will acquire.
+application will use.
 By default, it will acquire *all* cores in the cluster, which only makes sense if you just run one
-application at a time. You can cap the number of cores using
-`System.setProperty("spark.cores.max", "10")` (for example).
-This value must be set *before* initializing your SparkContext.
-
+application at a time. You can cap the number of cores by setting `spark.cores.max` in your
+[SparkConf](configuration.html#spark-properties). For example:
+
+{% highlight scala %}
+val conf = new SparkConf()
+             .setMaster(...)
+             .setAppName(...)
+             .set("spark.cores.max", "10")
+val sc = new SparkContext(conf)
+{% endhighlight %}
+
+In addition, you can configure `spark.deploy.defaultCores` on the cluster master process to change the
+default for applications that don't set `spark.cores.max` to something less than infinite.
+Do this by adding the following to `conf/spark-env.sh`:
+
+{% highlight bash %}
+export SPARK_JAVA_OPTS="-Dspark.deploy.defaultCores=<value>"
+{% endhighlight %}
+
+This is useful on shared clusters where users might not have configured a maximum number of cores
+individually.
 
 # Monitoring and Logging
 
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 82f42e0b8d16a74e2d3e9b4881a6e6bfdd75c93a..1c9ece62707818cfc403103269a81c2448dc3552 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -245,7 +245,7 @@ $ nc -lk 9999
 Then, in a different terminal, you can start NetworkWordCount by using
 
 {% highlight bash %}
-$ ./run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+$ ./bin/run-example org.apache.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.
@@ -283,7 +283,7 @@ Time: 1357008430000 ms
 </td>
 </table>
 
-You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.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/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./bin/run-example org.apache.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()`.
@@ -326,7 +326,7 @@ Getting the best performance of a Spark Streaming application on a cluster requi
 There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones.
 
 ### Level of Parallelism
-Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default.
+Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default.
 
 ### Data Serialization
 The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it.
@@ -349,7 +349,7 @@ For a Spark Streaming application running on a cluster to be stable, the process
 A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size).
 
 ## 24/7 Operation
-By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
+By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the [config property](configuration.html#spark-properties) `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
 
 This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so.
 
diff --git a/docs/tuning.md b/docs/tuning.md
index a4be18816928e7fe08c24b76186af63131ed03cd..bbb870085c249ec4703b95640dda533ce153ce01 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -38,14 +38,15 @@ in your operations) and performance. It provides two serialization libraries:
   `Serializable` types and requires you to *register* the classes you'll use in the program in advance
   for best performance.
 
-You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`
-*before* creating your SparkContext. This setting configures the serializer used for not only shuffling data between worker
+You can switch to using Kryo by initializing your job with a [SparkConf](configuration.html#spark-properties)
+and calling `conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`.
+This setting configures the serializer used for not only shuffling data between worker
 nodes but also when serializing RDDs to disk.  The only reason Kryo is not the default is because of the custom
 registration requirement, but we recommend trying it in any network-intensive application.
 
 Finally, to register your classes with Kryo, create a public class that extends
 [`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the
-`spark.kryo.registrator` system property to point to it, as follows:
+`spark.kryo.registrator` config property to point to it, as follows:
 
 {% highlight scala %}
 import com.esotericsoftware.kryo.Kryo
@@ -58,17 +59,17 @@ class MyRegistrator extends KryoRegistrator {
   }
 }
 
-// Make sure to set these properties *before* creating a SparkContext!
-System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator")
-val sc = new SparkContext(...)
+val conf = new SparkConf().setMaster(...).setAppName(...)
+conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+conf.set("spark.kryo.registrator", "mypackage.MyRegistrator")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
 The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced
 registration options, such as adding custom serialization code.
 
 If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb`
-system property. The default is 2, but this value needs to be large enough to hold the *largest*
+config property. The default is 2, but this value needs to be large enough to hold the *largest*
 object you will serialize.
 
 Finally, if you don't register your classes, Kryo will still work, but it will have to store the
@@ -165,7 +166,7 @@ cache RDDs. This means that 33% of memory is available for any objects created d
 
 In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of
 memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call
-`System.setProperty("spark.storage.memoryFraction", "0.5")`. Combined with the use of serialized caching,
+`conf.set("spark.storage.memoryFraction", "0.5")` on your SparkConf. Combined with the use of serialized caching,
 using a smaller cache should be sufficient to mitigate most of the garbage collection problems.
 In case you are interested in further tuning the Java GC, continue reading below.
 
@@ -219,7 +220,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a
 distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest
 parent RDD's number of partitions. You can pass the level of parallelism as a second argument
 (see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation),
-or set the system property `spark.default.parallelism` to change the default.
+or set the config property `spark.default.parallelism` to change the default.
 In general, we recommend 2-3 tasks per CPU core in your cluster.
 
 ## Memory Usage of Reduce Tasks
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index a2b0e7e7f47480f2bfd42fbc4951067a161c82d1..d82a1e1490cc0e7125030a7b93a10bfd1ddca208 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -436,7 +436,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
 def setup_standalone_cluster(master, slave_nodes, opts):
   slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes])
   ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips))
-  ssh(master, opts, "/root/spark/bin/start-all.sh")
+  ssh(master, opts, "/root/spark/sbin/start-all.sh")
 
 def setup_spark_cluster(master, opts):
   ssh(master, opts, "chmod u+x spark-ec2/setup.sh")
diff --git a/examples/pom.xml b/examples/pom.xml
index 7a7032c319b8fefdae06675cd2b078da0eed7efe..cb4f7ee33b4a1b435d46c5c8e7044d7aeaf24870 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -71,6 +71,31 @@
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-twitter_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-kafka_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-flume_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-zeromq_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-mqtt_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase</artifactId>
@@ -86,21 +111,6 @@
         </exclusion>
       </exclusions>
     </dependency>
-    <dependency>
-      <groupId>com.sksamuel.kafka</groupId>
-      <artifactId>kafka_${scala.binary.version}</artifactId>
-      <version>0.8.0-beta1</version>
-      <exclusions>
-        <exclusion>
-          <groupId>com.sun.jmx</groupId>
-          <artifactId>jmxri</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jdmk</groupId>
-          <artifactId>jmxtools</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-server</artifactId>
@@ -174,7 +184,7 @@
         <artifactId>maven-shade-plugin</artifactId>
         <configuration>
           <shadedArtifactAttached>false</shadedArtifactAttached>
-          <outputFile>${project.build.directory}/scala-${scala.version}/${project.artifactId}-assembly-${project.version}.jar</outputFile>
+          <outputFile>${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar</outputFile>
           <artifactSet>
             <includes>
               <include>*:*</include>
@@ -203,6 +213,9 @@
                 <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
                   <resource>reference.conf</resource>
                 </transformer>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+                  <resource>log4j.properties</resource>
+                </transformer>
               </transformers>
             </configuration>
           </execution>
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
index be0d38589c5df6f2925e06f7a081ff6c72057530..d552c47b22231b4d142c68bd7241e628419bfbf1 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
@@ -24,19 +24,19 @@ import org.apache.spark.api.java.function.Function2;
 
 import java.io.Serializable;
 import java.util.Arrays;
-import java.util.StringTokenizer;
 import java.util.Random;
+import java.util.regex.Pattern;
 
 /**
  * Logistic regression based classification.
  */
-public class JavaHdfsLR {
+public final class JavaHdfsLR {
 
-  static int D = 10;   // Number of dimensions
-  static Random rand = new Random(42);
+  private static final int D = 10;   // Number of dimensions
+  private static final Random rand = new Random(42);
 
   static class DataPoint implements Serializable {
-    public DataPoint(double[] x, double y) {
+    DataPoint(double[] x, double y) {
       this.x = x;
       this.y = y;
     }
@@ -46,20 +46,22 @@ public class JavaHdfsLR {
   }
 
   static class ParsePoint extends Function<String, DataPoint> {
+    private static final Pattern SPACE = Pattern.compile(" ");
+
+    @Override
     public DataPoint call(String line) {
-      StringTokenizer tok = new StringTokenizer(line, " ");
-      double y = Double.parseDouble(tok.nextToken());
+      String[] tok = SPACE.split(line);
+      double y = Double.parseDouble(tok[0]);
       double[] x = new double[D];
-      int i = 0;
-      while (i < D) {
-        x[i] = Double.parseDouble(tok.nextToken());
-        i += 1;
+      for (int i = 0; i < D; i++) {
+        x[i] = Double.parseDouble(tok[i + 1]);
       }
       return new DataPoint(x, y);
     }
   }
 
   static class VectorSum extends Function2<double[], double[], double[]> {
+    @Override
     public double[] call(double[] a, double[] b) {
       double[] result = new double[D];
       for (int j = 0; j < D; j++) {
@@ -70,12 +72,13 @@ public class JavaHdfsLR {
   }
 
   static class ComputeGradient extends Function<DataPoint, double[]> {
-    double[] weights;
+    private final double[] weights;
 
-    public ComputeGradient(double[] weights) {
+    ComputeGradient(double[] weights) {
       this.weights = weights;
     }
 
+    @Override
     public double[] call(DataPoint p) {
       double[] gradient = new double[D];
       for (int i = 0; i < D; i++) {
@@ -106,7 +109,7 @@ public class JavaHdfsLR {
     }
 
     JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR",
-        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+        System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaHdfsLR.class));
     JavaRDD<String> lines = sc.textFile(args[1]);
     JavaRDD<DataPoint> points = lines.map(new ParsePoint()).cache();
     int ITERATIONS = Integer.parseInt(args[2]);
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
index 5a6afe7eaefd6112a7348e77db4a40e1bc9136d9..0dc879275a22a373f4b740ec9fd144c542ea6b56 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
@@ -27,19 +27,24 @@ import org.apache.spark.util.Vector;
 
 import java.util.List;
 import java.util.Map;
+import java.util.regex.Pattern;
 
 /**
  * K-means clustering using Java API.
  */
-public class JavaKMeans {
+public final class JavaKMeans {
+
+  private static final Pattern SPACE = Pattern.compile(" ");
 
   /** Parses numbers split by whitespace to a vector */
   static Vector parseVector(String line) {
-    String[] splits = line.split(" ");
+    String[] splits = SPACE.split(line);
     double[] data = new double[splits.length];
     int i = 0;
-    for (String s : splits)
-      data[i] = Double.parseDouble(splits[i++]);
+    for (String s : splits) {
+      data[i] = Double.parseDouble(s);
+      i++;
+    }
     return new Vector(data);
   }
 
@@ -74,7 +79,7 @@ public class JavaKMeans {
       System.exit(1);
     }
     JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans",
-      System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+      System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class));
     String path = args[1];
     int K = Integer.parseInt(args[2]);
     double convergeDist = Double.parseDouble(args[3]);
@@ -82,7 +87,7 @@ public class JavaKMeans {
     JavaRDD<Vector> data = sc.textFile(path).map(
       new Function<String, Vector>() {
         @Override
-        public Vector call(String line) throws Exception {
+        public Vector call(String line) {
           return parseVector(line);
         }
       }
@@ -96,7 +101,7 @@ public class JavaKMeans {
       JavaPairRDD<Integer, Vector> closest = data.map(
         new PairFunction<Vector, Integer, Vector>() {
           @Override
-          public Tuple2<Integer, Vector> call(Vector vector) throws Exception {
+          public Tuple2<Integer, Vector> call(Vector vector) {
             return new Tuple2<Integer, Vector>(
               closestPoint(vector, centroids), vector);
           }
@@ -107,7 +112,8 @@ public class JavaKMeans {
       JavaPairRDD<Integer, List<Vector>> pointsGroup = closest.groupByKey();
       Map<Integer, Vector> newCentroids = pointsGroup.mapValues(
         new Function<List<Vector>, Vector>() {
-          public Vector call(List<Vector> ps) throws Exception {
+          @Override
+          public Vector call(List<Vector> ps) {
             return average(ps);
           }
         }).collectAsMap();
@@ -122,8 +128,9 @@ public class JavaKMeans {
     } while (tempDist > convergeDist);
 
     System.out.println("Final centers:");
-    for (Vector c : centroids)
+    for (Vector c : centroids) {
       System.out.println(c);
+    }
 
     System.exit(0);
 
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
index 407cd7ccfaee0c964f7e98a81ffc847e1244d603..9eb1cadd71d22fcdb4b51e09e40d973259123e25 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
@@ -35,9 +35,9 @@ import java.util.regex.Pattern;
 /**
  * Executes a roll up-style query against Apache logs.
  */
-public class JavaLogQuery {
+public final class JavaLogQuery {
 
-  public static List<String> exampleApacheLogs = Lists.newArrayList(
+  public static final List<String> exampleApacheLogs = Lists.newArrayList(
     "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " +
       "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " +
       "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " +
@@ -51,14 +51,14 @@ public class JavaLogQuery {
       "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " +
       "0 73.23.2.15 images.com 1358492557 - Whatup");
 
-  public static Pattern apacheLogRegex = Pattern.compile(
+  public static final Pattern apacheLogRegex = Pattern.compile(
     "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*");
 
   /** Tracks the total query count and number of aggregate bytes for a particular group. */
   public static class Stats implements Serializable {
 
-    private int count;
-    private int numBytes;
+    private final int count;
+    private final int numBytes;
 
     public Stats(int count, int numBytes) {
       this.count = count;
@@ -92,32 +92,32 @@ public class JavaLogQuery {
     if (m.find()) {
       int bytes = Integer.parseInt(m.group(7));
       return new Stats(1, bytes);
-    }
-    else
+    } else {
       return new Stats(1, 0);
+    }
   }
 
-  public static void main(String[] args) throws Exception {
+  public static void main(String[] args) {
     if (args.length == 0) {
       System.err.println("Usage: JavaLogQuery <master> [logFile]");
       System.exit(1);
     }
 
     JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery",
-      System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+      System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLogQuery.class));
 
     JavaRDD<String> dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs);
 
     JavaPairRDD<Tuple3<String, String, String>, Stats> extracted = dataSet.map(new PairFunction<String, Tuple3<String, String, String>, Stats>() {
       @Override
-      public Tuple2<Tuple3<String, String, String>, Stats> call(String s) throws Exception {
+      public Tuple2<Tuple3<String, String, String>, Stats> call(String s) {
         return new Tuple2<Tuple3<String, String, String>, Stats>(extractKey(s), extractStats(s));
       }
     });
 
     JavaPairRDD<Tuple3<String, String, String>, Stats> counts = extracted.reduceByKey(new Function2<Stats, Stats, Stats>() {
       @Override
-      public Stats call(Stats stats, Stats stats2) throws Exception {
+      public Stats call(Stats stats, Stats stats2) {
         return stats.merge(stats2);
       }
     });
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
index 89aed8f279654d87db2eacc879a3c560cbfe6db1..a84245b0c7449627e188bcef5972a2745bfea45f 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
@@ -28,6 +28,7 @@ import org.apache.spark.api.java.function.PairFunction;
 
 import java.util.List;
 import java.util.ArrayList;
+import java.util.regex.Pattern;
 
 /**
  * Computes the PageRank of URLs from an input file. Input file should
@@ -38,7 +39,9 @@ import java.util.ArrayList;
  * ...
  * where URL and their neighbors are separated by space(s).
  */
-public class JavaPageRank {
+public final class JavaPageRank {
+  private static final Pattern SPACES = Pattern.compile("\\s+");
+
   private static class Sum extends Function2<Double, Double, Double> {
     @Override
     public Double call(Double a, Double b) {
@@ -53,7 +56,7 @@ public class JavaPageRank {
     }
 
     JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank",
-      System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+      System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaPageRank.class));
 
     // Loads in input file. It should be in format of:
     //     URL         neighbor URL
@@ -66,7 +69,7 @@ public class JavaPageRank {
     JavaPairRDD<String, List<String>> links = lines.map(new PairFunction<String, String, String>() {
       @Override
       public Tuple2<String, String> call(String s) {
-        String[] parts = s.split("\\s+");
+        String[] parts = SPACES.split(s);
         return new Tuple2<String, String>(parts[0], parts[1]);
       }
     }).distinct().groupByKey().cache();
@@ -74,7 +77,7 @@ public class JavaPageRank {
     // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one.
     JavaPairRDD<String, Double> ranks = links.mapValues(new Function<List<String>, Double>() {
       @Override
-      public Double call(List<String> rs) throws Exception {
+      public Double call(List<String> rs) {
         return 1.0;
       }
     });
@@ -97,7 +100,7 @@ public class JavaPageRank {
       // Re-calculates URL ranks based on neighbor contributions.
       ranks = contribs.reduceByKey(new Sum()).mapValues(new Function<Double, Double>() {
         @Override
-        public Double call(Double sum) throws Exception {
+        public Double call(Double sum) {
           return 0.15 + sum * 0.85;
         }
       });
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
index 4a2380caf5af554c1daa13f198326a7961699e5c..3ec4a58d48ed6cea7c676f678d081a723453cb39 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
@@ -26,8 +26,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 /** Computes an approximation to pi */
-public class JavaSparkPi {
-
+public final class JavaSparkPi {
 
   public static void main(String[] args) throws Exception {
     if (args.length == 0) {
@@ -36,26 +35,27 @@ public class JavaSparkPi {
     }
 
     JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery",
-      System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+      System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkPi.class));
 
     int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2;
     int n = 100000 * slices;
     List<Integer> l = new ArrayList<Integer>(n);
-    for (int i = 0; i < n; i++)
+    for (int i = 0; i < n; i++) {
       l.add(i);
+    }
 
     JavaRDD<Integer> dataSet = jsc.parallelize(l, slices);
 
     int count = dataSet.map(new Function<Integer, Integer>() {
       @Override
-      public Integer call(Integer integer) throws Exception {
+      public Integer call(Integer integer) {
         double x = Math.random() * 2 - 1;
         double y = Math.random() * 2 - 1;
         return (x * x + y * y < 1) ? 1 : 0;
       }
     }).reduce(new Function2<Integer, Integer, Integer>() {
       @Override
-      public Integer call(Integer integer, Integer integer2) throws Exception {
+      public Integer call(Integer integer, Integer integer2) {
         return integer + integer2;
       }
     });
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
index 17f21f6b776d152cfff1c733542fd9afb2c1975f..2ceb0fd94ba655c5819cf83c5863d6935d282a63 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
@@ -31,11 +31,11 @@ import java.util.Set;
 /**
  * Transitive closure on a graph, implemented in Java.
  */
-public class JavaTC {
+public final class JavaTC {
 
-  static int numEdges = 200;
-  static int numVertices = 100;
-  static Random rand = new Random(42);
+  private static final int numEdges = 200;
+  private static final int numVertices = 100;
+  private static final Random rand = new Random(42);
 
   static List<Tuple2<Integer, Integer>> generateGraph() {
     Set<Tuple2<Integer, Integer>> edges = new HashSet<Tuple2<Integer, Integer>>(numEdges);
@@ -43,15 +43,18 @@ public class JavaTC {
       int from = rand.nextInt(numVertices);
       int to = rand.nextInt(numVertices);
       Tuple2<Integer, Integer> e = new Tuple2<Integer, Integer>(from, to);
-      if (from != to) edges.add(e);
+      if (from != to) {
+        edges.add(e);
+      }
     }
     return new ArrayList<Tuple2<Integer, Integer>>(edges);
   }
 
   static class ProjectFn extends PairFunction<Tuple2<Integer, Tuple2<Integer, Integer>>,
       Integer, Integer> {
-    static ProjectFn INSTANCE = new ProjectFn();
+    static final ProjectFn INSTANCE = new ProjectFn();
 
+    @Override
     public Tuple2<Integer, Integer> call(Tuple2<Integer, Tuple2<Integer, Integer>> triple) {
       return new Tuple2<Integer, Integer>(triple._2()._2(), triple._2()._1());
     }
@@ -64,7 +67,7 @@ public class JavaTC {
     }
 
     JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC",
-        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+        System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaTC.class));
     Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2;
     JavaPairRDD<Integer, Integer> tc = sc.parallelizePairs(generateGraph(), slices).cache();
 
@@ -76,6 +79,7 @@ public class JavaTC {
     // Because join() joins on keys, the edges are stored in reversed order.
     JavaPairRDD<Integer, Integer> edges = tc.map(
       new PairFunction<Tuple2<Integer, Integer>, Integer, Integer>() {
+        @Override
         public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> e) {
           return new Tuple2<Integer, Integer>(e._2(), e._1());
         }
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
index bd6383e13df7a6c75180b26487a0ce0c961e49c5..6651f98d56711cb424dcf12138634fa1abf18812 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
@@ -27,8 +27,11 @@ import org.apache.spark.api.java.function.PairFunction;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.regex.Pattern;
+
+public final class JavaWordCount {
+  private static final Pattern SPACE = Pattern.compile(" ");
 
-public class JavaWordCount {
   public static void main(String[] args) throws Exception {
     if (args.length < 2) {
       System.err.println("Usage: JavaWordCount <master> <file>");
@@ -36,22 +39,25 @@ public class JavaWordCount {
     }
 
     JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount",
-        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+        System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaWordCount.class));
     JavaRDD<String> lines = ctx.textFile(args[1], 1);
 
     JavaRDD<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
+      @Override
       public Iterable<String> call(String s) {
-        return Arrays.asList(s.split(" "));
+        return Arrays.asList(SPACE.split(s));
       }
     });
     
     JavaPairRDD<String, Integer> ones = words.map(new PairFunction<String, String, Integer>() {
+      @Override
       public Tuple2<String, Integer> call(String s) {
         return new Tuple2<String, Integer>(s, 1);
       }
     });
     
     JavaPairRDD<String, Integer> counts = ones.reduceByKey(new Function2<Integer, Integer, Integer>() {
+      @Override
       public Integer call(Integer i1, Integer i2) {
         return i1 + i2;
       }
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
index 45a0d237da314187ab2869a854583d6f330656f2..435a86e62abc5943d881ce90f3fc9d6a7d97ffbe 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
@@ -26,28 +26,32 @@ import org.apache.spark.mllib.recommendation.MatrixFactorizationModel;
 import org.apache.spark.mllib.recommendation.Rating;
 
 import java.util.Arrays;
-import java.util.StringTokenizer;
+import java.util.regex.Pattern;
 
 import scala.Tuple2;
 
 /**
  * Example using MLLib ALS from Java.
  */
-public class  JavaALS {
+public final class JavaALS {
 
   static class ParseRating extends Function<String, Rating> {
+    private static final Pattern COMMA = Pattern.compile(",");
+
+    @Override
     public Rating call(String line) {
-      StringTokenizer tok = new StringTokenizer(line, ",");
-      int x = Integer.parseInt(tok.nextToken());
-      int y = Integer.parseInt(tok.nextToken());
-      double rating = Double.parseDouble(tok.nextToken());
+      String[] tok = COMMA.split(line);
+      int x = Integer.parseInt(tok[0]);
+      int y = Integer.parseInt(tok[1]);
+      double rating = Double.parseDouble(tok[2]);
       return new Rating(x, y, rating);
     }
   }
 
   static class FeaturesToString extends Function<Tuple2<Object, double[]>, String> {
+    @Override
     public String call(Tuple2<Object, double[]> element) {
-      return element._1().toString() + "," + Arrays.toString(element._2());
+      return element._1() + "," + Arrays.toString(element._2());
     }
   }
 
@@ -68,7 +72,7 @@ public class  JavaALS {
     }
 
     JavaSparkContext sc = new JavaSparkContext(args[0], "JavaALS",
-        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+        System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaALS.class));
     JavaRDD<String> lines = sc.textFile(args[1]);
 
     JavaRDD<Rating> ratings = lines.map(new ParseRating());
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
index cd59a139b9fee00c5faa413d2ca9a453d1ba9526..4b2658f257b3c9757d4260119da2b045d64853ef 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
@@ -25,20 +25,22 @@ import org.apache.spark.mllib.clustering.KMeans;
 import org.apache.spark.mllib.clustering.KMeansModel;
 
 import java.util.Arrays;
-import java.util.StringTokenizer;
+import java.util.regex.Pattern;
 
 /**
  * Example using MLLib KMeans from Java.
  */
-public class JavaKMeans {
+public final class JavaKMeans {
 
   static class ParsePoint extends Function<String, double[]> {
+    private static final Pattern SPACE = Pattern.compile(" ");
+
+    @Override
     public double[] call(String line) {
-      StringTokenizer tok = new StringTokenizer(line, " ");
-      int numTokens = tok.countTokens();
-      double[] point = new double[numTokens];
-      for (int i = 0; i < numTokens; ++i) {
-        point[i] = Double.parseDouble(tok.nextToken());
+      String[] tok = SPACE.split(line);
+      double[] point = new double[tok.length];
+      for (int i = 0; i < tok.length; ++i) {
+        point[i] = Double.parseDouble(tok[i]);
       }
       return point;
     }
@@ -62,7 +64,7 @@ public class JavaKMeans {
     }
 
     JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans",
-        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+        System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class));
     JavaRDD<String> lines = sc.textFile(args[1]);
 
     JavaRDD<double[]> points = lines.map(new ParsePoint());
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
index 258061c8e6ba627fd388b4991a1be62d40eb3277..21586ce817d093babf53d5b60b0ab72414c04fae 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
@@ -27,22 +27,25 @@ import org.apache.spark.mllib.classification.LogisticRegressionModel;
 import org.apache.spark.mllib.regression.LabeledPoint;
 
 import java.util.Arrays;
-import java.util.StringTokenizer;
+import java.util.regex.Pattern;
 
 /**
  * Logistic regression based classification using ML Lib.
  */
-public class JavaLR {
+public final class JavaLR {
 
   static class ParsePoint extends Function<String, LabeledPoint> {
+    private static final Pattern COMMA = Pattern.compile(",");
+    private static final Pattern SPACE = Pattern.compile(" ");
+
+    @Override
     public LabeledPoint call(String line) {
-      String[] parts = line.split(",");
+      String[] parts = COMMA.split(line);
       double y = Double.parseDouble(parts[0]);
-      StringTokenizer tok = new StringTokenizer(parts[1], " ");
-      int numTokens = tok.countTokens();
-      double[] x = new double[numTokens];
-      for (int i = 0; i < numTokens; ++i) {
-        x[i] = Double.parseDouble(tok.nextToken());
+      String[] tok = SPACE.split(parts[1]);
+      double[] x = new double[tok.length];
+      for (int i = 0; i < tok.length; ++i) {
+        x[i] = Double.parseDouble(tok[i]);
       }
       return new LabeledPoint(y, x);
     }
@@ -59,7 +62,7 @@ public class JavaLR {
     }
 
     JavaSparkContext sc = new JavaSparkContext(args[0], "JavaLR",
-        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+        System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLR.class));
     JavaRDD<String> lines = sc.textFile(args[1]);
     JavaRDD<LabeledPoint> points = lines.map(new ParsePoint()).cache();
     double stepSize = Double.parseDouble(args[2]);
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
index 261813bf2f39c49f61df463e864df12fdedad858..b11cfa667eb9238671b1126357e5d4e5642b8b51 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
@@ -20,7 +20,8 @@ package org.apache.spark.streaming.examples;
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.streaming.*;
 import org.apache.spark.streaming.api.java.*;
-import org.apache.spark.streaming.dstream.SparkFlumeEvent;
+import org.apache.spark.streaming.flume.FlumeUtils;
+import org.apache.spark.streaming.flume.SparkFlumeEvent;
 
 /**
  *  Produces a count of events received from Flume.
@@ -36,7 +37,10 @@ import org.apache.spark.streaming.dstream.SparkFlumeEvent;
  *           creates a server and listens for flume events.
  *    <port> is the port the Flume receiver will listen on.
  */
-public class JavaFlumeEventCount {
+public final class JavaFlumeEventCount {
+  private JavaFlumeEventCount() {
+  }
+
   public static void main(String[] args) {
     if (args.length != 3) {
       System.err.println("Usage: JavaFlumeEventCount <master> <host> <port>");
@@ -49,10 +53,10 @@ public class JavaFlumeEventCount {
 
     Duration batchInterval = new Duration(2000);
 
-    JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
-            System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
-
-    JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
+    JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
+            System.getenv("SPARK_HOME"),
+            JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class));
+    JavaDStream<SparkFlumeEvent> flumeStream = FlumeUtils.createStream(ssc, "localhost", port);
 
     flumeStream.count();
 
@@ -63,6 +67,6 @@ public class JavaFlumeEventCount {
       }
     }).print();
 
-    sc.start();
+    ssc.start();
   }
 }
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
index 9a8e4209eddc7da8d87401dafe81b46f3456065a..16b8a948e6154ad7527041efdfb546a2f859d11f 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -19,6 +19,7 @@ package org.apache.spark.streaming.examples;
 
 import java.util.Map;
 import java.util.HashMap;
+import java.util.regex.Pattern;
 
 import com.google.common.collect.Lists;
 import org.apache.spark.api.java.function.FlatMapFunction;
@@ -29,6 +30,7 @@ import org.apache.spark.streaming.Duration;
 import org.apache.spark.streaming.api.java.JavaDStream;
 import org.apache.spark.streaming.api.java.JavaPairDStream;
 import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.kafka.KafkaUtils;
 import scala.Tuple2;
 
 /**
@@ -41,11 +43,16 @@ import scala.Tuple2;
  *   <numThreads> is the number of threads the kafka consumer should use
  *
  * Example:
- *    `./run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
+ *    `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
  *    zoo03 my-consumer-group topic1,topic2 1`
  */
 
-public class JavaKafkaWordCount {
+public final class JavaKafkaWordCount {
+  private static final Pattern SPACE = Pattern.compile(" ");
+
+  private JavaKafkaWordCount() {
+  }
+
   public static void main(String[] args) {
     if (args.length < 5) {
       System.err.println("Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>");
@@ -53,8 +60,9 @@ public class JavaKafkaWordCount {
     }
 
     // Create the context with a 1 second batch size
-    JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount",
-            new Duration(2000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+    JavaStreamingContext jssc = new JavaStreamingContext(args[0], "KafkaWordCount",
+            new Duration(2000), System.getenv("SPARK_HOME"),
+            JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class));
 
     int numThreads = Integer.parseInt(args[4]);
     Map<String, Integer> topicMap = new HashMap<String, Integer>();
@@ -63,11 +71,11 @@ public class JavaKafkaWordCount {
       topicMap.put(topic, numThreads);
     }
 
-    JavaPairDStream<String, String> messages = ssc.kafkaStream(args[1], args[2], topicMap);
+    JavaPairDStream<String, String> messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap);
 
     JavaDStream<String> lines = messages.map(new Function<Tuple2<String, String>, String>() {
       @Override
-      public String call(Tuple2<String, String> tuple2) throws Exception {
+      public String call(Tuple2<String, String> tuple2) {
         return tuple2._2();
       }
     });
@@ -75,24 +83,24 @@ public class JavaKafkaWordCount {
     JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
       @Override
       public Iterable<String> call(String x) {
-        return Lists.newArrayList(x.split(" "));
+        return Lists.newArrayList(SPACE.split(x));
       }
     });
 
     JavaPairDStream<String, Integer> wordCounts = words.map(
       new PairFunction<String, String, Integer>() {
         @Override
-        public Tuple2<String, Integer> call(String s) throws Exception {
+        public Tuple2<String, Integer> call(String s) {
           return new Tuple2<String, Integer>(s, 1);
         }
       }).reduceByKey(new Function2<Integer, Integer, Integer>() {
         @Override
-        public Integer call(Integer i1, Integer i2) throws Exception {
+        public Integer call(Integer i1, Integer i2) {
           return i1 + i2;
         }
       });
 
     wordCounts.print();
-    ssc.start();
+    jssc.start();
   }
 }
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
index d8d6046914f72795604bccde378e56bb0ac07428..e96996aa75ce5dc11815716104a55b047d17edb5 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
@@ -27,6 +27,8 @@ import org.apache.spark.streaming.api.java.JavaDStream;
 import org.apache.spark.streaming.api.java.JavaPairDStream;
 import org.apache.spark.streaming.api.java.JavaStreamingContext;
 
+import java.util.regex.Pattern;
+
 /**
  * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
  * Usage: NetworkWordCount <master> <hostname> <port>
@@ -38,7 +40,12 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext;
  * and then run the example
  *    `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999`
  */
-public class JavaNetworkWordCount {
+public final class JavaNetworkWordCount {
+  private static final Pattern SPACE = Pattern.compile(" ");
+
+  private JavaNetworkWordCount() {
+  }
+
   public static void main(String[] args) {
     if (args.length < 3) {
       System.err.println("Usage: JavaNetworkWordCount <master> <hostname> <port>\n" +
@@ -48,7 +55,8 @@ public class JavaNetworkWordCount {
 
     // Create the context with a 1 second batch size
     JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount",
-            new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+            new Duration(1000), System.getenv("SPARK_HOME"),
+            JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class));
 
     // Create a NetworkInputDStream on target ip:port and count the
     // words in input stream of \n delimited text (eg. generated by 'nc')
@@ -56,18 +64,18 @@ public class JavaNetworkWordCount {
     JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
       @Override
       public Iterable<String> call(String x) {
-        return Lists.newArrayList(x.split(" "));
+        return Lists.newArrayList(SPACE.split(x));
       }
     });
     JavaPairDStream<String, Integer> wordCounts = words.map(
       new PairFunction<String, String, Integer>() {
         @Override
-        public Tuple2<String, Integer> call(String s) throws Exception {
+        public Tuple2<String, Integer> call(String s) {
           return new Tuple2<String, Integer>(s, 1);
         }
       }).reduceByKey(new Function2<Integer, Integer, Integer>() {
         @Override
-        public Integer call(Integer i1, Integer i2) throws Exception {
+        public Integer call(Integer i1, Integer i2) {
           return i1 + i2;
         }
       });
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
index c8c7389dd1bbaa11407f4d28ea1bbf08925c5b40..e05551ab833010df77cb761a17709583967fa84a 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
@@ -31,8 +31,11 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Queue;
 
-public class JavaQueueStream {
-  public static void main(String[] args) throws InterruptedException {
+public final class JavaQueueStream {
+  private JavaQueueStream() {
+  }
+
+  public static void main(String[] args) throws Exception {
     if (args.length < 1) {
       System.err.println("Usage: JavaQueueStream <master>");
       System.exit(1);
@@ -40,7 +43,7 @@ public class JavaQueueStream {
 
     // Create the context
     JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000),
-            System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+            System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaQueueStream.class));
 
     // Create the queue through which RDDs can be pushed to
     // a QueueInputDStream
@@ -62,14 +65,14 @@ public class JavaQueueStream {
     JavaPairDStream<Integer, Integer> mappedStream = inputStream.map(
         new PairFunction<Integer, Integer, Integer>() {
           @Override
-          public Tuple2<Integer, Integer> call(Integer i) throws Exception {
+          public Tuple2<Integer, Integer> call(Integer i) {
             return new Tuple2<Integer, Integer>(i % 10, 1);
           }
         });
     JavaPairDStream<Integer, Integer> reducedStream = mappedStream.reduceByKey(
       new Function2<Integer, Integer, Integer>() {
         @Override
-        public Integer call(Integer i1, Integer i2) throws Exception {
+        public Integer call(Integer i1, Integer i2) {
           return i1 + i2;
         }
     });
diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
index a1199809923bce78865d96dff4f23e53198aa4ab..0097dade190f6117ec70d9791eec01541396a800 100644
--- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -33,7 +33,7 @@ object BroadcastTest {
     System.setProperty("spark.broadcast.blockSize", blockSize)
 
     val sc = new SparkContext(args(0), "Broadcast Test",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     
     val slices = if (args.length > 1) args(1).toInt else 2
     val num = if (args.length > 2) args(2).toInt else 1000000
diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
index 92eb96bd8e0c2a04f5a26adbad79cc2a757f2be0..b3eb611dd228f7f6bc48f609b980f30ce30ca8a7 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
@@ -27,7 +27,7 @@ object ExceptionHandlingTest {
     }
 
     val sc = new SparkContext(args(0), "ExceptionHandlingTest",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     sc.parallelize(0 until sc.defaultParallelism).foreach { i =>
       if (math.random > 0.75)
         throw new Exception("Testing exception handling")
diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
index 42c2e0e8e19c4af07e28ea8cd19bda9b935b871b..39752fdd0eec4aba8ba3c02cb7678863e5923c71 100644
--- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
@@ -34,7 +34,7 @@ object GroupByTest {
     var numReducers = if (args.length > 4) args(4).toInt else numMappers
 
     val sc = new SparkContext(args(0), "GroupBy Test",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     
     val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
       val ranGen = new Random
diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
index efe2e93b0dc91cb7a77c532ca1084bc6fca3d593..65d67356be2f63b3c7750a5ef4795b3697c8491d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.mapreduce.TableInputFormat
 object HBaseTest {
   def main(args: Array[String]) {
     val sc = new SparkContext(args(0), "HBaseTest",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
 
     val conf = HBaseConfiguration.create()
 
diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
index d6a88d3032c494bbf2bf5f651a0041993fe09b79..c3597d94a224e27aaac8d40722155410e80e296a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
@@ -22,7 +22,7 @@ import org.apache.spark._
 object HdfsTest {
   def main(args: Array[String]) {
     val sc = new SparkContext(args(0), "HdfsTest",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     val file = sc.textFile(args(1))
     val mapped = file.map(s => s.length).cache()
     for (iter <- 1 to 10) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
index 17ff3ce76497f6297b583e3678c9ac06fcc9550a..bddb54b39cdd5468f35326f75a458b2122968506 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
@@ -45,7 +45,7 @@ object LogQuery {
     }
 
     val sc = new SparkContext(args(0), "Log Query",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
 
     val dataSet =
       if (args.length == 2) sc.textFile(args(1))
diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index e1afc29f9aca9cb170af59a07f7f3ea47da69bb6..4aef04fc060b6924cc21efd250493c138038266a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -28,7 +28,7 @@ object MultiBroadcastTest {
     }
 
     val sc = new SparkContext(args(0), "Multi-Broadcast Test",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
 
     val slices = if (args.length > 1) args(1).toInt else 2
     val num = if (args.length > 2) args(2).toInt else 1000000
diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
index 37ddfb5db7635ca2d7303b77105b74580a515efb..73b0e216cac9875300ea69c97587a4f8ed270f95 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
@@ -36,7 +36,7 @@ object SimpleSkewedGroupByTest {
     var ratio = if (args.length > 5) args(5).toInt else 5.0
 
     val sc = new SparkContext(args(0), "GroupBy Test",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
 
     val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
       val ranGen = new Random
diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
index 9c954b2b5baa996a0cdf433fe7c373371cdfd60c..31c6d108f34aed514bb10cb2b9a7765ba212cd26 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
@@ -34,7 +34,7 @@ object SkewedGroupByTest {
     var numReducers = if (args.length > 4) args(4).toInt else numMappers
 
     val sc = new SparkContext(args(0), "GroupBy Test",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
 
     val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
       val ranGen = new Random
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
index 814944ba1c6bf63f591ae3ca04b3e0b864f3fef8..30c86d83e688c5755f5c00c34ce4cd701ea7d160 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
@@ -112,7 +112,7 @@ object SparkALS {
     printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS)
 
     val sc = new SparkContext(host, "SparkALS",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     
     val R = generateR()
 
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
index 86dd9ca1b3e58032196de6d669307ab1ce009557..ff72532db1f17adedd5e4a7c7e76aa95223c18ae 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
@@ -54,7 +54,7 @@ object SparkHdfsLR {
     val inputPath = args(1)
     val conf = SparkHadoopUtil.get.newConfiguration()
     val sc = new SparkContext(args(0), "SparkHdfsLR",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), 
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), 
       InputFormatInfo.computePreferredLocations(
           Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath))))
     val lines = sc.textFile(inputPath)
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
index bc2db39c1231ba71af9a5fe1b346d692554ffc8e..8c99025eaa6da472335f74ca8371d3d60d8a73c5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -55,7 +55,7 @@ object SparkKMeans {
         System.exit(1)
     }
     val sc = new SparkContext(args(0), "SparkLocalKMeans",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     val lines = sc.textFile(args(1))
     val data = lines.map(parseVector _).cache()
     val K = args(2).toInt
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
index 9ed9fe4d761d5f8ab7a58b9b98137fe59085e779..c54a55bdb4a11f2b0b41216895a521742d3c2c25 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
@@ -49,7 +49,7 @@ object SparkLR {
       System.exit(1)
     }
     val sc = new SparkContext(args(0), "SparkLR",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     val numSlices = if (args.length > 1) args(1).toInt else 2
     val points = sc.parallelize(generateData, numSlices).cache()
 
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
index a508c0df577cde72510048067cf17412befbd597..d203f4d20e15f80bf0c6f4138837787d249e909a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
@@ -38,7 +38,7 @@ object SparkPageRank {
     }
     var iters = args(2).toInt
     val ctx = new SparkContext(args(0), "PageRank",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     val lines = ctx.textFile(args(1), 1)
     val links = lines.map{ s =>
       val parts = s.split("\\s+")
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
index a689e5a360b6bf0eae164ebfb63fe37ea83726e2..e5a09ecec006f2727b1611505547fd5665f26c30 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
@@ -29,7 +29,7 @@ object SparkPi {
       System.exit(1)
     }
     val spark = new SparkContext(args(0), "SparkPi",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     val slices = if (args.length > 1) args(1).toInt else 2
     val n = 100000 * slices
     val count = spark.parallelize(1 to n, slices).map { i =>
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
index 8543ce0e3285e3fe8f45121045012bba51712b54..24e8afa26bc5faca1bc8d129759cdf19f613d5a2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
@@ -46,7 +46,7 @@ object SparkTC {
       System.exit(1)
     }
     val spark = new SparkContext(args(0), "SparkTC",
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
     val slices = if (args.length > 1) args(1).toInt else 2
     var tc = spark.parallelize(generateGraph, slices).cache()
 
diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
index 72b5c7b88e19bb8c8e8dd4b08c3d7e32c07b02e5..4c0de469645ab401a45fa2ef4f096da3cee58229 100644
--- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
@@ -36,16 +36,18 @@ object WikipediaPageRank {
       System.err.println("Usage: WikipediaPageRank <inputFile> <threshold> <numPartitions> <host> <usePartitioner>")
       System.exit(-1)
     }
-
-    System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-    System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName)
+    val sparkConf = new SparkConf()
+    sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+    sparkConf.set("spark.kryo.registrator",  classOf[PRKryoRegistrator].getName)
 
     val inputFile = args(0)
     val threshold = args(1).toDouble
     val numPartitions = args(2).toInt
     val host = args(3)
     val usePartitioner = args(4).toBoolean
-    val sc = new SparkContext(host, "WikipediaPageRank")
+
+    sparkConf.setMaster(host).setAppName("WikipediaPageRank")
+    val sc = new SparkContext(sparkConf)
 
     // Parse the Wikipedia page data into a graph
     val input = sc.textFile(inputFile)
diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
index ddf6855325e732529fa13eb1781643dacde7e162..2cf273a702d24be266eb54f3a1ea59c381c39cc5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
@@ -34,15 +34,19 @@ object WikipediaPageRankStandalone {
       System.err.println("Usage: WikipediaPageRankStandalone <inputFile> <threshold> <numIterations> <host> <usePartitioner>")
       System.exit(-1)
     }
+    val sparkConf = new SparkConf()
+    sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer")
 
-    System.setProperty("spark.serializer", "spark.bagel.examples.WPRSerializer")
 
     val inputFile = args(0)
     val threshold = args(1).toDouble
     val numIterations = args(2).toInt
     val host = args(3)
     val usePartitioner = args(4).toBoolean
-    val sc = new SparkContext(host, "WikipediaPageRankStandalone")
+
+    sparkConf.setMaster(host).setAppName("WikipediaPageRankStandalone")
+
+    val sc = new SparkContext(sparkConf)
 
     val input = sc.textFile(inputFile)
     val partitioner = new HashPartitioner(sc.defaultParallelism)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index 50e3f9639cd5b81d5fa20d94ab9f30ca2367102d..4e0058cd707777c32ce3baa4d68e7d6e0d9d1e09 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -26,6 +26,7 @@ import akka.actor.ActorRef
 import akka.actor.Props
 import akka.actor.actorRef2Scala
 
+import org.apache.spark.SparkConf
 import org.apache.spark.streaming.Seconds
 import org.apache.spark.streaming.StreamingContext
 import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions
@@ -116,7 +117,7 @@ object FeederActor {
     val Seq(host, port) = args.toSeq
 
 
-    val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1
+    val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = new SparkConf)._1
     val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor")
 
     println("Feeder started as:" + feeder)
@@ -133,9 +134,9 @@ object FeederActor {
  *   <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
  *
  * To run this example locally, you may run Feeder Actor as
- *    `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.FeederActor 127.0.1.1 9999`
  * and then run the example
- *    `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
  */
 object ActorWordCount {
   def main(args: Array[String]) {
@@ -150,7 +151,7 @@ object ActorWordCount {
 
     // Create the context and set the batch size
     val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     /*
      * Following is the use of actorStream to plug in custom actor as receiver
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
index 9f6e163454a64fe062989c7a5794f9ffd07f165e..ae3709b3d97f5561f857b7d6fab8ef0436888cbc 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
@@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples
 import org.apache.spark.util.IntParam
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming._
+import org.apache.spark.streaming.flume._
 
 /**
  *  Produces a count of events received from Flume.
@@ -48,10 +49,10 @@ object FlumeEventCount {
     val batchInterval = Milliseconds(2000)
     // Create the context and set the batch size
     val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval,
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     // Create a flume stream
-    val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY)
+    val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2)
 
     // Print out the count of events received from this server in each batch
     stream.count().map(cnt => "Received " + cnt + " flume events." ).print()
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
index bc8564b3ba080aa3ed3f367fb1211736d9035411..ea6ea674196a1effb94ab7138998d4076cb41799 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
@@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._
  *   <directory> 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-example spark.streaming.examples.HdfsWordCount local[2] localdir`
+ *    `$ ./bin/run-example org.apache.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 {
@@ -40,7 +40,7 @@ object HdfsWordCount {
 
     // Create the context
     val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     // Create the FileInputDStream on the directory and use the
     // stream to count words in new files created
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index 570ba4c81a1d2c3cc00144727f04bc8a45dc071f..31a94bd224a45f8f68f177777c7d37b9ff423de7 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -24,6 +24,7 @@ import kafka.producer._
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.streaming.util.RawTextHelper._
+import org.apache.spark.streaming.kafka._
 
 /**
  * Consumes messages from one or more topics in Kafka and does wordcount.
@@ -35,7 +36,7 @@ import org.apache.spark.streaming.util.RawTextHelper._
  *   <numThreads> is the number of threads the kafka consumer should use
  *
  * Example:
- *    `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
+ *    `./bin/run-example org.apache.spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
  */
 object KafkaWordCount {
   def main(args: Array[String]) {
@@ -48,11 +49,11 @@ object KafkaWordCount {
     val Array(master, zkQuorum, group, topics, numThreads) = args
 
     val ssc =  new StreamingContext(master, "KafkaWordCount", Seconds(2),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
     ssc.checkpoint("checkpoint")
 
     val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
-    val lines = ssc.kafkaStream(zkQuorum, group, topicpMap).map(_._2)
+    val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2)
     val words = lines.flatMap(_.split(" "))
     val wordCounts = words.map(x => (x, 1l))
       .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
index ff332a02821290f98faebcbf6a06ec6c091bc31b..325290b66f4decbe55ed025a865816d09ecbcc5c 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
@@ -17,11 +17,6 @@
 
 package org.apache.spark.streaming.examples
 
-import org.apache.spark.streaming.{ Seconds, StreamingContext }
-import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.streaming.dstream.MQTTReceiver
-import org.apache.spark.storage.StorageLevel
-
 import org.eclipse.paho.client.mqttv3.MqttClient
 import org.eclipse.paho.client.mqttv3.MqttClientPersistence
 import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence
@@ -29,6 +24,11 @@ import org.eclipse.paho.client.mqttv3.MqttException
 import org.eclipse.paho.client.mqttv3.MqttMessage
 import org.eclipse.paho.client.mqttv3.MqttTopic
 
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.mqtt._
+
 /**
  * A simple Mqtt publisher for demonstration purposes, repeatedly publishes 
  * Space separated String Message "hello mqtt demo for spark streaming"
@@ -79,9 +79,9 @@ object MQTTPublisher {
  *   <MqttbrokerUrl> and <topic> describe where Mqtt publisher is running.
  *
  * To run this example locally, you may run publisher as
- *    `$ ./run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
  * and run the example as
- *    `$ ./run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
  */
 object MQTTWordCount {
 
@@ -96,8 +96,8 @@ object MQTTWordCount {
     val Seq(master, brokerUrl, topic) = args.toSeq
 
     val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), 
-    Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-    val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY)
+    StreamingContext.jarOfClass(this.getClass))
+    val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2)
 
     val words = lines.flatMap(x => x.toString.split(" "))
     val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
index 5ad4875980d1d9872439164b77808c3192b7347a..aba17048252d04fb60e9530cf1cf3b99ef823b55 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.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-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
  */
 object NetworkWordCount {
   def main(args: Array[String]) {
@@ -41,7 +41,7 @@ object NetworkWordCount {
 
     // Create the context with a 1 second batch size
     val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     // Create a NetworkInputDStream on target ip:port and count the
     // words in input stream of \n delimited text (eg. generated by 'nc')
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
index fad512eebad1292b3b74edf81339080a0c81d32d..9d640e716bca978b74c76bb5cab3ffef6ad37ebf 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
@@ -33,7 +33,7 @@ object QueueStream {
     
     // Create the context
     val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     // Create the queue through which RDDs can be pushed to 
     // a QueueInputDStream
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
index 0b45c30d20dc35ec492239c155de742def77860f..c0706d07249824cc740968d141e2d874c9c185c6 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
@@ -49,7 +49,7 @@ object RawNetworkGrep {
 
     // Create the context
     val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     // Warm up the JVMs on master and slave for JIT compilation to kick in
     RawTextHelper.warmUp(ssc.sparkContext)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
index cb30c4edb30ce408faff73dc504c50f3a426f882..002db57d592b2a88f491ca1813d95cd9e1a14ca6 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.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-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
  */
 object StatefulNetworkWordCount {
   def main(args: Array[String]) {
@@ -49,7 +49,7 @@ object StatefulNetworkWordCount {
 
     // Create the context with a 1 second batch size
     val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
     ssc.checkpoint(".")
 
     // Create a NetworkInputDStream on target ip:port and count the
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
index 35b6329ab3152caca3dd7564a88448cf2e9d284f..3ccdc908e23c43d0c3424a5b6719950e7c658702 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -23,6 +23,8 @@ import com.twitter.algebird._
 import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.SparkContext._
 
+import org.apache.spark.streaming.twitter._
+
 /**
  * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute
  * windowed and global Top-K estimates of user IDs occurring in a Twitter stream.
@@ -33,7 +35,7 @@ import org.apache.spark.SparkContext._
  * <p>
  * <p>
  *   <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/">
- *   This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure
+ *   This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a data structure
  *   for approximate frequency estimation in data streams (e.g. Top-K elements, frequency of any given element, etc),
  *   that uses space sub-linear in the number of elements in the stream. Once elements are added to the CMS, the
  *   estimated count of an element can be computed, as well as "heavy-hitters" that occur more than a threshold
@@ -60,8 +62,8 @@ object TwitterAlgebirdCMS {
     val (master, filters) = (args.head, args.tail)
 
     val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-    val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER)
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
+    val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2)
 
     val users = stream.map(status => status.getUser.getId)
 
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
index 8bfde2a8297c1a133c80d3be08f8ad6ae0b145f3..c7e83e76b00570e721ee1fe965d119178b14528d 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -21,7 +21,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext}
 import org.apache.spark.storage.StorageLevel
 import com.twitter.algebird.HyperLogLog._
 import com.twitter.algebird.HyperLogLogMonoid
-import org.apache.spark.streaming.dstream.TwitterInputDStream
+import org.apache.spark.streaming.twitter._
 
 /**
  * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute
@@ -49,8 +49,8 @@ object TwitterAlgebirdHLL {
     val (master, filters) = (args.head, args.tail)
 
     val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-    val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER)
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
+    val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER)
 
     val users = stream.map(status => status.getUser.getId)
 
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
index 27aa6b14bf221ab08752ce721a2548e85e67f98b..e2b0418d55d2b14b08d50aa1500a6806306a7f5c 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
@@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples
 import org.apache.spark.streaming.{Seconds, StreamingContext}
 import StreamingContext._
 import org.apache.spark.SparkContext._
+import org.apache.spark.streaming.twitter._
 
 /**
  * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter
@@ -38,8 +39,8 @@ object TwitterPopularTags {
     val (master, filters) = (args.head, args.tail)
 
     val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-    val stream = ssc.twitterStream(None, filters)
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
+    val stream = TwitterUtils.createStream(ssc, None, filters)
 
     val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
 
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index e83ce78aa5f1b6e602f7891727ab811f21c4a13c..03902ec353babfbc6d2991d6af01d8e6ed19d09e 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -20,11 +20,13 @@ package org.apache.spark.streaming.examples
 import akka.actor.ActorSystem
 import akka.actor.actorRef2Scala
 import akka.zeromq._
-import org.apache.spark.streaming.{ Seconds, StreamingContext }
-import org.apache.spark.streaming.StreamingContext._
 import akka.zeromq.Subscribe
 import akka.util.ByteString
 
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.zeromq._
+
 /**
  * A simple publisher for demonstration purposes, repeatedly publishes random Messages
  * every one second.
@@ -62,9 +64,9 @@ object SimpleZeroMQPublisher {
  *   <zeroMQurl> and <topic> describe where zeroMq publisher is running.
  *
  * To run this example locally, you may run publisher as
- *    `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
  * and run the example as
- *    `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
  */
 object ZeroMQWordCount {
   def main(args: Array[String]) {
@@ -78,16 +80,15 @@ object ZeroMQWordCount {
 
     // Create the context and set the batch size
     val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator
 
     //For this stream, a zeroMQ publisher should be running.
-    val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator)
+    val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _)
     val words = lines.flatMap(_.split(" "))
     val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
     wordCounts.print()
     ssc.start()
   }
-
 }
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
index de70c50473558a5cf07df0eceacbef6f55fb785b..4fe57de4a4058fe3ebf13332c5fe10be491bf8d9 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -39,8 +39,8 @@ object PageView extends Serializable {
 /** Generates streaming events to simulate page views on a website.
   *
   * This should be used in tandem with PageViewStream.scala. Example:
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
   *
   * When running this, you may want to set the root logging level to ERROR in
   * conf/log4j.properties to reduce the verbosity of the output.
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
index 8282cc9269c13fc287e0e7dc38c0e921139efce5..807af199f4fd0813b2961784f1dca9ed88d9a69a 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._
   * operators available in Spark streaming.
   *
   * This should be used in tandem with PageViewStream.scala. Example:
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
   */
 object PageViewStream {
   def main(args: Array[String]) {
@@ -42,7 +42,7 @@ object PageViewStream {
 
     // Create the context
     val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1),
-      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+      System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     // Create a NetworkInputDStream on target host:port and convert each line to a PageView
     val pageViews = ssc.socketTextStream(host, port)
diff --git a/external/flume/pom.xml b/external/flume/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..443910a03a94ece13a26ebe9653d666339c48e04
--- /dev/null
+++ b/external/flume/pom.xml
@@ -0,0 +1,93 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-flume_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External Flume</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-sdk</artifactId>
+      <version>1.2.0</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.xerial.snappy</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
similarity index 98%
rename from streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
rename to external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
index 60d79175f1e8a32ca64806342761e38018fbd57c..ce3ef47cfe4bcb5137c367c430b45f82d69542e5 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.flume
 
 import java.net.InetSocketAddress
 import java.io.{ObjectInput, ObjectOutput, Externalizable}
@@ -30,9 +30,10 @@ import org.apache.flume.source.avro.Status
 import org.apache.avro.ipc.specific.SpecificResponder
 import org.apache.avro.ipc.NettyServer
 
-import org.apache.spark.streaming.StreamingContext
 import org.apache.spark.util.Utils
 import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
 
 private[streaming]
 class FlumeInputDStream[T: ClassTag](
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
new file mode 100644
index 0000000000000000000000000000000000000000..834b775d4fd2b09ea1c2a24dce48af963c57871b
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.flume
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+
+object FlumeUtils {
+  /**
+   * Create a input stream from a Flume source.
+   * @param ssc      StreamingContext object
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def createStream (
+      ssc: StreamingContext,
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[SparkFlumeEvent] = {
+    val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      hostname: String,
+      port: Int
+    ): JavaDStream[SparkFlumeEvent] = {
+    createStream(jssc.ssc, hostname, port)
+  }
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel
+    ): JavaDStream[SparkFlumeEvent] = {
+    createStream(jssc.ssc, hostname, port, storageLevel)
+  }
+}
diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
new file mode 100644
index 0000000000000000000000000000000000000000..733389b98d22d60e5d60ea56d3eaa1aa8ffc407f
--- /dev/null
+++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.flume;
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+import org.junit.Test;
+
+public class JavaFlumeStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testFlumeStream() {
+    // tests the API, does not actually test data receiving
+    JavaDStream<SparkFlumeEvent> test1 = FlumeUtils.createStream(ssc, "localhost", 12345);
+    JavaDStream<SparkFlumeEvent> test2 = FlumeUtils.createStream(ssc, "localhost", 12345,
+      StorageLevel.MEMORY_AND_DISK_SER_2());
+  }
+}
diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000000000000000000000000000000..063529a9cbc67fe9cbc58c63cad51c43be18ad8a
--- /dev/null
+++ b/external/flume/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2e8e9fac455535e32f4d770de0534334d3d0d1d4
--- /dev/null
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
@@ -0,0 +1,86 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.flume
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
+
+import java.net.InetSocketAddress
+import java.nio.ByteBuffer
+import java.nio.charset.Charset
+
+import org.apache.avro.ipc.NettyTransceiver
+import org.apache.avro.ipc.specific.SpecificRequestor
+import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol}
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase}
+import org.apache.spark.streaming.util.ManualClock
+
+class FlumeStreamSuite extends TestSuiteBase {
+
+  val testPort = 9999
+
+  test("flume input stream") {
+    // Set up the streaming context and input streams
+    val ssc = new StreamingContext(conf, batchDuration)
+    val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+    val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
+      with SynchronizedBuffer[Seq[SparkFlumeEvent]]
+    val outputStream = new TestOutputStream(flumeStream, outputBuffer)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    val input = Seq(1, 2, 3, 4, 5)
+    Thread.sleep(1000)
+    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
+    val client = SpecificRequestor.getClient(
+      classOf[AvroSourceProtocol], transceiver)
+
+    for (i <- 0 until input.size) {
+      val event = new AvroFlumeEvent
+      event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
+      event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
+      client.append(event)
+      Thread.sleep(500)
+      clock.addToTime(batchDuration.milliseconds)
+    }
+
+    val startTime = System.currentTimeMillis()
+    while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+      logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
+      Thread.sleep(100)
+    }
+    Thread.sleep(1000)
+    val timeTaken = System.currentTimeMillis() - startTime
+    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+    logInfo("Stopping context")
+    ssc.stop()
+
+    val decoder = Charset.forName("UTF-8").newDecoder()
+
+    assert(outputBuffer.size === input.length)
+    for (i <- 0 until outputBuffer.size) {
+      assert(outputBuffer(i).size === 1)
+      val str = decoder.decode(outputBuffer(i).head.event.getBody)
+      assert(str.toString === input(i).toString)
+      assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
+    }
+  }
+}
diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..f782e0e126d4528e67305955bdd7fe39a397e188
--- /dev/null
+++ b/external/kafka/pom.xml
@@ -0,0 +1,97 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-kafka_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External Kafka</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.sksamuel.kafka</groupId>
+      <artifactId>kafka_${scala.binary.version}</artifactId>
+      <version>0.8.0-beta1</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.sun.jmx</groupId>
+          <artifactId>jmxri</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jdmk</groupId>
+          <artifactId>jmxtools</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.sf.jopt-simple</groupId>
+          <artifactId>jopt-simple</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
similarity index 98%
rename from streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
rename to external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
index 526f5564c733d48161bf49563a6f920e4e6a2442..a2cd49c573fb539e281db3d532773e0cb3cbb103 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.kafka
 
-import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.StreamingContext
+import scala.collection.Map
+import scala.reflect.ClassTag
 
 import java.util.Properties
 import java.util.concurrent.Executors
@@ -30,8 +29,10 @@ import kafka.utils.VerifiableProperties
 import kafka.utils.ZKStringSerializer
 import org.I0Itec.zkclient._
 
-import scala.collection.Map
-import scala.reflect.ClassTag
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
 
 /**
  * Input stream that pulls messages from a Kafka Broker.
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c2d851f94311d48fe7ce19ca6a1d14c49d5c48e4
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
@@ -0,0 +1,153 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import java.lang.{Integer => JInt}
+import java.util.{Map => JMap}
+
+import kafka.serializer.{Decoder, StringDecoder}
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream}
+
+
+object KafkaUtils {
+  /**
+   * Create an input stream that pulls messages from a Kafka Broker.
+   * @param ssc       StreamingContext object
+   * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..)
+   * @param groupId   The group id for this consumer
+   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                  in its own thread
+   * @param storageLevel  Storage level to use for storing the received objects
+   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+   */
+  def createStream(
+      ssc: StreamingContext,
+      zkQuorum: String,
+      groupId: String,
+      topics: Map[String, Int],
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[(String, String)] = {
+    val kafkaParams = Map[String, String](
+      "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
+      "zookeeper.connection.timeout.ms" -> "10000")
+    createStream[String, String, StringDecoder, StringDecoder](
+      ssc, kafkaParams, topics, storageLevel)
+  }
+
+  /**
+   * Create an input stream that pulls messages from a Kafka Broker.
+   * @param ssc         StreamingContext object
+   * @param kafkaParams Map of kafka configuration parameters,
+   *                    see http://kafka.apache.org/08/configuration.html
+   * @param topics      Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                    in its own thread.
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest](
+      ssc: StreamingContext,
+      kafkaParams: Map[String, String],
+      topics: Map[String, Int],
+      storageLevel: StorageLevel
+    ): DStream[(K, V)] = {
+    val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param jssc      JavaStreamingContext object
+   * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..)
+   * @param groupId   The group id for this consumer
+   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                  in its own thread
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt]
+    ): JavaPairDStream[String, String] = {
+    implicit val cmt: ClassTag[String] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param jssc      JavaStreamingContext object
+   * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..).
+   * @param groupId   The group id for this consumer.
+   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                  in its own thread.
+   * @param storageLevel RDD storage level.
+   *
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[String, String] = {
+    implicit val cmt: ClassTag[String] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param jssc      JavaStreamingContext object
+   * @param keyTypeClass Key type of RDD
+   * @param valueTypeClass value type of RDD
+   * @param keyDecoderClass Type of kafka key decoder
+   * @param valueDecoderClass Type of kafka value decoder
+   * @param kafkaParams Map of kafka configuration parameters,
+   *                    see http://kafka.apache.org/08/configuration.html
+   * @param topics  Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                in its own thread
+   * @param storageLevel RDD storage level. Defaults to MEMORY_AND_DISK_2.
+   */
+  def createStream[K, V, U <: Decoder[_], T <: Decoder[_]](
+      jssc: JavaStreamingContext,
+      keyTypeClass: Class[K],
+      valueTypeClass: Class[V],
+      keyDecoderClass: Class[U],
+      valueDecoderClass: Class[T],
+      kafkaParams: JMap[String, String],
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[K, V] = {
+    implicit val keyCmt: ClassTag[K] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+    implicit val valueCmt: ClassTag[V] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
+
+    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
+    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
+
+    createStream[K, V, U, T](
+      jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+}
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
new file mode 100644
index 0000000000000000000000000000000000000000..7b4999447ee69539c76a60244c668e8a5e6d586c
--- /dev/null
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.kafka;
+
+import java.util.HashMap;
+import org.junit.Test;
+import com.google.common.collect.Maps;
+import kafka.serializer.StringDecoder;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+
+public class JavaKafkaStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testKafkaStream() {
+    HashMap<String, Integer> topics = Maps.newHashMap();
+
+    // tests the API, does not actually test data receiving
+    JavaPairDStream<String, String> test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics);
+    JavaPairDStream<String, String> test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics,
+      StorageLevel.MEMORY_AND_DISK_SER_2());
+
+    HashMap<String, String> kafkaParams = Maps.newHashMap();
+    kafkaParams.put("zookeeper.connect", "localhost:12345");
+    kafkaParams.put("group.id","consumer-group");
+    JavaPairDStream<String, String> test3 = KafkaUtils.createStream(ssc,
+      String.class, String.class, StringDecoder.class, StringDecoder.class,
+      kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2());
+  }
+}
diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000000000000000000000000000000..063529a9cbc67fe9cbc58c63cad51c43be18ad8a
--- /dev/null
+++ b/external/kafka/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9c81f23c191180c3f11262965c4eea1d21ae9747
--- /dev/null
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import kafka.serializer.StringDecoder
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+
+class KafkaStreamSuite extends TestSuiteBase {
+
+  test("kafka input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val topics = Map("my-topic" -> 1)
+
+    // tests the API, does not actually test data receiving
+    val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics)
+    val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
+    val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
+      ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+
+    // TODO: Actually test receiving data
+  }
+}
diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..31b4fa87de77213305f65e6520b1bbc021973e11
--- /dev/null
+++ b/external/mqtt/pom.xml
@@ -0,0 +1,108 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-mqtt_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External MQTT</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <repositories>
+    <repository>
+      <id>mqtt-repo</id>
+      <name>MQTT Repository</name>
+      <url>https://repo.eclipse.org/content/repositories/paho-releases</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.paho</groupId>
+      <artifactId>mqtt-client</artifactId>
+       <version>0.4.0</version>
+    </dependency>
+    <dependency>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
+      <version>${akka.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
similarity index 90%
rename from streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
rename to external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
index ef4a7375685d36ea3205dd73f07f46730bb79842..c8987a3ee06bc74921fdb26dfe30ece76185963b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.mqtt
 
-import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{ Time, DStreamCheckpointData, StreamingContext }
+import scala.collection.Map
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
 
 import java.util.Properties
 import java.util.concurrent.Executors
@@ -34,10 +35,10 @@ import org.eclipse.paho.client.mqttv3.MqttException
 import org.eclipse.paho.client.mqttv3.MqttMessage
 import org.eclipse.paho.client.mqttv3.MqttTopic
 
-import scala.collection.Map
-import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
-import scala.reflect.ClassTag
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
 
 /**
  * Input stream that subscribe messages from a Mqtt Broker.
@@ -49,15 +50,14 @@ import scala.reflect.ClassTag
 
 private[streaming] 
 class MQTTInputDStream[T: ClassTag](
-  @transient ssc_ : StreamingContext,
-  brokerUrl: String,
-  topic: String,
-  storageLevel: StorageLevel
+    @transient ssc_ : StreamingContext,
+    brokerUrl: String,
+    topic: String,
+    storageLevel: StorageLevel
   ) extends NetworkInputDStream[T](ssc_) with Logging {
   
   def getReceiver(): NetworkReceiver[T] = {
-    new MQTTReceiver(brokerUrl, topic, storageLevel)
-      .asInstanceOf[NetworkReceiver[T]]
+    new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]]
   }
 }
 
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
new file mode 100644
index 0000000000000000000000000000000000000000..0e6c25dbee8fbe5e63f0648c890143e9a4ae4509
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+import scala.reflect.ClassTag
+
+object MQTTUtils {
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param ssc           StreamingContext object
+   * @param brokerUrl     Url of remote MQTT publisher
+   * @param topic         Topic name to subscribe to
+   * @param storageLevel  RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+   */
+  def createStream(
+      ssc: StreamingContext,
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[String] = {
+    val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param jssc      JavaStreamingContext object
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic     Topic name to subscribe to
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      brokerUrl: String,
+      topic: String
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, brokerUrl, topic)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param jssc      JavaStreamingContext object
+   * @param brokerUrl     Url of remote MQTT publisher
+   * @param topic         Topic name to subscribe to
+   * @param storageLevel  RDD storage level.
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, brokerUrl, topic, storageLevel)
+  }
+}
diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
new file mode 100644
index 0000000000000000000000000000000000000000..44743aaecf986e902f64b2ce3f7ffbe6f0f5513d
--- /dev/null
+++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.mqtt;
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.junit.Test;
+
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+
+public class JavaMQTTStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testMQTTStream() {
+    String brokerUrl = "abc";
+    String topic = "def";
+
+    // tests the API, does not actually test data receiving
+    JavaDStream<String> test1 = MQTTUtils.createStream(ssc, brokerUrl, topic);
+    JavaDStream<String> test2 = MQTTUtils.createStream(ssc, brokerUrl, topic,
+      StorageLevel.MEMORY_AND_DISK_SER_2());
+  }
+}
diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000000000000000000000000000000..063529a9cbc67fe9cbc58c63cad51c43be18ad8a
--- /dev/null
+++ b/external/mqtt/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..fcc159e85a85bf918f587dc0c626bca6bebe8231
--- /dev/null
+++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+
+class MQTTStreamSuite extends TestSuiteBase {
+
+  test("MQTT input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val brokerUrl = "abc"
+    val topic = "def"
+
+    // tests the API, does not actually test data receiving
+    val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic)
+    val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2)
+
+    // TODO: Actually test receiving data
+  }
+}
diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..216e6c1d8ff449b236f64f1c7499dae024f907d8
--- /dev/null
+++ b/external/twitter/pom.xml
@@ -0,0 +1,89 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-twitter_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External Twitter</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.twitter4j</groupId>
+      <artifactId>twitter4j-stream</artifactId>
+      <version>3.0.3</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
similarity index 96%
rename from streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
rename to external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
index 387e15b0e6d809f94fcbe94f3ac00ddd8026ebe8..5cc721d7f90ea9c7478a162c0d65f48c04363179 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
@@ -15,18 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.twitter
 
-import org.apache.spark._
-import org.apache.spark.streaming._
-import storage.StorageLevel
+import java.util.prefs.Preferences
 import twitter4j._
 import twitter4j.auth.Authorization
-import java.util.prefs.Preferences
 import twitter4j.conf.ConfigurationBuilder
 import twitter4j.conf.PropertyConfiguration
 import twitter4j.auth.OAuthAuthorization
 import twitter4j.auth.AccessToken
+import org.apache.spark._
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.storage.StorageLevel
 
 /* A stream of Twitter statuses, potentially filtered by one or more keywords.
 *
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5e506ffabcfc4a796bc0cf1a10247ad6023d7757
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import twitter4j.Status
+import twitter4j.auth.Authorization
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+
+object TwitterUtils {
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param ssc         StreamingContext object
+   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
+   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
+   *        twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   *        twitter4j.oauth.accessTokenSecret
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream(
+      ssc: StreamingContext,
+      twitterAuth: Option[Authorization],
+      filters: Seq[String] = Nil,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[Status] = {
+    val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param jssc   JavaStreamingContext object
+   */
+  def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = {
+    createStream(jssc.ssc, None)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param jssc    JavaStreamingContext object
+   * @param filters Set of filter strings to get only those tweets that match them
+   */
+  def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = {
+    createStream(jssc.ssc, None, filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param jssc         JavaStreamingContext object
+   * @param filters      Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      filters: Array[String],
+      storageLevel: StorageLevel
+    ): JavaDStream[Status] = {
+    createStream(jssc.ssc, None, filters, storageLevel)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param jssc        JavaStreamingContext object
+   * @param twitterAuth Twitter4J Authorization
+   */
+  def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = {
+    createStream(jssc.ssc, Some(twitterAuth))
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param jssc        JavaStreamingContext object
+   * @param twitterAuth Twitter4J Authorization
+   * @param filters     Set of filter strings to get only those tweets that match them
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      twitterAuth: Authorization,
+      filters: Array[String]
+    ): JavaDStream[Status] = {
+    createStream(jssc.ssc, Some(twitterAuth), filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param jssc         JavaStreamingContext object
+   * @param twitterAuth  Twitter4J Authorization object
+   * @param filters      Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      twitterAuth: Authorization,
+      filters: Array[String],
+      storageLevel: StorageLevel
+    ): JavaDStream[Status] = {
+    createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel)
+  }
+}
diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
new file mode 100644
index 0000000000000000000000000000000000000000..e46b4e5c7531dac29123589ee6a5a22e0dd6f560
--- /dev/null
+++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.twitter;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+import twitter4j.Status;
+import twitter4j.auth.Authorization;
+import twitter4j.auth.NullAuthorization;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+public class JavaTwitterStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testTwitterStream() {
+    String[] filters = (String[])Arrays.<String>asList("filter1", "filter2").toArray();
+    Authorization auth = NullAuthorization.getInstance();
+
+    // tests the API, does not actually test data receiving
+    JavaDStream<Status> test1 = TwitterUtils.createStream(ssc);
+    JavaDStream<Status> test2 = TwitterUtils.createStream(ssc, filters);
+    JavaDStream<Status> test3 = TwitterUtils.createStream(
+      ssc, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<Status> test4 = TwitterUtils.createStream(ssc, auth);
+    JavaDStream<Status> test5 = TwitterUtils.createStream(ssc, auth, filters);
+    JavaDStream<Status> test6 = TwitterUtils.createStream(ssc,
+      auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+  }
+}
diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000000000000000000000000000000..063529a9cbc67fe9cbc58c63cad51c43be18ad8a
--- /dev/null
+++ b/external/twitter/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a0a8fe617b134c7c39ae8df77690f12a1fa42797
--- /dev/null
+++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+import twitter4j.auth.{NullAuthorization, Authorization}
+
+class TwitterStreamSuite extends TestSuiteBase {
+
+  test("kafka input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val filters = Seq("filter1", "filter2")
+    val authorization: Authorization = NullAuthorization.getInstance()
+
+    // tests the API, does not actually test data receiving
+    val test1 = TwitterUtils.createStream(ssc, None)
+    val test2 = TwitterUtils.createStream(ssc, None, filters)
+    val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test4 = TwitterUtils.createStream(ssc, Some(authorization))
+    val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters)
+    val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters,
+      StorageLevel.MEMORY_AND_DISK_SER_2)
+
+    // Note that actually testing the data receiving is hard as authentication keys are
+    // necessary for accessing Twitter live stream
+  }
+}
diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..c240d595742cffd5faf76e1ce782ae7fbb48bc21
--- /dev/null
+++ b/external/zeromq/pom.xml
@@ -0,0 +1,89 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-zeromq_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External ZeroMQ</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
+      <version>${akka.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
similarity index 95%
rename from streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
rename to external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
index f164d516b0b3f8da846ac243f17cf4efdf02a30d..769761e3b860c54d7f94eb83c82f6e8f238216bf 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.streaming.receivers
+package org.apache.spark.streaming.zeromq
 
 import scala.reflect.ClassTag
 
@@ -24,6 +24,7 @@ import akka.util.ByteString
 import akka.zeromq._
 
 import org.apache.spark.Logging
+import org.apache.spark.streaming.receivers._
 
 /**
  * A receiver to subscribe to ZeroMQ stream.
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala
new file mode 100644
index 0000000000000000000000000000000000000000..546d9df3b5df9b0a8232a74356dba607151c9ad8
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+import akka.actor.{Props, SupervisorStrategy}
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+
+object ZeroMQUtils {
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param ssc            StreamingContext object
+   * @param publisherUrl   Url of remote zeromq publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
+   *                       and each frame has sequence of byte thus it needs the converter
+   *                       (which might be deserializer of bytes) to translate from sequence
+   *                       of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel   RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+   */
+  def createStream[T: ClassTag](
+      ssc: StreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2,
+      supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
+    ): DStream[T] = {
+    ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
+      "ZeroMQReceiver", storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param jssc           JavaStreamingContext object
+   * @param publisherUrl   Url of remote ZeroMQ publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def createStream[T](
+      jssc: JavaStreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel,
+      supervisorStrategy: SupervisorStrategy
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param jssc           JavaStreamingContext object
+   * @param publisherUrl   Url of remote zeromq publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel   RDD storage level.
+   */
+  def createStream[T](
+      jssc: JavaStreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param jssc           JavaStreamingContext object
+   * @param publisherUrl   Url of remote zeromq publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   */
+  def createStream[T](
+      jssc: JavaStreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    createStream[T](jssc.ssc, publisherUrl, subscribe, fn)
+  }
+}
diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
new file mode 100644
index 0000000000000000000000000000000000000000..d2361e14b898a629775d304c1ce3c6e687981cd8
--- /dev/null
+++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.zeromq;
+
+import org.junit.Test;
+import akka.actor.SupervisorStrategy;
+import akka.util.ByteString;
+import akka.zeromq.Subscribe;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext {
+
+  @Test // tests the API, does not actually test data receiving
+  public void testZeroMQStream() {
+    String publishUrl = "abc";
+    Subscribe subscribe = new Subscribe((ByteString)null);
+    Function<byte[][], Iterable<String>> bytesToObjects = new Function<byte[][], Iterable<String>>() {
+      @Override
+      public Iterable<String> call(byte[][] bytes) throws Exception {
+        return null;
+      }
+    };
+
+    JavaDStream<String> test1 = ZeroMQUtils.<String>createStream(
+      ssc, publishUrl, subscribe, bytesToObjects);
+    JavaDStream<String> test2 = ZeroMQUtils.<String>createStream(
+      ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<String> test3 = ZeroMQUtils.<String>createStream(
+      ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(),
+      SupervisorStrategy.defaultStrategy());
+  }
+}
diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000000000000000000000000000000..063529a9cbc67fe9cbc58c63cad51c43be18ad8a
--- /dev/null
+++ b/external/zeromq/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..4193b8a02f14abefb1b17634e2f03709128588e2
--- /dev/null
+++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import akka.actor.SupervisorStrategy
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+
+class ZeroMQStreamSuite extends TestSuiteBase {
+
+  test("zeromq input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val publishUrl = "abc"
+    val subscribe = new Subscribe(null.asInstanceOf[ByteString])
+    val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]]
+
+    // tests the API, does not actually test data receiving
+    val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects)
+    val test2 = ZeroMQUtils.createStream(
+      ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects,
+      StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy)
+
+    // TODO: Actually test data receiving
+  }
+}
diff --git a/make-distribution.sh b/make-distribution.sh
index 32bbdb90a5bdad25237147e366cf02f1c6bb9613..1a3a5d0209ccf8a295338e26735dfac06b41408c 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -31,10 +31,10 @@
 #
 # Recommended deploy/testing procedure (standalone mode):
 # 1) Rsync / deploy the dist/ dir to one host
-# 2) cd to deploy dir; ./bin/start-master.sh
+# 2) cd to deploy dir; ./sbin/start-master.sh
 # 3) Verify master is up by visiting web page, ie http://master-ip:8080.  Note the spark:// URL.
-# 4) ./bin/start-slave.sh 1 <<spark:// URL>>
-# 5) MASTER="spark://my-master-ip:7077" ./spark-shell
+# 4) ./sbin/start-slave.sh 1 <<spark:// URL>>
+# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell
 #
 
 # Figure out where the Spark framework is installed
@@ -43,7 +43,17 @@ DISTDIR="$FWDIR/dist"
 
 # Get version from SBT
 export TERM=dumb   # Prevents color codes in SBT output
-VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/')
+
+VERSIONSTRING=$FWDIR/sbt/sbt "show version"
+
+if [ $? == -1 ] ;then
+    echo -e "You need sbt installed and available on your path."
+    echo -e "Download sbt from http://www.scala-sbt.org/"
+    exit -1;
+fi
+
+VERSION=$(echo "${VERSIONSTRING}" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/')
+echo "Version is ${VERSION}"
 
 # Initialize defaults
 SPARK_HADOOP_VERSION=1.0.4
@@ -83,7 +93,9 @@ fi
 # Build fat JAR
 export SPARK_HADOOP_VERSION
 export SPARK_YARN
-"$FWDIR/sbt/sbt" "assembly/assembly"
+cd $FWDIR
+
+"sbt/sbt" "assembly/assembly"
 
 # Make directories
 rm -rf "$DISTDIR"
@@ -98,10 +110,7 @@ mkdir "$DISTDIR"/conf
 cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
 cp -r "$FWDIR/bin" "$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"
+cp -r "$FWDIR/sbin" "$DISTDIR"
 
 
 if [ "$MAKE_TGZ" == "true" ]; then
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2d8623392eb4e42cfaaf8fbfd329b1646b8c3322
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -0,0 +1,262 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.mllib.api.python
+import org.apache.spark.api.java.JavaRDD
+import org.apache.spark.mllib.regression._
+import org.apache.spark.mllib.classification._
+import org.apache.spark.mllib.clustering._
+import org.apache.spark.mllib.recommendation._
+import org.apache.spark.rdd.RDD
+import java.nio.ByteBuffer
+import java.nio.ByteOrder
+import java.nio.DoubleBuffer
+
+/**
+ * The Java stubs necessary for the Python mllib bindings.
+ */
+class PythonMLLibAPI extends Serializable {
+  private def deserializeDoubleVector(bytes: Array[Byte]): Array[Double] = {
+    val packetLength = bytes.length
+    if (packetLength < 16) {
+      throw new IllegalArgumentException("Byte array too short.")
+    }
+    val bb = ByteBuffer.wrap(bytes)
+    bb.order(ByteOrder.nativeOrder())
+    val magic = bb.getLong()
+    if (magic != 1) {
+      throw new IllegalArgumentException("Magic " + magic + " is wrong.")
+    }
+    val length = bb.getLong()
+    if (packetLength != 16 + 8 * length) {
+      throw new IllegalArgumentException("Length " + length + " is wrong.")
+    }
+    val db = bb.asDoubleBuffer()
+    val ans = new Array[Double](length.toInt)
+    db.get(ans)
+    return ans
+  }
+
+  private def serializeDoubleVector(doubles: Array[Double]): Array[Byte] = {
+    val len = doubles.length
+    val bytes = new Array[Byte](16 + 8 * len)
+    val bb = ByteBuffer.wrap(bytes)
+    bb.order(ByteOrder.nativeOrder())
+    bb.putLong(1)
+    bb.putLong(len)
+    val db = bb.asDoubleBuffer()
+    db.put(doubles)
+    return bytes
+  }
+
+  private def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = {
+    val packetLength = bytes.length
+    if (packetLength < 24) {
+      throw new IllegalArgumentException("Byte array too short.")
+    }
+    val bb = ByteBuffer.wrap(bytes)
+    bb.order(ByteOrder.nativeOrder())
+    val magic = bb.getLong()
+    if (magic != 2) {
+      throw new IllegalArgumentException("Magic " + magic + " is wrong.")
+    }
+    val rows = bb.getLong()
+    val cols = bb.getLong()
+    if (packetLength != 24 + 8 * rows * cols) {
+      throw new IllegalArgumentException("Size " + rows + "x" + cols + " is wrong.")
+    }
+    val db = bb.asDoubleBuffer()
+    val ans = new Array[Array[Double]](rows.toInt)
+    var i = 0
+    for (i <- 0 until rows.toInt) {
+      ans(i) = new Array[Double](cols.toInt)
+      db.get(ans(i))
+    }
+    return ans
+  }
+
+  private def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = {
+    val rows = doubles.length
+    var cols = 0
+    if (rows > 0) {
+      cols = doubles(0).length
+    }
+    val bytes = new Array[Byte](24 + 8 * rows * cols)
+    val bb = ByteBuffer.wrap(bytes)
+    bb.order(ByteOrder.nativeOrder())
+    bb.putLong(2)
+    bb.putLong(rows)
+    bb.putLong(cols)
+    val db = bb.asDoubleBuffer()
+    var i = 0
+    for (i <- 0 until rows) {
+      db.put(doubles(i))
+    }
+    return bytes
+  }
+
+  private def trainRegressionModel(trainFunc: (RDD[LabeledPoint], Array[Double]) => GeneralizedLinearModel,
+      dataBytesJRDD: JavaRDD[Array[Byte]], initialWeightsBA: Array[Byte]):
+      java.util.LinkedList[java.lang.Object] = {
+    val data = dataBytesJRDD.rdd.map(xBytes => {
+        val x = deserializeDoubleVector(xBytes)
+        LabeledPoint(x(0), x.slice(1, x.length))
+    })
+    val initialWeights = deserializeDoubleVector(initialWeightsBA)
+    val model = trainFunc(data, initialWeights)
+    val ret = new java.util.LinkedList[java.lang.Object]()
+    ret.add(serializeDoubleVector(model.weights))
+    ret.add(model.intercept: java.lang.Double)
+    return ret
+  }
+
+  /**
+   * Java stub for Python mllib LinearRegressionWithSGD.train()
+   */
+  def trainLinearRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]],
+      numIterations: Int, stepSize: Double, miniBatchFraction: Double,
+      initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+    return trainRegressionModel((data, initialWeights) =>
+        LinearRegressionWithSGD.train(data, numIterations, stepSize,
+                                      miniBatchFraction, initialWeights),
+        dataBytesJRDD, initialWeightsBA)
+  }
+
+  /**
+   * Java stub for Python mllib LassoWithSGD.train()
+   */
+  def trainLassoModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
+      stepSize: Double, regParam: Double, miniBatchFraction: Double,
+      initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+    return trainRegressionModel((data, initialWeights) =>
+        LassoWithSGD.train(data, numIterations, stepSize, regParam,
+                           miniBatchFraction, initialWeights),
+        dataBytesJRDD, initialWeightsBA)
+  }
+
+  /**
+   * Java stub for Python mllib RidgeRegressionWithSGD.train()
+   */
+  def trainRidgeModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
+      stepSize: Double, regParam: Double, miniBatchFraction: Double,
+      initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+    return trainRegressionModel((data, initialWeights) =>
+        RidgeRegressionWithSGD.train(data, numIterations, stepSize, regParam,
+                                     miniBatchFraction, initialWeights),
+        dataBytesJRDD, initialWeightsBA)
+  }
+
+  /**
+   * Java stub for Python mllib SVMWithSGD.train()
+   */
+  def trainSVMModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
+      stepSize: Double, regParam: Double, miniBatchFraction: Double,
+      initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+    return trainRegressionModel((data, initialWeights) =>
+        SVMWithSGD.train(data, numIterations, stepSize, regParam,
+                                     miniBatchFraction, initialWeights),
+        dataBytesJRDD, initialWeightsBA)
+  }
+
+  /**
+   * Java stub for Python mllib LogisticRegressionWithSGD.train()
+   */
+  def trainLogisticRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]],
+      numIterations: Int, stepSize: Double, miniBatchFraction: Double,
+      initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+    return trainRegressionModel((data, initialWeights) =>
+        LogisticRegressionWithSGD.train(data, numIterations, stepSize,
+                                     miniBatchFraction, initialWeights),
+        dataBytesJRDD, initialWeightsBA)
+  }
+
+  /**
+   * Java stub for Python mllib KMeans.train()
+   */
+  def trainKMeansModel(dataBytesJRDD: JavaRDD[Array[Byte]], k: Int,
+      maxIterations: Int, runs: Int, initializationMode: String):
+      java.util.List[java.lang.Object] = {
+    val data = dataBytesJRDD.rdd.map(xBytes => deserializeDoubleVector(xBytes))
+    val model = KMeans.train(data, k, maxIterations, runs, initializationMode)
+    val ret = new java.util.LinkedList[java.lang.Object]()
+    ret.add(serializeDoubleMatrix(model.clusterCenters))
+    return ret
+  }
+
+  /** Unpack a Rating object from an array of bytes */
+  private def unpackRating(ratingBytes: Array[Byte]): Rating = {
+    val bb = ByteBuffer.wrap(ratingBytes)
+    bb.order(ByteOrder.nativeOrder())
+    val user = bb.getInt()
+    val product = bb.getInt()
+    val rating = bb.getDouble()
+    return new Rating(user, product, rating)
+  }
+
+  /** Unpack a tuple of Ints from an array of bytes */
+  private[spark] def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = {
+    val bb = ByteBuffer.wrap(tupleBytes)
+    bb.order(ByteOrder.nativeOrder())
+    val v1 = bb.getInt()
+    val v2 = bb.getInt()
+    (v1, v2)
+  }
+
+  /**
+    * Serialize a Rating object into an array of bytes.
+    * It can be deserialized using RatingDeserializer().
+    *
+    * @param rate
+    * @return
+    */
+  private[spark] def serializeRating(rate: Rating): Array[Byte] = {
+    val len = 3
+    val bytes = new Array[Byte](4 + 8 * len)
+    val bb = ByteBuffer.wrap(bytes)
+    bb.order(ByteOrder.nativeOrder())
+    bb.putInt(len)
+    val db = bb.asDoubleBuffer()
+    db.put(rate.user.toDouble)
+    db.put(rate.product.toDouble)
+    db.put(rate.rating)
+    bytes
+  }
+
+  /**
+   * Java stub for Python mllib ALS.train().  This stub returns a handle
+   * to the Java object instead of the content of the Java object.  Extra care
+   * needs to be taken in the Python code to ensure it gets freed on exit; see
+   * the Py4J documentation.
+   */
+  def trainALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int,
+      iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = {
+    val ratings = ratingsBytesJRDD.rdd.map(unpackRating)
+    return ALS.train(ratings, rank, iterations, lambda, blocks)
+  }
+
+  /**
+   * Java stub for Python mllib ALS.trainImplicit().  This stub returns a
+   * handle to the Java object instead of the content of the Java object.
+   * Extra care needs to be taken in the Python code to ensure it gets freed on
+   * exit; see the Py4J documentation.
+   */
+  def trainImplicitALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int,
+      iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = {
+    val ratings = ratingsBytesJRDD.rdd.map(unpackRating)
+    return ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha)
+  }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
new file mode 100644
index 0000000000000000000000000000000000000000..524300d6aebc59f0b0ac475388fd913ae1503a9d
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
@@ -0,0 +1,119 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.mllib.classification
+
+import scala.collection.mutable
+
+import org.jblas.DoubleMatrix
+
+import org.apache.spark.Logging
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.rdd.RDD
+
+/**
+ * Model for Naive Bayes Classifiers.
+ *
+ * @param pi Log of class priors, whose dimension is C.
+ * @param theta Log of class conditional probabilities, whose dimension is CXD.
+ */
+class NaiveBayesModel(pi: Array[Double], theta: Array[Array[Double]])
+  extends ClassificationModel with Serializable {
+
+  // Create a column vector that can be used for predictions
+  private val _pi = new DoubleMatrix(pi.length, 1, pi: _*)
+  private val _theta = new DoubleMatrix(theta)
+
+  def predict(testData: RDD[Array[Double]]): RDD[Double] = testData.map(predict)
+
+  def predict(testData: Array[Double]): Double = {
+    val dataMatrix = new DoubleMatrix(testData.length, 1, testData: _*)
+    val result = _pi.add(_theta.mmul(dataMatrix))
+    result.argmax()
+  }
+}
+
+/**
+ * Trains a Naive Bayes model given an RDD of `(label, features)` pairs.
+ *
+ * @param lambda The smooth parameter
+ */
+class NaiveBayes private (val lambda: Double = 1.0)
+  extends Serializable with Logging {
+
+  /**
+   * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries.
+   *
+   * @param data RDD of (label, array of features) pairs.
+   */
+  def run(data: RDD[LabeledPoint]) = {
+    // Aggregates all sample points to driver side to get sample count and summed feature vector
+    // for each label.  The shape of `zeroCombiner` & `aggregated` is:
+    //
+    //    label: Int -> (count: Int, featuresSum: DoubleMatrix)
+    val zeroCombiner = mutable.Map.empty[Int, (Int, DoubleMatrix)]
+    val aggregated = data.aggregate(zeroCombiner)({ (combiner, point) =>
+      point match {
+        case LabeledPoint(label, features) =>
+          val (count, featuresSum) = combiner.getOrElse(label.toInt, (0, DoubleMatrix.zeros(1)))
+          val fs = new DoubleMatrix(features.length, 1, features: _*)
+          combiner += label.toInt -> (count + 1, featuresSum.addi(fs))
+      }
+    }, { (lhs, rhs) =>
+      for ((label, (c, fs)) <- rhs) {
+        val (count, featuresSum) = lhs.getOrElse(label, (0, DoubleMatrix.zeros(1)))
+        lhs(label) = (count + c, featuresSum.addi(fs))
+      }
+      lhs
+    })
+
+    // Kinds of label
+    val C = aggregated.size
+    // Total sample count
+    val N = aggregated.values.map(_._1).sum
+
+    val pi = new Array[Double](C)
+    val theta = new Array[Array[Double]](C)
+    val piLogDenom = math.log(N + C * lambda)
+
+    for ((label, (count, fs)) <- aggregated) {
+      val thetaLogDenom = math.log(fs.sum() + fs.length * lambda)
+      pi(label) = math.log(count + lambda) - piLogDenom
+      theta(label) = fs.toArray.map(f => math.log(f + lambda) - thetaLogDenom)
+    }
+
+    new NaiveBayesModel(pi, theta)
+  }
+}
+
+object NaiveBayes {
+  /**
+   * Trains a Naive Bayes model given an RDD of `(label, features)` pairs.
+   *
+   * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of
+   * discrete data.  For example, by converting documents into TF-IDF vectors, it can be used for
+   * document classification.  By making every vector a 0-1 vector. it can also be used as
+   * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]).
+   *
+   * @param input RDD of `(label, array of features)` pairs.  Every vector should be a frequency
+   *              vector or a count vector.
+   * @param lambda The smooth parameter
+   */
+  def train(input: RDD[LabeledPoint], lambda: Double = 1.0): NaiveBayesModel = {
+    new NaiveBayes(lambda).run(input)
+  }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
index 749e7364f4de1afde6371cec3cf5f9515bfd50d1..c590492e7abd659d5af0ca6c494bfd1fe5371cfb 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
@@ -50,8 +50,8 @@ class LogisticGradient extends Gradient {
 
     val gradient = data.mul(gradientMultiplier)
     val loss =
-      if (margin > 0) {
-        math.log(1 + math.exp(0 - margin))
+      if (label > 0) {
+        math.log(1 + math.exp(margin))
       } else {
         math.log(1 + math.exp(margin)) - margin
       }
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index 36853acab514dd631d31f9b881ff57654bd5a0be..8b27ecf82c06d7686873da088bbf3b3de84dffe0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -578,14 +578,13 @@ object ALS {
     val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false
     val alpha = if (args.length >= 8) args(7).toDouble else 1
     val blocks = if (args.length == 9) args(8).toInt else -1
-
-    System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-    System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName)
-    System.setProperty("spark.kryo.referenceTracking", "false")
-    System.setProperty("spark.kryoserializer.buffer.mb", "8")
-    System.setProperty("spark.locality.wait", "10000")
-
     val sc = new SparkContext(master, "ALS")
+    sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+    sc.conf.set("spark.kryo.registrator",  classOf[ALSRegistrator].getName)
+    sc.conf.set("spark.kryo.referenceTracking", "false")
+    sc.conf.set("spark.kryoserializer.buffer.mb", "8")
+    sc.conf.set("spark.locality.wait", "10000")
+
     val ratings = sc.textFile(ratingsFile).map { line =>
       val fields = line.split(',')
       Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
index af43d89c70f05948ae50c539804ef1de4a8fe6f2..443fc5de5bf045f17a13da5c8adba9cd4db404b6 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
@@ -19,8 +19,11 @@ package org.apache.spark.mllib.recommendation
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.api.python.PythonMLLibAPI
 
 import org.jblas._
+import org.apache.spark.api.java.JavaRDD
+
 
 /**
  * Model representing the result of matrix factorization.
@@ -44,6 +47,39 @@ class MatrixFactorizationModel(
     userVector.dot(productVector)
   }
 
-  // TODO: Figure out what good bulk prediction methods would look like.
+  /**
+    * Predict the rating of many users for many products.
+    * The output RDD has an element per each element in the input RDD (including all duplicates)
+    * unless a user or product is missing in the training set.
+    *
+    * @param usersProducts  RDD of (user, product) pairs.
+    * @return RDD of Ratings.
+    */
+  def predict(usersProducts: RDD[(Int, Int)]): RDD[Rating] = {
+    val users = userFeatures.join(usersProducts).map{
+      case (user, (uFeatures, product)) => (product, (user, uFeatures))
+    }
+    users.join(productFeatures).map {
+      case (product, ((user, uFeatures), pFeatures)) =>
+        val userVector = new DoubleMatrix(uFeatures)
+        val productVector = new DoubleMatrix(pFeatures)
+        Rating(user, product, userVector.dot(productVector))
+    }
+  }
+
+  /**
+   * Predict the rating of many users for many products.
+   * This is a Java stub for python predictAll()
+   *
+   * @param usersProductsJRDD A JavaRDD with serialized tuples (user, product)
+   * @return JavaRDD of serialized Rating objects.
+   */
+  def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = {
+    val pythonAPI = new PythonMLLibAPI()
+    val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes))
+    predict(usersProducts).map(rate => pythonAPI.serializeRating(rate))
+  }
+
+  // TODO: Figure out what other good bulk prediction methods would look like.
   // Probably want a way to get the top users for a product or vice-versa.
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b615f76e66cf91cfad2d4042f0f4dc6e20e68b74
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.mllib.classification
+
+import scala.util.Random
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.FunSuite
+
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.SparkContext
+
+object NaiveBayesSuite {
+
+  private def calcLabel(p: Double, pi: Array[Double]): Int = {
+    var sum = 0.0
+    for (j <- 0 until pi.length) {
+      sum += pi(j)
+      if (p < sum) return j
+    }
+    -1
+  }
+
+  // Generate input of the form Y = (theta * x).argmax()
+  def generateNaiveBayesInput(
+      pi: Array[Double],            // 1XC
+      theta: Array[Array[Double]],  // CXD
+      nPoints: Int,
+      seed: Int): Seq[LabeledPoint] = {
+    val D = theta(0).length
+    val rnd = new Random(seed)
+
+    val _pi = pi.map(math.pow(math.E, _))
+    val _theta = theta.map(row => row.map(math.pow(math.E, _)))
+
+    for (i <- 0 until nPoints) yield {
+      val y = calcLabel(rnd.nextDouble(), _pi)
+      val xi = Array.tabulate[Double](D) { j =>
+        if (rnd.nextDouble() < _theta(y)(j)) 1 else 0
+      }
+
+      LabeledPoint(y, xi)
+    }
+  }
+}
+
+class NaiveBayesSuite extends FunSuite with BeforeAndAfterAll {
+  @transient private var sc: SparkContext = _
+
+  override def beforeAll() {
+    sc = new SparkContext("local", "test")
+  }
+
+  override def afterAll() {
+    sc.stop()
+    System.clearProperty("spark.driver.port")
+  }
+
+  def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) {
+    val numOfPredictions = predictions.zip(input).count {
+      case (prediction, expected) =>
+        prediction != expected.label
+    }
+    // At least 80% of the predictions should be on.
+    assert(numOfPredictions < input.length / 5)
+  }
+
+  test("Naive Bayes") {
+    val nPoints = 10000
+
+    val pi = Array(0.5, 0.3, 0.2).map(math.log)
+    val theta = Array(
+      Array(0.91, 0.03, 0.03, 0.03), // label 0
+      Array(0.03, 0.91, 0.03, 0.03), // label 1
+      Array(0.03, 0.03, 0.91, 0.03)  // label 2
+    ).map(_.map(math.log))
+
+    val testData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 42)
+    val testRDD = sc.parallelize(testData, 2)
+    testRDD.cache()
+
+    val model = NaiveBayes.train(testRDD)
+
+    val validationData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 17)
+    val validationRDD = sc.parallelize(validationData, 2)
+
+    // Test prediction on RDD.
+    validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData)
+
+    // Test prediction on Array.
+    validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
+  }
+}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a6028a1e981dd6ff80b0f7cdcac94e5dca26149f
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.mllib.optimization
+
+import scala.util.Random
+import scala.collection.JavaConversions._
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.regression._
+
+object GradientDescentSuite {
+
+  def generateLogisticInputAsList(
+      offset: Double,
+      scale: Double,
+      nPoints: Int,
+      seed: Int): java.util.List[LabeledPoint] = {
+    seqAsJavaList(generateGDInput(offset, scale, nPoints, seed))
+  }
+
+  // Generate input of the form Y = logistic(offset + scale * X)
+  def generateGDInput(
+      offset: Double,
+      scale: Double,
+      nPoints: Int,
+      seed: Int): Seq[LabeledPoint]  = {
+    val rnd = new Random(seed)
+    val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian())
+
+    val unifRand = new scala.util.Random(45)
+    val rLogis = (0 until nPoints).map { i =>
+      val u = unifRand.nextDouble()
+      math.log(u) - math.log(1.0-u)
+    }
+
+    val y: Seq[Int] = (0 until nPoints).map { i =>
+      val yVal = offset + scale * x1(i) + rLogis(i)
+      if (yVal > 0) 1 else 0
+    }
+
+    val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i))))
+    testData
+  }
+}
+
+class GradientDescentSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers {
+  @transient private var sc: SparkContext = _
+
+  override def beforeAll() {
+    sc = new SparkContext("local", "test")
+  }
+
+  override def afterAll() {
+    sc.stop()
+    System.clearProperty("spark.driver.port")
+  }
+
+  test("Assert the loss is decreasing.") {
+    val nPoints = 10000
+    val A = 2.0
+    val B = -1.5
+
+    val initialB = -1.0
+    val initialWeights = Array(initialB)
+
+    val gradient = new LogisticGradient()
+    val updater = new SimpleUpdater()
+    val stepSize = 1.0
+    val numIterations = 10
+    val regParam = 0
+    val miniBatchFrac = 1.0
+
+    // Add a extra variable consisting of all 1.0's for the intercept.
+    val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42)
+    val data = testData.map { case LabeledPoint(label, features) =>
+      label -> Array(1.0, features: _*)
+    }
+
+    val dataRDD = sc.parallelize(data, 2).cache()
+    val initialWeightsWithIntercept = Array(1.0, initialWeights: _*)
+
+    val (_, loss) = GradientDescent.runMiniBatchSGD(
+      dataRDD,
+      gradient,
+      updater,
+      stepSize,
+      numIterations,
+      regParam,
+      miniBatchFrac,
+      initialWeightsWithIntercept)
+
+    assert(loss.last - loss.head < 0, "loss isn't decreasing.")
+
+    val lossDiff = loss.init.zip(loss.tail).map { case (lhs, rhs) => lhs - rhs }
+    assert(lossDiff.count(_ > 0).toDouble / lossDiff.size > 0.8)
+  }
+}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
index fafc5ec5f2245fbfa446e24003224e6d39cdec65..e683a90f57aba9a0229ee4e1b84a255536c5d71c 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
@@ -90,18 +90,34 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
     testALS(50, 100, 1, 15, 0.7, 0.3)
   }
 
+  test("rank-1 matrices bulk") {
+    testALS(50, 100, 1, 15, 0.7, 0.3, false, true)
+  }
+
   test("rank-2 matrices") {
     testALS(100, 200, 2, 15, 0.7, 0.3)
   }
 
+  test("rank-2 matrices bulk") {
+    testALS(100, 200, 2, 15, 0.7, 0.3, false, true)
+  }
+
   test("rank-1 matrices implicit") {
     testALS(80, 160, 1, 15, 0.7, 0.4, true)
   }
 
+  test("rank-1 matrices implicit bulk") {
+    testALS(80, 160, 1, 15, 0.7, 0.4, true, true)
+  }
+
   test("rank-2 matrices implicit") {
     testALS(100, 200, 2, 15, 0.7, 0.4, true)
   }
 
+  test("rank-2 matrices implicit bulk") {
+    testALS(100, 200, 2, 15, 0.7, 0.4, true, true)
+  }
+
   /**
    * Test if we can correctly factorize R = U * P where U and P are of known rank.
    *
@@ -111,9 +127,12 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
    * @param iterations     number of iterations to run
    * @param samplingRate   what fraction of the user-product pairs are known
    * @param matchThreshold max difference allowed to consider a predicted rating correct
+   * @param implicitPrefs  flag to test implicit feedback
+   * @param bulkPredict    flag to test bulk prediciton
    */
   def testALS(users: Int, products: Int, features: Int, iterations: Int,
-    samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false)
+    samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false,
+    bulkPredict: Boolean = false)
   {
     val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products,
       features, samplingRate, implicitPrefs)
@@ -130,7 +149,17 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
     for ((p, vec) <- model.productFeatures.collect(); i <- 0 until features) {
       predictedP.put(p, i, vec(i))
     }
-    val predictedRatings = predictedU.mmul(predictedP.transpose)
+    val predictedRatings = bulkPredict match {
+      case false => predictedU.mmul(predictedP.transpose)
+      case true =>
+        val allRatings = new DoubleMatrix(users, products)
+        val usersProducts = for (u <- 0 until users; p <- 0 until products) yield (u, p)
+        val userProductsRDD = sc.parallelize(usersProducts)
+        model.predict(userProductsRDD).collect().foreach { elem =>
+          allRatings.put(elem.user, elem.product, elem.rating)
+        }
+        allRatings
+    }
 
     if (!implicitPrefs) {
       for (u <- 0 until users; p <- 0 until products) {
diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml
deleted file mode 100644
index 4cd28f34e3cbd743e7883f1a20faaf1c5ff2ba2a..0000000000000000000000000000000000000000
--- a/new-yarn/pom.xml
+++ /dev/null
@@ -1,161 +0,0 @@
-<?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">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent</artifactId>
-    <version>0.9.0-incubating-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <groupId>org.apache.spark</groupId>
-  <artifactId>spark-yarn_2.10</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project YARN Support</name>
-  <url>http://spark.incubator.apache.org/</url>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_2.10</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-      <version>${yarn.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-ipc</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_2.10</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <configuration>
-          <shadedArtifactAttached>false</shadedArtifactAttached>
-          <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.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>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>test</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <exportAntProperties>true</exportAntProperties>
-              <tasks>
-                <property name="spark.classpath" refid="maven.test.classpath" />
-                <property environment="env" />
-                <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">  
-                  <condition>
-                    <not>
-                      <or>
-                        <isset property="env.SCALA_HOME" />
-                        <isset property="env.SCALA_LIBRARY_PATH" />
-                      </or>
-                    </not>
-                  </condition>
-                </fail>
-              </tasks>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.scalatest</groupId>
-        <artifactId>scalatest-maven-plugin</artifactId>
-        <configuration>
-          <environmentVariables>
-            <SPARK_HOME>${basedir}/..</SPARK_HOME>
-            <SPARK_TESTING>1</SPARK_TESTING>
-            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
-          </environmentVariables>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
deleted file mode 100644
index 63a0449e5a0730085554d2b8ae86067135fa8dba..0000000000000000000000000000000000000000
--- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.scheduler.cluster
-
-import org.apache.spark._
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.deploy.yarn.YarnAllocationHandler
-import org.apache.spark.util.Utils
-
-/**
- *
- * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM.
- */
-private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
-
-  def this(sc: SparkContext) = this(sc, new Configuration())
-
-  // By default, rack is unknown
-  override def getRackForHost(hostPort: String): Option[String] = {
-    val host = Utils.parseHostPort(hostPort)._1
-    val retval = YarnAllocationHandler.lookupRack(conf, host)
-    if (retval != null) Some(retval) else None
-  }
-
-  override def postStartHook() {
-
-    // The yarn application is running, but the worker might not yet ready
-    // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
-    Thread.sleep(2000L)
-    logInfo("YarnClientClusterScheduler.postStartHook done")
-  }
-}
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
deleted file mode 100644
index b206780c7806e15c84944db05876f89c8f848040..0000000000000000000000000000000000000000
--- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.scheduler.cluster
-
-import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState}
-import org.apache.spark.{SparkException, Logging, SparkContext}
-import org.apache.spark.deploy.yarn.{Client, ClientArguments}
-
-private[spark] class YarnClientSchedulerBackend(
-    scheduler: ClusterScheduler,
-    sc: SparkContext)
-  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
-  with Logging {
-
-  var client: Client = null
-  var appId: ApplicationId = null
-
-  override def start() {
-    super.start()
-
-    val defalutWorkerCores = "2"
-    val defalutWorkerMemory = "512m"
-    val defaultWorkerNumber = "1"
-
-    val userJar = System.getenv("SPARK_YARN_APP_JAR")
-    var workerCores = System.getenv("SPARK_WORKER_CORES")
-    var workerMemory = System.getenv("SPARK_WORKER_MEMORY")
-    var workerNumber = System.getenv("SPARK_WORKER_INSTANCES")
-
-    if (userJar == null)
-      throw new SparkException("env SPARK_YARN_APP_JAR is not set")
-
-    if (workerCores == null)
-      workerCores = defalutWorkerCores
-    if (workerMemory == null)
-      workerMemory = defalutWorkerMemory
-    if (workerNumber == null)
-      workerNumber = defaultWorkerNumber
-
-    val driverHost = System.getProperty("spark.driver.host")
-    val driverPort = System.getProperty("spark.driver.port")
-    val hostport = driverHost + ":" + driverPort
-
-    val argsArray = Array[String](
-      "--class", "notused",
-      "--jar", userJar,
-      "--args", hostport,
-      "--worker-memory", workerMemory,
-      "--worker-cores", workerCores,
-      "--num-workers", workerNumber,
-      "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher"
-    )
-
-    val args = new ClientArguments(argsArray)
-    client = new Client(args)
-    appId = client.runApp()
-    waitForApp()
-  }
-
-  def waitForApp() {
-
-    // TODO : need a better way to find out whether the workers are ready or not
-    // maybe by resource usage report?
-    while(true) {
-      val report = client.getApplicationReport(appId)
-
-      logInfo("Application report from ASM: \n" +
-        "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
-        "\t appStartTime: " + report.getStartTime() + "\n" +
-        "\t yarnAppState: " + report.getYarnApplicationState() + "\n"
-      )
-
-      // Ready to go, or already gone.
-      val state = report.getYarnApplicationState()
-      if (state == YarnApplicationState.RUNNING) {
-        return
-      } else if (state == YarnApplicationState.FINISHED ||
-        state == YarnApplicationState.FAILED ||
-        state == YarnApplicationState.KILLED) {
-        throw new SparkException("Yarn application already ended," +
-          "might be killed or not able to launch application master.")
-      }
-
-      Thread.sleep(1000)
-    }
-  }
-
-  override def stop() {
-    super.stop()
-    client.stop()
-    logInfo("Stoped")
-  }
-
-}
diff --git a/pom.xml b/pom.xml
index 57e843596fe35450f86b471684c4be6e3d42e0f5..c2b1a7795a25657694d7525b78f6b373945b106b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -87,12 +87,17 @@
   <modules>
     <module>core</module>
     <module>bagel</module>
-    <module>examples</module>
     <module>mllib</module>
     <module>tools</module>
     <module>streaming</module>
     <module>repl</module>
     <module>assembly</module>
+    <module>external/twitter</module>
+    <module>external/kafka</module>
+    <module>external/flume</module>
+    <module>external/zeromq</module>
+    <module>external/mqtt</module>
+    <module>examples</module>
   </modules>
 
   <properties>
@@ -140,17 +145,6 @@
         <enabled>false</enabled>
       </snapshots>
     </repository>
-    <repository>
-      <id>mqtt-repo</id>
-      <name>MQTT Repository</name>
-      <url>https://repo.eclipse.org/content/repositories/paho-releases</url>
-      <releases>
-        <enabled>true</enabled>
-      </releases>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </repository>
   </repositories>
 
   <dependencyManagement>
@@ -188,7 +182,7 @@
       <dependency>
         <groupId>com.ning</groupId>
         <artifactId>compress-lzf</artifactId>
-        <version>0.8.4</version>
+        <version>1.0.0</version>
       </dependency>
       <dependency>
         <groupId>org.xerial.snappy</groupId>
@@ -200,6 +194,11 @@
         <artifactId>asm</artifactId>
         <version>4.0</version>
       </dependency>
+      <dependency>
+        <groupId>com.clearspring.analytics</groupId>
+        <artifactId>stream</artifactId>
+        <version>2.4.0</version>
+      </dependency>
       <!-- In theory we need not directly depend on protobuf since Spark does not directly
            use it. However, when building with Hadoop/YARN 2.2 Maven doesn't correctly bump
            the protobuf version up from the one Mesos gives. For now we include this variable 
@@ -255,7 +254,7 @@
       </dependency>
       <dependency>
         <groupId>${akka.group}</groupId>
-        <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
+        <artifactId>akka-testkit_${scala.binary.version}</artifactId>
         <version>${akka.version}</version>
         <exclusions>
           <exclusion>
@@ -282,7 +281,7 @@
       <dependency>
         <groupId>io.netty</groupId>
         <artifactId>netty-all</artifactId>
-        <version>4.0.0.CR1</version>
+        <version>4.0.13.Final</version>
       </dependency>
       <dependency>
         <groupId>org.apache.derby</groupId>
@@ -352,6 +351,12 @@
         <version>1.9.1</version>
         <scope>test</scope>
       </dependency>
+      <dependency>
+          <groupId>org.mockito</groupId>
+          <artifactId>mockito-all</artifactId>
+          <scope>test</scope>
+          <version>1.8.5</version>
+      </dependency>
       <dependency>
         <groupId>commons-io</groupId>
         <artifactId>commons-io</artifactId>
@@ -717,12 +722,11 @@
 
   <profiles>
     <profile>
-      <id>hadoop2-yarn</id>
+      <id>yarn-alpha</id>
       <properties>
         <hadoop.major.version>2</hadoop.major.version>
         <!-- 0.23.* is same as 2.0.* - except hardened to run production jobs -->
         <hadoop.version>0.23.7</hadoop.version>
-        <protobuf.version>2.5.0</protobuf.version>
         <!--<hadoop.version>2.0.5-alpha</hadoop.version> -->
       </properties>
 
@@ -730,57 +734,20 @@
         <module>yarn</module>
       </modules>
 
-      <repositories>
-        <repository>
-          <id>maven-root</id>
-          <name>Maven root repository</name>
-          <url>http://repo1.maven.org/maven2</url>
-          <releases>
-            <enabled>true</enabled>
-          </releases>
-          <snapshots>
-            <enabled>false</enabled>
-          </snapshots>
-        </repository>
-      </repositories>
-
-      <dependencyManagement>
-        <dependencies>
-        </dependencies>
-      </dependencyManagement>
     </profile>
 
     <profile>
-      <id>new-yarn</id>
-       <properties>
-         <hadoop.major.version>2</hadoop.major.version>
-         <hadoop.version>2.2.0</hadoop.version>
-         <protobuf.version>2.5.0</protobuf.version>
-       </properties>
-
-       <modules>
-         <module>new-yarn</module>
-       </modules>
-
-       <repositories>
-         <repository>
-           <id>maven-root</id>
-           <name>Maven root repository</name>
-           <url>http://repo1.maven.org/maven2</url>
-           <releases>
-             <enabled>true</enabled>
-           </releases>
-           <snapshots>
-             <enabled>false</enabled>
-           </snapshots>
-         </repository>
-       </repositories>
+      <id>yarn</id>
+      <properties>
+        <hadoop.major.version>2</hadoop.major.version>
+        <hadoop.version>2.2.0</hadoop.version>
+        <protobuf.version>2.5.0</protobuf.version>
+      </properties>
+      <modules>
+        <module>yarn</module>
+      </modules>
 
-       <dependencyManagement>
-         <dependencies>
-         </dependencies>
-       </dependencyManagement>
-     </profile>
+    </profile>
 
     <profile>
       <id>repl-bin</id>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 7bcbd90bd3f9e38f47443b1ab6985455b0365a9b..f794918d22e595c6faa79fc8b21cb451cd3c762e 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -20,6 +20,7 @@ import sbt.Classpaths.publishTask
 import Keys._
 import sbtassembly.Plugin._
 import AssemblyKeys._
+import scala.util.Properties
 // For Sonatype publishing
 //import com.jsuereth.pgp.sbtplugin.PgpKeys._
 
@@ -49,9 +50,6 @@ object SparkBuild extends Build {
   lazy val repl = Project("repl", file("repl"), settings = replSettings)
     .dependsOn(core, bagel, mllib)
 
-  lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
-    .dependsOn(core, mllib, bagel, 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)
@@ -68,10 +66,11 @@ object SparkBuild extends Build {
   // A configuration to set an alternative publishLocalConfiguration
   lazy val MavenCompile = config("m2r") extend(Compile)
   lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
+  val sparkHome = System.getProperty("user.dir")
 
   // Allows build configuration to be set through environment variables
-  lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
-  lazy val isNewHadoop = scala.util.Properties.envOrNone("SPARK_IS_NEW_HADOOP") match {
+  lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
+  lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match {
     case None => {
       val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined
       (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP)
@@ -79,23 +78,43 @@ object SparkBuild extends Build {
     case Some(v) => v.toBoolean
   }
 
-  lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match {
+  lazy val isYarnEnabled = Properties.envOrNone("SPARK_YARN") match {
     case None => DEFAULT_YARN
     case Some(v) => v.toBoolean
   }
 
   // Conditionally include the yarn sub-project
-  lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core)
-
-  //lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
-
-  lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
-  lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
+  lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core)
+  lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core)
+
+  lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]()
+  lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]()
+
+  lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) 
+    .dependsOn(streaming % "compile->compile;test->test")
+
+  lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+
+  lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+  
+  lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+  
+  lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+
+  lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
+  lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
+  
+  lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
+    .dependsOn(core, mllib, bagel, streaming, externalTwitter) dependsOn(allExternal: _*)
 
   // Everything except assembly, tools and examples belong to packageProjects
   lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef
 
-  lazy val allProjects = packageProjects ++ Seq[ProjectReference](examples, tools, assemblyProj)
+  lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) 
 
   def sharedSettings = Defaults.defaultSettings ++ Seq(
     organization       := "org.apache.spark",
@@ -112,8 +131,9 @@ object SparkBuild extends Build {
 
     // Fork new JVMs for tests and set Java options for those
     fork := true,
+    javaOptions in Test += "-Dspark.home=" + sparkHome,
+    javaOptions in Test += "-Dspark.testing=1",
     javaOptions += "-Xmx3g",
-
     // Show full stack trace and duration in test cases.
     testOptions in Test += Tests.Argument("-oDF"),
 
@@ -123,7 +143,7 @@ object SparkBuild extends Build {
     // also check the local Maven repository ~/.m2
     resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))),
 
-   // For Sonatype publishing
+    // For Sonatype publishing
     resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
       "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"),
 
@@ -165,7 +185,7 @@ object SparkBuild extends Build {
       </issueManagement>
     ),
 
-/*
+    /*
     publishTo <<= version { (v: String) =>
       val nexus = "https://oss.sonatype.org/"
       if (v.trim.endsWith("SNAPSHOT"))
@@ -174,11 +194,10 @@ object SparkBuild extends Build {
         Some("sonatype-staging"  at nexus + "service/local/staging/deploy/maven2")
     },
 
-*/
-
+    */
 
     libraryDependencies ++= Seq(
-        "io.netty"          % "netty-all"       % "4.0.0.CR1",
+        "io.netty"          % "netty-all"       % "4.0.13.Final",
         "org.eclipse.jetty" % "jetty-server"    % "7.6.8.v20121106",
         /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */
         "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"),
@@ -226,11 +245,12 @@ object SparkBuild extends Build {
         "org.slf4j"                % "slf4j-api"        % slf4jVersion,
         "org.slf4j"                % "slf4j-log4j12"    % slf4jVersion,
         "commons-daemon"           % "commons-daemon"   % "1.0.10", // workaround for bug HADOOP-9407
-        "com.ning"                 % "compress-lzf"     % "0.8.4",
+        "com.ning"                 % "compress-lzf"     % "1.0.0",
         "org.xerial.snappy"        % "snappy-java"      % "1.0.5",
         "org.ow2.asm"              % "asm"              % "4.0",
         "org.spark-project.akka"  %% "akka-remote"      % "2.2.3-shaded-protobuf"  excludeAll(excludeNetty),
         "org.spark-project.akka"  %% "akka-slf4j"       % "2.2.3-shaded-protobuf"  excludeAll(excludeNetty),
+        "org.spark-project.akka"  %% "akka-testkit"     % "2.2.3-shaded-protobuf" % "test",
         "net.liftweb"             %% "lift-json"        % "2.5.1"  excludeAll(excludeNetty),
         "it.unimi.dsi"             % "fastutil"         % "6.4.4",
         "colt"                     % "colt"             % "1.2.0",
@@ -247,7 +267,9 @@ object SparkBuild extends Build {
         "com.codahale.metrics"     % "metrics-ganglia"  % "3.0.0",
         "com.codahale.metrics"     % "metrics-graphite" % "3.0.0",
         "com.twitter"             %% "chill"            % "0.3.1",
-        "com.twitter"              % "chill-java"       % "0.3.1"
+        "com.twitter"              % "chill-java"       % "0.3.1",
+        "com.typesafe"             % "config"           % "1.0.2",
+        "com.clearspring.analytics" % "stream"          % "2.5.1"
       )
   )
 
@@ -262,7 +284,6 @@ object SparkBuild extends Build {
    libraryDependencies <+= scalaVersion(v => "org.scala-lang"  % "scala-reflect"  % v )
   )
 
-
   def examplesSettings = sharedSettings ++ Seq(
     name := "spark-examples",
     libraryDependencies ++= Seq(
@@ -299,29 +320,34 @@ object SparkBuild extends Build {
 
   def streamingSettings = sharedSettings ++ Seq(
     name := "spark-streaming",
-    resolvers ++= Seq(
-      "Eclipse Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/",
-      "Apache repo" at "https://repository.apache.org/content/repositories/releases"
-    ),
-
     libraryDependencies ++= Seq(
-      "org.apache.flume"        % "flume-ng-sdk"     % "1.2.0" % "compile"     excludeAll(excludeNetty, excludeSnappy),
-      "com.sksamuel.kafka"     %% "kafka"            % "0.8.0-beta1"
-        exclude("com.sun.jdmk", "jmxtools")
-        exclude("com.sun.jmx", "jmxri")
-        exclude("net.sf.jopt-simple", "jopt-simple")
-        excludeAll(excludeNetty),
-      "org.eclipse.paho"        % "mqtt-client"      % "0.4.0",
-      "com.github.sgroschupf"   % "zkclient"         % "0.1"                   excludeAll(excludeNetty),
-      "org.twitter4j"           % "twitter4j-stream" % "3.0.3"                 excludeAll(excludeNetty),
-      "org.spark-project.akka" %% "akka-zeromq"      % "2.2.3-shaded-protobuf" excludeAll(excludeNetty)
+      "commons-io" % "commons-io" % "2.4" 
     )
   )
 
-  def yarnSettings = sharedSettings ++ Seq(
-    name := "spark-yarn"
+  def yarnCommonSettings = sharedSettings ++ Seq(
+    unmanagedSourceDirectories in Compile <++= baseDirectory { base =>
+      Seq(
+         base / "../common/src/main/scala"
+      )
+    },
+
+    unmanagedSourceDirectories in Test <++= baseDirectory { base =>
+      Seq(
+         base / "../common/src/test/scala"
+      )
+    }
+
   ) ++ extraYarnSettings
 
+  def yarnAlphaSettings = yarnCommonSettings ++ Seq(
+    name := "spark-yarn-alpha"
+  )
+
+  def yarnSettings = yarnCommonSettings ++ Seq(
+    name := "spark-yarn"
+  )
+
   // 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).
   def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq()
@@ -329,14 +355,15 @@ object SparkBuild extends Build {
   def yarnEnabledSettings = Seq(
     libraryDependencies ++= Seq(
       // Exclude rule required for all ?
-      "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
-      "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+      "org.apache.hadoop" % "hadoop-client"      % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+      "org.apache.hadoop" % "hadoop-yarn-api"    % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
       "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
       "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib)
     )
   )
 
   def assemblyProjSettings = sharedSettings ++ Seq(
+    libraryDependencies += "net.sf.py4j" % "py4j" % "0.8.1",
     name := "spark-assembly",
     assembleDeps in Compile <<= (packageProjects.map(packageBin in Compile in _) ++ Seq(packageDependency in Compile)).dependOn,
     jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" },
@@ -354,4 +381,43 @@ object SparkBuild extends Build {
       case _ => MergeStrategy.first
     }
   )
+
+  def twitterSettings() = sharedSettings ++ Seq(
+    name := "spark-streaming-twitter",
+    libraryDependencies ++= Seq(
+      "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty)
+    )
+  )
+  
+  def kafkaSettings() = sharedSettings ++ Seq(
+    name := "spark-streaming-kafka",
+    libraryDependencies ++= Seq(
+      "com.github.sgroschupf"    % "zkclient"   % "0.1"          excludeAll(excludeNetty),
+      "com.sksamuel.kafka"      %% "kafka"      % "0.8.0-beta1"
+        exclude("com.sun.jdmk", "jmxtools")
+        exclude("com.sun.jmx", "jmxri")
+        exclude("net.sf.jopt-simple", "jopt-simple")
+        excludeAll(excludeNetty)
+    )
+  )
+  
+  def flumeSettings() = sharedSettings ++ Seq(
+    name := "spark-streaming-flume",
+    libraryDependencies ++= Seq(
+      "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy)
+    )
+  )
+
+  def zeromqSettings() = sharedSettings ++ Seq(
+    name := "spark-streaming-zeromq",
+    libraryDependencies ++= Seq(
+      "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty)
+    )
+  )
+
+  def mqttSettings() = streamingSettings ++ Seq(
+    name := "spark-streaming-mqtt",
+    resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"),
+    libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0")
+  )
 }
diff --git a/project/build.properties b/project/build.properties
index 96472771622d8e2ef4519ff645d6d195c1f9cdd6..839f5fbb0c34721bdb79c07ce77a0314ad5fc4c9 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -14,5 +14,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
 sbt.version=0.12.4
diff --git a/python/epydoc.conf b/python/epydoc.conf
index 0b42e729f8dcc756c711584de2b2a4f071b480c5..95a6af09748065a3e1939e8662a818dcc9211d44 100644
--- a/python/epydoc.conf
+++ b/python/epydoc.conf
@@ -34,4 +34,4 @@ private: no
 
 exclude: pyspark.cloudpickle pyspark.worker pyspark.join
          pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test
-         pyspark.rddsampler pyspark.daemon
+         pyspark.rddsampler pyspark.daemon pyspark.mllib._common
diff --git a/python/lib/py4j-0.8.1-src.zip b/python/lib/py4j-0.8.1-src.zip
new file mode 100644
index 0000000000000000000000000000000000000000..2069a328d1f2e6a94df057c6a3930048ae3f3832
Binary files /dev/null and b/python/lib/py4j-0.8.1-src.zip differ
diff --git a/python/lib/py4j0.7.egg b/python/lib/py4j0.7.egg
deleted file mode 100644
index f8a339d8eef591afce7e7e0e5beb73355bf0a4bf..0000000000000000000000000000000000000000
Binary files a/python/lib/py4j0.7.egg and /dev/null differ
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index 1f35f6f939d8e70c16ec3ebaff5d0b70df59c7bc..a51d5af79b0dafd6e685b21d76efd69405e613aa 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -20,28 +20,34 @@ PySpark is the Python API for Spark.
 
 Public classes:
 
-    - L{SparkContext<pyspark.context.SparkContext>}
-        Main entry point for Spark functionality.
-    - L{RDD<pyspark.rdd.RDD>}
-        A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
-    - L{Broadcast<pyspark.broadcast.Broadcast>}
-        A broadcast variable that gets reused across tasks.
-    - L{Accumulator<pyspark.accumulators.Accumulator>}
-        An "add-only" shared variable that tasks can only add values to.
-    - L{SparkFiles<pyspark.files.SparkFiles>}
-        Access files shipped with jobs.
-    - L{StorageLevel<pyspark.storagelevel.StorageLevel>}
-        Finer-grained cache persistence levels.
+  - L{SparkContext<pyspark.context.SparkContext>}
+      Main entry point for Spark functionality.
+  - L{RDD<pyspark.rdd.RDD>}
+      A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
+  - L{Broadcast<pyspark.broadcast.Broadcast>}
+      A broadcast variable that gets reused across tasks.
+  - L{Accumulator<pyspark.accumulators.Accumulator>}
+      An "add-only" shared variable that tasks can only add values to.
+  - L{SparkConf<pyspark.conf.SparkConf>}
+      For configuring Spark.
+  - L{SparkFiles<pyspark.files.SparkFiles>}
+      Access files shipped with jobs.
+  - L{StorageLevel<pyspark.storagelevel.StorageLevel>}
+      Finer-grained cache persistence levels.
 """
+
+
+
 import sys
 import os
-sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.egg"))
+sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j-0.8.1-src.zip"))
 
 
+from pyspark.conf import SparkConf
 from pyspark.context import SparkContext
 from pyspark.rdd import RDD
 from pyspark.files import SparkFiles
 from pyspark.storagelevel import StorageLevel
 
 
-__all__ = ["SparkContext", "RDD", "SparkFiles", "StorageLevel"]
+__all__ = ["SparkConf", "SparkContext", "RDD", "SparkFiles", "StorageLevel"]
diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py
index dfdaba274f43a86f97c66f17fdb6553f1a24676a..43f40f8783bfd426cb94f031527cc11ee43059c7 100644
--- a/python/pyspark/broadcast.py
+++ b/python/pyspark/broadcast.py
@@ -45,7 +45,18 @@ def _from_id(bid):
 
 
 class Broadcast(object):
+    """
+    A broadcast variable created with
+    L{SparkContext.broadcast()<pyspark.context.SparkContext.broadcast>}.
+    Access its value through C{.value}.
+    """
+
     def __init__(self, bid, value, java_broadcast=None, pickle_registry=None):
+        """
+        Should not be called directly by users -- use
+        L{SparkContext.broadcast()<pyspark.context.SparkContext.broadcast>}
+        instead.
+        """
         self.value = value
         self.bid = bid
         self._jbroadcast = java_broadcast
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
new file mode 100644
index 0000000000000000000000000000000000000000..d72aed6a30ec1eeb75ec2b6ced790283b210a378
--- /dev/null
+++ b/python/pyspark/conf.py
@@ -0,0 +1,171 @@
+#
+# 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.
+#
+
+"""
+>>> from pyspark.conf import SparkConf
+>>> from pyspark.context import SparkContext
+>>> conf = SparkConf()
+>>> conf.setMaster("local").setAppName("My app")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.master")
+u'local'
+>>> conf.get("spark.app.name")
+u'My app'
+>>> sc = SparkContext(conf=conf)
+>>> sc.master
+u'local'
+>>> sc.appName
+u'My app'
+>>> sc.sparkHome == None
+True
+
+>>> conf = SparkConf()
+>>> conf.setSparkHome("/path")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.home")
+u'/path'
+>>> conf.setExecutorEnv("VAR1", "value1")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")])
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.executorEnv.VAR1")
+u'value1'
+>>> print conf.toDebugString()
+spark.executorEnv.VAR1=value1
+spark.executorEnv.VAR3=value3
+spark.executorEnv.VAR4=value4
+spark.home=/path
+>>> sorted(conf.getAll(), key=lambda p: p[0])
+[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')]
+"""
+
+
+class SparkConf(object):
+    """
+    Configuration for a Spark application. Used to set various Spark
+    parameters as key-value pairs.
+
+    Most of the time, you would create a SparkConf object with
+    C{SparkConf()}, which will load values from C{spark.*} Java system
+    properties and any C{spark.conf} on your Spark classpath. In this
+    case, system properties take priority over C{spark.conf}, and any
+    parameters you set directly on the C{SparkConf} object take priority
+    over both of those.
+
+    For unit tests, you can also call C{SparkConf(false)} to skip
+    loading external settings and get the same configuration no matter
+    what is on the classpath.
+
+    All setter methods in this class support chaining. For example,
+    you can write C{conf.setMaster("local").setAppName("My app")}.
+
+    Note that once a SparkConf object is passed to Spark, it is cloned
+    and can no longer be modified by the user.
+    """
+
+    def __init__(self, loadDefaults=True, _jvm=None):
+        """
+        Create a new Spark configuration.
+
+        @param loadDefaults: whether to load values from Java system
+               properties and classpath (True by default)
+        @param _jvm: internal parameter used to pass a handle to the
+               Java VM; does not need to be set by users
+        """
+        from pyspark.context import SparkContext
+        SparkContext._ensure_initialized()
+        _jvm = _jvm or SparkContext._jvm
+        self._jconf = _jvm.SparkConf(loadDefaults)
+
+    def set(self, key, value):
+        """Set a configuration property."""
+        self._jconf.set(key, unicode(value))
+        return self
+
+    def setMaster(self, value):
+        """Set master URL to connect to."""
+        self._jconf.setMaster(value)
+        return self
+
+    def setAppName(self, value):
+        """Set application name."""
+        self._jconf.setAppName(value)
+        return self
+
+    def setSparkHome(self, value):
+        """Set path where Spark is installed on worker nodes."""
+        self._jconf.setSparkHome(value)
+        return self
+
+    def setExecutorEnv(self, key=None, value=None, pairs=None):
+        """Set an environment variable to be passed to executors."""
+        if (key != None and pairs != None) or (key == None and pairs == None):
+            raise Exception("Either pass one key-value pair or a list of pairs")
+        elif key != None:
+            self._jconf.setExecutorEnv(key, value)
+        elif pairs != None:
+            for (k, v) in pairs:
+                self._jconf.setExecutorEnv(k, v)
+        return self
+
+    def setAll(self, pairs):
+        """
+        Set multiple parameters, passed as a list of key-value pairs.
+
+        @param pairs: list of key-value pairs to set
+        """
+        for (k, v) in pairs:
+            self._jconf.set(k, v)
+        return self
+
+    def get(self, key, defaultValue=None):
+        """Get the configured value for some key, or return a default otherwise."""
+        if defaultValue == None:   # Py4J doesn't call the right get() if we pass None
+            if not self._jconf.contains(key):
+                return None
+            return self._jconf.get(key)
+        else:
+            return self._jconf.get(key, defaultValue)
+
+    def getAll(self):
+        """Get all values as a list of key-value pairs."""
+        pairs = []
+        for elem in self._jconf.getAll():
+            pairs.append((elem._1(), elem._2()))
+        return pairs
+
+    def contains(self, key):
+        """Does this configuration contain a given key?"""
+        return self._jconf.contains(key)
+
+    def toDebugString(self):
+        """
+        Returns a printable version of the configuration, as a list of
+        key=value pairs, one per line.
+        """
+        return self._jconf.toDebugString()
+
+
+def _test():
+    import doctest
+    (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS)
+    if failure_count:
+        exit(-1)
+
+
+if __name__ == "__main__":
+    _test()
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 108f36576a0fb39d2c4a503a2ceab8e02cd32187..f955aad7a4f12bc21c0aabc4fd2b9ffa58caf576 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -24,6 +24,7 @@ from tempfile import NamedTemporaryFile
 from pyspark import accumulators
 from pyspark.accumulators import Accumulator
 from pyspark.broadcast import Broadcast
+from pyspark.conf import SparkConf
 from pyspark.files import SparkFiles
 from pyspark.java_gateway import launch_gateway
 from pyspark.serializers import PickleSerializer, BatchedSerializer, MUTF8Deserializer
@@ -49,14 +50,15 @@ class SparkContext(object):
     _python_includes = None # zip and egg files that need to be added to PYTHONPATH
 
 
-    def __init__(self, master, jobName, sparkHome=None, pyFiles=None,
-        environment=None, batchSize=1024, serializer=PickleSerializer()):
+    def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
+        environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None):
         """
-        Create a new SparkContext.
+        Create a new SparkContext. At least the master and app name should be set,
+        either through the named parameters here or through C{conf}.
 
         @param master: Cluster URL to connect to
                (e.g. mesos://host:port, spark://host:port, local[4]).
-        @param jobName: A name for your job, to display on the cluster web UI
+        @param appName: A name for your job, to display on the cluster web UI.
         @param sparkHome: Location where Spark is installed on cluster nodes.
         @param pyFiles: Collection of .zip or .py files to send to the cluster
                and add to PYTHONPATH.  These can be paths on the local file
@@ -67,6 +69,7 @@ class SparkContext(object):
                Java object.  Set 1 to disable batching or -1 to use an
                unlimited batch size.
         @param serializer: The serializer for RDDs.
+        @param conf: A L{SparkConf} object setting Spark properties.
 
 
         >>> from pyspark.context import SparkContext
@@ -79,10 +82,8 @@ class SparkContext(object):
         """
         SparkContext._ensure_initialized(self)
 
-        self.master = master
-        self.jobName = jobName
-        self.sparkHome = sparkHome or None # None becomes null in Py4J
         self.environment = environment or {}
+        self._conf = conf or SparkConf(_jvm=self._jvm)
         self._batchSize = batchSize  # -1 represents an unlimited batch size
         self._unbatched_serializer = serializer
         if batchSize == 1:
@@ -91,10 +92,35 @@ class SparkContext(object):
             self.serializer = BatchedSerializer(self._unbatched_serializer,
                                                 batchSize)
 
+        # Set any parameters passed directly to us on the conf
+        if master:
+            self._conf.setMaster(master)
+        if appName:
+            self._conf.setAppName(appName)
+        if sparkHome:
+            self._conf.setSparkHome(sparkHome)
+        if environment:
+            for key, value in environment.iteritems():
+                self._conf.setExecutorEnv(key, value)
+
+        # Check that we have at least the required parameters
+        if not self._conf.contains("spark.master"):
+            raise Exception("A master URL must be set in your configuration")
+        if not self._conf.contains("spark.app.name"):
+            raise Exception("An application name must be set in your configuration")
+
+        # Read back our properties from the conf in case we loaded some of them from
+        # the classpath or an external config file
+        self.master = self._conf.get("spark.master")
+        self.appName = self._conf.get("spark.app.name")
+        self.sparkHome = self._conf.get("spark.home", None)
+        for (k, v) in self._conf.getAll():
+            if k.startswith("spark.executorEnv."):
+                varName = k[len("spark.executorEnv."):]
+                self.environment[varName] = v
+
         # Create the Java SparkContext through Py4J
-        empty_string_array = self._gateway.new_array(self._jvm.String, 0)
-        self._jsc = self._jvm.JavaSparkContext(master, jobName, sparkHome,
-                                              empty_string_array)
+        self._jsc = self._jvm.JavaSparkContext(self._conf._jconf)
 
         # Create a single Accumulator in Java that we'll send all our updates through;
         # they will be passed back to us through a TCP server
@@ -105,6 +131,7 @@ class SparkContext(object):
                 self._jvm.PythonAccumulatorParam(host, port))
 
         self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python')
+
         # Broadcast's __reduce__ method stores Broadcast instances here.
         # This allows other code to determine which Broadcast instances have
         # been pickled, so it can determine which Java broadcast objects to
@@ -121,7 +148,7 @@ class SparkContext(object):
             self.addPyFile(path)
 
         # Create a temporary directory inside spark.local.dir:
-        local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir()
+        local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf())
         self._temp_dir = \
             self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath()
 
@@ -131,8 +158,7 @@ class SparkContext(object):
             if not SparkContext._gateway:
                 SparkContext._gateway = launch_gateway()
                 SparkContext._jvm = SparkContext._gateway.jvm
-                SparkContext._writeToFile = \
-                    SparkContext._jvm.PythonRDD.writeToFile
+                SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile
 
             if instance:
                 if SparkContext._active_spark_context and SparkContext._active_spark_context != instance:
@@ -143,8 +169,8 @@ class SparkContext(object):
     @classmethod
     def setSystemProperty(cls, key, value):
         """
-        Set a system property, such as spark.executor.memory. This must be
-        invoked before instantiating SparkContext.
+        Set a Java system property, such as spark.executor.memory. This must
+        must be invoked before instantiating SparkContext.
         """
         SparkContext._ensure_initialized()
         SparkContext._jvm.java.lang.System.setProperty(key, value)
@@ -243,7 +269,8 @@ class SparkContext(object):
 
     def broadcast(self, value):
         """
-        Broadcast a read-only variable to the cluster, returning a C{Broadcast}
+        Broadcast a read-only variable to the cluster, returning a
+        L{Broadcast<pyspark.broadcast.Broadcast>}
         object for reading it in distributed functions. The variable will be
         sent to each cluster only once.
         """
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index e615c1e9b63a46a857aa575561522410c2a6ce7e..c15add52375072bd1e5167c7b26f086727e8ccef 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -31,7 +31,7 @@ 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
     on_windows = platform.system() == "Windows"
-    script = "spark-class.cmd" if on_windows else "spark-class"
+    script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
     command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
                "--die-on-broken-pipe", "0"]
     if not on_windows:
@@ -60,7 +60,9 @@ def launch_gateway():
     # Connect to the gateway
     gateway = JavaGateway(GatewayClient(port=port), auto_convert=False)
     # Import the classes used by PySpark
+    java_import(gateway.jvm, "org.apache.spark.SparkConf")
     java_import(gateway.jvm, "org.apache.spark.api.java.*")
     java_import(gateway.jvm, "org.apache.spark.api.python.*")
+    java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*")
     java_import(gateway.jvm, "scala.Tuple2")
     return gateway
diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py
new file mode 100644
index 0000000000000000000000000000000000000000..b1a5df109b46edc3e5221401dcadf2d7e248ca74
--- /dev/null
+++ b/python/pyspark/mllib/__init__.py
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+
+"""
+Python bindings for MLlib.
+"""
diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py
new file mode 100644
index 0000000000000000000000000000000000000000..769d88dfb9b56a8c77b7e2fd2779015fa2dbfef3
--- /dev/null
+++ b/python/pyspark/mllib/_common.py
@@ -0,0 +1,252 @@
+#
+# 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.
+#
+
+from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape
+from pyspark import SparkContext
+
+from pyspark.serializers import Serializer
+import struct
+
+# Double vector format:
+#
+# [8-byte 1] [8-byte length] [length*8 bytes of data]
+#
+# Double matrix format:
+#
+# [8-byte 2] [8-byte rows] [8-byte cols] [rows*cols*8 bytes of data]
+#
+# This is all in machine-endian.  That means that the Java interpreter and the
+# Python interpreter must agree on what endian the machine is.
+
+def _deserialize_byte_array(shape, ba, offset):
+    """Wrapper around ndarray aliasing hack.
+
+    >>> x = array([1.0, 2.0, 3.0, 4.0, 5.0])
+    >>> array_equal(x, _deserialize_byte_array(x.shape, x.data, 0))
+    True
+    >>> x = array([1.0, 2.0, 3.0, 4.0]).reshape(2,2)
+    >>> array_equal(x, _deserialize_byte_array(x.shape, x.data, 0))
+    True
+    """
+    ar = ndarray(shape=shape, buffer=ba, offset=offset, dtype="float64",
+            order='C')
+    return ar.copy()
+
+def _serialize_double_vector(v):
+    """Serialize a double vector into a mutually understood format."""
+    if type(v) != ndarray:
+        raise TypeError("_serialize_double_vector called on a %s; "
+                "wanted ndarray" % type(v))
+    if v.dtype != float64:
+        raise TypeError("_serialize_double_vector called on an ndarray of %s; "
+                "wanted ndarray of float64" % v.dtype)
+    if v.ndim != 1:
+        raise TypeError("_serialize_double_vector called on a %ddarray; "
+                "wanted a 1darray" % v.ndim)
+    length = v.shape[0]
+    ba = bytearray(16 + 8*length)
+    header = ndarray(shape=[2], buffer=ba, dtype="int64")
+    header[0] = 1
+    header[1] = length
+    copyto(ndarray(shape=[length], buffer=ba, offset=16,
+            dtype="float64"), v)
+    return ba
+
+def _deserialize_double_vector(ba):
+    """Deserialize a double vector from a mutually understood format.
+
+    >>> x = array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0])
+    >>> array_equal(x, _deserialize_double_vector(_serialize_double_vector(x)))
+    True
+    """
+    if type(ba) != bytearray:
+        raise TypeError("_deserialize_double_vector called on a %s; "
+                "wanted bytearray" % type(ba))
+    if len(ba) < 16:
+        raise TypeError("_deserialize_double_vector called on a %d-byte array, "
+                "which is too short" % len(ba))
+    if (len(ba) & 7) != 0:
+        raise TypeError("_deserialize_double_vector called on a %d-byte array, "
+                "which is not a multiple of 8" % len(ba))
+    header = ndarray(shape=[2], buffer=ba, dtype="int64")
+    if header[0] != 1:
+        raise TypeError("_deserialize_double_vector called on bytearray "
+                        "with wrong magic")
+    length = header[1]
+    if len(ba) != 8*length + 16:
+        raise TypeError("_deserialize_double_vector called on bytearray "
+                        "with wrong length")
+    return _deserialize_byte_array([length], ba, 16)
+
+def _serialize_double_matrix(m):
+    """Serialize a double matrix into a mutually understood format."""
+    if (type(m) == ndarray and m.dtype == float64 and m.ndim == 2):
+        rows = m.shape[0]
+        cols = m.shape[1]
+        ba = bytearray(24 + 8 * rows * cols)
+        header = ndarray(shape=[3], buffer=ba, dtype="int64")
+        header[0] = 2
+        header[1] = rows
+        header[2] = cols
+        copyto(ndarray(shape=[rows, cols], buffer=ba, offset=24,
+                       dtype="float64", order='C'), m)
+        return ba
+    else:
+        raise TypeError("_serialize_double_matrix called on a "
+                        "non-double-matrix")
+
+def _deserialize_double_matrix(ba):
+    """Deserialize a double matrix from a mutually understood format."""
+    if type(ba) != bytearray:
+        raise TypeError("_deserialize_double_matrix called on a %s; "
+                "wanted bytearray" % type(ba))
+    if len(ba) < 24:
+        raise TypeError("_deserialize_double_matrix called on a %d-byte array, "
+                "which is too short" % len(ba))
+    if (len(ba) & 7) != 0:
+        raise TypeError("_deserialize_double_matrix called on a %d-byte array, "
+                "which is not a multiple of 8" % len(ba))
+    header = ndarray(shape=[3], buffer=ba, dtype="int64")
+    if (header[0] != 2):
+        raise TypeError("_deserialize_double_matrix called on bytearray "
+                        "with wrong magic")
+    rows = header[1]
+    cols = header[2]
+    if (len(ba) != 8*rows*cols + 24):
+        raise TypeError("_deserialize_double_matrix called on bytearray "
+                        "with wrong length")
+    return _deserialize_byte_array([rows, cols], ba, 24)
+
+def _linear_predictor_typecheck(x, coeffs):
+    """Check that x is a one-dimensional vector of the right shape.
+    This is a temporary hackaround until I actually implement bulk predict."""
+    if type(x) == ndarray:
+        if x.ndim == 1:
+            if x.shape == coeffs.shape:
+                pass
+            else:
+                raise RuntimeError("Got array of %d elements; wanted %d"
+                        % (shape(x)[0], shape(coeffs)[0]))
+        else:
+            raise RuntimeError("Bulk predict not yet supported.")
+    elif (type(x) == RDD):
+        raise RuntimeError("Bulk predict not yet supported.")
+    else:
+        raise TypeError("Argument of type " + type(x).__name__ + " unsupported")
+
+def _get_unmangled_rdd(data, serializer):
+    dataBytes = data.map(serializer)
+    dataBytes._bypass_serializer = True
+    dataBytes.cache()
+    return dataBytes
+
+# Map a pickled Python RDD of numpy double vectors to a Java RDD of
+# _serialized_double_vectors
+def _get_unmangled_double_vector_rdd(data):
+    return _get_unmangled_rdd(data, _serialize_double_vector)
+
+class LinearModel(object):
+    """Something that has a vector of coefficients and an intercept."""
+    def __init__(self, coeff, intercept):
+        self._coeff = coeff
+        self._intercept = intercept
+
+class LinearRegressionModelBase(LinearModel):
+    """A linear regression model.
+
+    >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1)
+    >>> abs(lrmb.predict(array([-1.03, 7.777])) - 14.624) < 1e-6
+    True
+    """
+    def predict(self, x):
+        """Predict the value of the dependent variable given a vector x"""
+        """containing values for the independent variables."""
+        _linear_predictor_typecheck(x, self._coeff)
+        return dot(self._coeff, x) + self._intercept
+
+# If we weren't given initial weights, take a zero vector of the appropriate
+# length.
+def _get_initial_weights(initial_weights, data):
+    if initial_weights is None:
+        initial_weights = data.first()
+        if type(initial_weights) != ndarray:
+            raise TypeError("At least one data element has type "
+                    + type(initial_weights).__name__ + " which is not ndarray")
+        if initial_weights.ndim != 1:
+            raise TypeError("At least one data element has "
+                    + initial_weights.ndim + " dimensions, which is not 1")
+        initial_weights = ones([initial_weights.shape[0] - 1])
+    return initial_weights
+
+# train_func should take two parameters, namely data and initial_weights, and
+# return the result of a call to the appropriate JVM stub.
+# _regression_train_wrapper is responsible for setup and error checking.
+def _regression_train_wrapper(sc, train_func, klass, data, initial_weights):
+    initial_weights = _get_initial_weights(initial_weights, data)
+    dataBytes = _get_unmangled_double_vector_rdd(data)
+    ans = train_func(dataBytes, _serialize_double_vector(initial_weights))
+    if len(ans) != 2:
+        raise RuntimeError("JVM call result had unexpected length")
+    elif type(ans[0]) != bytearray:
+        raise RuntimeError("JVM call result had first element of type "
+                + type(ans[0]).__name__ + " which is not bytearray")
+    elif type(ans[1]) != float:
+        raise RuntimeError("JVM call result had second element of type "
+                + type(ans[0]).__name__ + " which is not float")
+    return klass(_deserialize_double_vector(ans[0]), ans[1])
+
+def _serialize_rating(r):
+    ba = bytearray(16)
+    intpart = ndarray(shape=[2], buffer=ba, dtype=int32)
+    doublepart = ndarray(shape=[1], buffer=ba, dtype=float64, offset=8)
+    intpart[0], intpart[1], doublepart[0] = r
+    return ba
+
+class RatingDeserializer(Serializer):
+    def loads(self, stream):
+        length = struct.unpack("!i", stream.read(4))[0]
+        ba = stream.read(length)
+        res = ndarray(shape=(3, ), buffer=ba, dtype="float64", offset=4)
+        return int(res[0]), int(res[1]), res[2]
+
+    def load_stream(self, stream):
+        while True:
+            try:
+                yield self.loads(stream)
+            except struct.error:
+                return
+            except EOFError:
+                return
+
+def _serialize_tuple(t):
+    ba = bytearray(8)
+    intpart = ndarray(shape=[2], buffer=ba, dtype=int32)
+    intpart[0], intpart[1] = t
+    return ba
+
+def _test():
+    import doctest
+    globs = globals().copy()
+    globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+    (failure_count, test_count) = doctest.testmod(globs=globs,
+            optionflags=doctest.ELLIPSIS)
+    globs['sc'].stop()
+    if failure_count:
+        exit(-1)
+
+if __name__ == "__main__":
+    _test()
diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py
new file mode 100644
index 0000000000000000000000000000000000000000..70de332d3468ea06e851ec4dbd797926dd32495a
--- /dev/null
+++ b/python/pyspark/mllib/classification.py
@@ -0,0 +1,86 @@
+#
+# 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.
+#
+
+from numpy import array, dot, shape
+from pyspark import SparkContext
+from pyspark.mllib._common import \
+    _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+    _serialize_double_matrix, _deserialize_double_matrix, \
+    _serialize_double_vector, _deserialize_double_vector, \
+    _get_initial_weights, _serialize_rating, _regression_train_wrapper, \
+    LinearModel, _linear_predictor_typecheck
+from math import exp, log
+
+class LogisticRegressionModel(LinearModel):
+    """A linear binary classification model derived from logistic regression.
+
+    >>> data = array([0.0, 0.0, 1.0, 1.0, 1.0, 2.0, 1.0, 3.0]).reshape(4,2)
+    >>> lrm = LogisticRegressionWithSGD.train(sc, sc.parallelize(data))
+    >>> lrm.predict(array([1.0])) != None
+    True
+    """
+    def predict(self, x):
+        _linear_predictor_typecheck(x, self._coeff)
+        margin = dot(x, self._coeff) + self._intercept
+        prob = 1/(1 + exp(-margin))
+        return 1 if prob > 0.5 else 0
+
+class LogisticRegressionWithSGD(object):
+    @classmethod
+    def train(cls, sc, data, iterations=100, step=1.0,
+              mini_batch_fraction=1.0, initial_weights=None):
+        """Train a logistic regression model on the given data."""
+        return _regression_train_wrapper(sc, lambda d, i:
+                sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD(d._jrdd,
+                        iterations, step, mini_batch_fraction, i),
+                LogisticRegressionModel, data, initial_weights)
+
+class SVMModel(LinearModel):
+    """A support vector machine.
+
+    >>> data = array([0.0, 0.0, 1.0, 1.0, 1.0, 2.0, 1.0, 3.0]).reshape(4,2)
+    >>> svm = SVMWithSGD.train(sc, sc.parallelize(data))
+    >>> svm.predict(array([1.0])) != None
+    True
+    """
+    def predict(self, x):
+        _linear_predictor_typecheck(x, self._coeff)
+        margin = dot(x, self._coeff) + self._intercept
+        return 1 if margin >= 0 else 0
+
+class SVMWithSGD(object):
+    @classmethod
+    def train(cls, sc, data, iterations=100, step=1.0, reg_param=1.0,
+              mini_batch_fraction=1.0, initial_weights=None):
+        """Train a support vector machine on the given data."""
+        return _regression_train_wrapper(sc, lambda d, i:
+                sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD(d._jrdd,
+                        iterations, step, reg_param, mini_batch_fraction, i),
+                SVMModel, data, initial_weights)
+
+def _test():
+    import doctest
+    globs = globals().copy()
+    globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+    (failure_count, test_count) = doctest.testmod(globs=globs,
+            optionflags=doctest.ELLIPSIS)
+    globs['sc'].stop()
+    if failure_count:
+        exit(-1)
+
+if __name__ == "__main__":
+    _test()
diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py
new file mode 100644
index 0000000000000000000000000000000000000000..8cf20e591af7b70ad87880217421cffa50a13bf8
--- /dev/null
+++ b/python/pyspark/mllib/clustering.py
@@ -0,0 +1,79 @@
+#
+# 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.
+#
+
+from numpy import array, dot
+from math import sqrt
+from pyspark import SparkContext
+from pyspark.mllib._common import \
+    _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+    _serialize_double_matrix, _deserialize_double_matrix, \
+    _serialize_double_vector, _deserialize_double_vector, \
+    _get_initial_weights, _serialize_rating, _regression_train_wrapper
+
+class KMeansModel(object):
+    """A clustering model derived from the k-means method.
+
+    >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2)
+    >>> clusters = KMeans.train(sc, sc.parallelize(data), 2, maxIterations=10, runs=30, initialization_mode="random")
+    >>> clusters.predict(array([0.0, 0.0])) == clusters.predict(array([1.0, 1.0]))
+    True
+    >>> clusters.predict(array([8.0, 9.0])) == clusters.predict(array([9.0, 8.0]))
+    True
+    >>> clusters = KMeans.train(sc, sc.parallelize(data), 2)
+    """
+    def __init__(self, centers_):
+        self.centers = centers_
+
+    def predict(self, x):
+        """Find the cluster to which x belongs in this model."""
+        best = 0
+        best_distance = 1e75
+        for i in range(0, self.centers.shape[0]):
+            diff = x - self.centers[i]
+            distance = sqrt(dot(diff, diff))
+            if distance < best_distance:
+                best = i
+                best_distance = distance
+        return best
+
+class KMeans(object):
+    @classmethod
+    def train(cls, sc, data, k, maxIterations=100, runs=1,
+            initialization_mode="k-means||"):
+        """Train a k-means clustering model."""
+        dataBytes = _get_unmangled_double_vector_rdd(data)
+        ans = sc._jvm.PythonMLLibAPI().trainKMeansModel(dataBytes._jrdd,
+                k, maxIterations, runs, initialization_mode)
+        if len(ans) != 1:
+            raise RuntimeError("JVM call result had unexpected length")
+        elif type(ans[0]) != bytearray:
+            raise RuntimeError("JVM call result had first element of type "
+                    + type(ans[0]) + " which is not bytearray")
+        return KMeansModel(_deserialize_double_matrix(ans[0]))
+
+def _test():
+    import doctest
+    globs = globals().copy()
+    globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+    (failure_count, test_count) = doctest.testmod(globs=globs,
+            optionflags=doctest.ELLIPSIS)
+    globs['sc'].stop()
+    if failure_count:
+        exit(-1)
+
+if __name__ == "__main__":
+    _test()
diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py
new file mode 100644
index 0000000000000000000000000000000000000000..0eeb5bb66b9d4e11c31805827a2c43dccea9eeaa
--- /dev/null
+++ b/python/pyspark/mllib/recommendation.py
@@ -0,0 +1,84 @@
+#
+# 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.
+#
+
+from pyspark import SparkContext
+from pyspark.mllib._common import \
+    _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+    _serialize_double_matrix, _deserialize_double_matrix, \
+    _serialize_double_vector, _deserialize_double_vector, \
+    _get_initial_weights, _serialize_rating, _regression_train_wrapper, \
+    _serialize_tuple, RatingDeserializer
+from pyspark.rdd import RDD
+
+class MatrixFactorizationModel(object):
+    """A matrix factorisation model trained by regularized alternating
+    least-squares.
+
+    >>> r1 = (1, 1, 1.0)
+    >>> r2 = (1, 2, 2.0)
+    >>> r3 = (2, 1, 2.0)
+    >>> ratings = sc.parallelize([r1, r2, r3])
+    >>> model = ALS.trainImplicit(sc, ratings, 1)
+    >>> model.predict(2,2) is not None
+    True
+    >>> testset = sc.parallelize([(1, 2), (1, 1)])
+    >>> model.predictAll(testset).count == 2
+    True
+    """
+
+    def __init__(self, sc, java_model):
+        self._context = sc
+        self._java_model = java_model
+
+    def __del__(self):
+        self._context._gateway.detach(self._java_model)
+
+    def predict(self, user, product):
+        return self._java_model.predict(user, product)
+
+    def predictAll(self, usersProducts):
+        usersProductsJRDD = _get_unmangled_rdd(usersProducts, _serialize_tuple)
+        return RDD(self._java_model.predict(usersProductsJRDD._jrdd),
+                   self._context, RatingDeserializer())
+
+class ALS(object):
+    @classmethod
+    def train(cls, sc, ratings, rank, iterations=5, lambda_=0.01, blocks=-1):
+        ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating)
+        mod = sc._jvm.PythonMLLibAPI().trainALSModel(ratingBytes._jrdd,
+                rank, iterations, lambda_, blocks)
+        return MatrixFactorizationModel(sc, mod)
+
+    @classmethod
+    def trainImplicit(cls, sc, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01):
+        ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating)
+        mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel(ratingBytes._jrdd,
+                rank, iterations, lambda_, blocks, alpha)
+        return MatrixFactorizationModel(sc, mod)
+
+def _test():
+    import doctest
+    globs = globals().copy()
+    globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+    (failure_count, test_count) = doctest.testmod(globs=globs,
+            optionflags=doctest.ELLIPSIS)
+    globs['sc'].stop()
+    if failure_count:
+        exit(-1)
+
+if __name__ == "__main__":
+    _test()
diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py
new file mode 100644
index 0000000000000000000000000000000000000000..a3a68b29e01cbd45f233a346d85c9855bf9ec74e
--- /dev/null
+++ b/python/pyspark/mllib/regression.py
@@ -0,0 +1,110 @@
+#
+# 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.
+#
+
+from numpy import array, dot
+from pyspark import SparkContext
+from pyspark.mllib._common import \
+    _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+    _serialize_double_matrix, _deserialize_double_matrix, \
+    _serialize_double_vector, _deserialize_double_vector, \
+    _get_initial_weights, _serialize_rating, _regression_train_wrapper, \
+    _linear_predictor_typecheck
+
+class LinearModel(object):
+    """Something that has a vector of coefficients and an intercept."""
+    def __init__(self, coeff, intercept):
+        self._coeff = coeff
+        self._intercept = intercept
+
+class LinearRegressionModelBase(LinearModel):
+    """A linear regression model.
+
+    >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1)
+    >>> abs(lrmb.predict(array([-1.03, 7.777])) - 14.624) < 1e-6
+    True
+    """
+    def predict(self, x):
+        """Predict the value of the dependent variable given a vector x"""
+        """containing values for the independent variables."""
+        _linear_predictor_typecheck(x, self._coeff)
+        return dot(self._coeff, x) + self._intercept
+
+class LinearRegressionModel(LinearRegressionModelBase):
+    """A linear regression model derived from a least-squares fit.
+
+    >>> data = array([0.0, 0.0, 1.0, 1.0, 3.0, 2.0, 2.0, 3.0]).reshape(4,2)
+    >>> lrm = LinearRegressionWithSGD.train(sc, sc.parallelize(data), initial_weights=array([1.0]))
+    """
+
+class LinearRegressionWithSGD(object):
+    @classmethod
+    def train(cls, sc, data, iterations=100, step=1.0,
+              mini_batch_fraction=1.0, initial_weights=None):
+        """Train a linear regression model on the given data."""
+        return _regression_train_wrapper(sc, lambda d, i:
+                sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD(
+                        d._jrdd, iterations, step, mini_batch_fraction, i),
+                LinearRegressionModel, data, initial_weights)
+
+class LassoModel(LinearRegressionModelBase):
+    """A linear regression model derived from a least-squares fit with an
+    l_1 penalty term.
+
+    >>> data = array([0.0, 0.0, 1.0, 1.0, 3.0, 2.0, 2.0, 3.0]).reshape(4,2)
+    >>> lrm = LassoWithSGD.train(sc, sc.parallelize(data), initial_weights=array([1.0]))
+    """
+    
+class LassoWithSGD(object):
+    @classmethod
+    def train(cls, sc, data, iterations=100, step=1.0, reg_param=1.0,
+              mini_batch_fraction=1.0, initial_weights=None):
+        """Train a Lasso regression model on the given data."""
+        return _regression_train_wrapper(sc, lambda d, i:
+                sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD(d._jrdd,
+                        iterations, step, reg_param, mini_batch_fraction, i),
+                LassoModel, data, initial_weights)
+
+class RidgeRegressionModel(LinearRegressionModelBase):
+    """A linear regression model derived from a least-squares fit with an
+    l_2 penalty term.
+
+    >>> data = array([0.0, 0.0, 1.0, 1.0, 3.0, 2.0, 2.0, 3.0]).reshape(4,2)
+    >>> lrm = RidgeRegressionWithSGD.train(sc, sc.parallelize(data), initial_weights=array([1.0]))
+    """
+
+class RidgeRegressionWithSGD(object):
+    @classmethod
+    def train(cls, sc, data, iterations=100, step=1.0, reg_param=1.0,
+              mini_batch_fraction=1.0, initial_weights=None):
+        """Train a ridge regression model on the given data."""
+        return _regression_train_wrapper(sc, lambda d, i:
+                sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD(d._jrdd,
+                        iterations, step, reg_param, mini_batch_fraction, i),
+                RidgeRegressionModel, data, initial_weights)
+
+def _test():
+    import doctest
+    globs = globals().copy()
+    globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+    (failure_count, test_count) = doctest.testmod(globs=globs,
+            optionflags=doctest.ELLIPSIS)
+    globs['sc'].stop()
+    if failure_count:
+        exit(-1)
+
+if __name__ == "__main__":
+    _test()
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index f87923e6fa4ebde8c8e5bad3b4b8ae53b673a68d..6fb4a7b3be25dfaeea2a7bdca0512868ade70a58 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -23,6 +23,7 @@ import operator
 import os
 import sys
 import shlex
+import traceback
 from subprocess import Popen, PIPE
 from tempfile import NamedTemporaryFile
 from threading import Thread
@@ -39,6 +40,46 @@ from py4j.java_collections import ListConverter, MapConverter
 
 __all__ = ["RDD"]
 
+def _extract_concise_traceback():
+    tb = traceback.extract_stack()
+    if len(tb) == 0:
+        return "I'm lost!"
+    # HACK:  This function is in a file called 'rdd.py' in the top level of
+    # everything PySpark.  Just trim off the directory name and assume
+    # everything in that tree is PySpark guts.
+    file, line, module, what = tb[len(tb) - 1]
+    sparkpath = os.path.dirname(file)
+    first_spark_frame = len(tb) - 1
+    for i in range(0, len(tb)):
+        file, line, fun, what = tb[i]
+        if file.startswith(sparkpath):
+            first_spark_frame = i
+            break
+    if first_spark_frame == 0:
+        file, line, fun, what = tb[0]
+        return "%s at %s:%d" % (fun, file, line)
+    sfile, sline, sfun, swhat = tb[first_spark_frame]
+    ufile, uline, ufun, uwhat = tb[first_spark_frame-1]
+    return "%s at %s:%d" % (sfun, ufile, uline)
+
+_spark_stack_depth = 0
+
+class _JavaStackTrace(object):
+    def __init__(self, sc):
+        self._traceback = _extract_concise_traceback()
+        self._context = sc
+
+    def __enter__(self):
+        global _spark_stack_depth
+        if _spark_stack_depth == 0:
+            self._context._jsc.setCallSite(self._traceback)
+        _spark_stack_depth += 1
+
+    def __exit__(self, type, value, tb):
+        global _spark_stack_depth
+        _spark_stack_depth -= 1
+        if _spark_stack_depth == 0:
+            self._context._jsc.setCallSite(None)
 
 class RDD(object):
     """
@@ -401,7 +442,8 @@ class RDD(object):
         """
         Return a list that contains all of the elements in this RDD.
         """
-        bytesInJava = self._jrdd.collect().iterator()
+        with _JavaStackTrace(self.context) as st:
+          bytesInJava = self._jrdd.collect().iterator()
         return list(self._collect_iterator_through_file(bytesInJava))
 
     def _collect_iterator_through_file(self, iterator):
@@ -582,13 +624,14 @@ class RDD(object):
         # TODO(shivaram): Similar to the scala implementation, update the take 
         # method to scan multiple splits based on an estimate of how many elements 
         # we have per-split.
-        for partition in range(mapped._jrdd.splits().size()):
-            partitionsToTake = self.ctx._gateway.new_array(self.ctx._jvm.int, 1)
-            partitionsToTake[0] = partition
-            iterator = mapped._jrdd.collectPartitions(partitionsToTake)[0].iterator()
-            items.extend(mapped._collect_iterator_through_file(iterator))
-            if len(items) >= num:
-                break
+        with _JavaStackTrace(self.context) as st:
+            for partition in range(mapped._jrdd.splits().size()):
+                partitionsToTake = self.ctx._gateway.new_array(self.ctx._jvm.int, 1)
+                partitionsToTake[0] = partition
+                iterator = mapped._jrdd.collectPartitions(partitionsToTake)[0].iterator()
+                items.extend(mapped._collect_iterator_through_file(iterator))
+                if len(items) >= num:
+                    break
         return items[:num]
 
     def first(self):
@@ -765,9 +808,10 @@ class RDD(object):
                 yield outputSerializer.dumps(items)
         keyed = PipelinedRDD(self, add_shuffle_key)
         keyed._bypass_serializer = True
-        pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
-        partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
-                                                     id(partitionFunc))
+        with _JavaStackTrace(self.context) as st:
+            pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
+            partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
+                                                          id(partitionFunc))
         jrdd = pairRDD.partitionBy(partitioner).values()
         rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer))
         # This is required so that id(partitionFunc) remains unique, even if
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index 811fa6f018b23f3c9883bd2a770f03c044786850..2a500ab919beaf3b8fcc4c50e47de610d3af7b0b 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -308,4 +308,4 @@ def write_int(value, stream):
 
 def write_with_length(obj, stream):
     write_int(len(obj), stream)
-    stream.write(obj)
\ No newline at end of file
+    stream.write(obj)
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index a47595909029754590cf2a4fac27e4d3b0d6b17a..1602227a273e78d325933dace3d6b5c1e124615c 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -42,12 +42,12 @@ print "Using Python version %s (%s, %s)" % (
     platform.python_version(),
     platform.python_build()[0],
     platform.python_build()[1])
-print "Spark context avaiable as sc."
+print "Spark context available as sc."
 
 if add_files != None:
     print "Adding files: [%s]" % ", ".join(add_files)
 
-# The ./pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
+# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
 # which allows us to execute the user's PYTHONSTARTUP file:
 _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP')
 if _pythonstartup and os.path.isfile(_pythonstartup):
diff --git a/python/run-tests b/python/run-tests
index d4dad672d284299a0a1fc116b24f7833eefe1b7b..feba97cee0a8e926d1f08009316e2bbf2a340d5a 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -29,12 +29,13 @@ FAILED=0
 rm -f unit-tests.log
 
 function run_test() {
-    $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log
+    SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log
     FAILED=$((PIPESTATUS[0]||$FAILED))
 }
 
 run_test "pyspark/rdd.py"
 run_test "pyspark/context.py"
+run_test "pyspark/conf.py"
 run_test "-m doctest pyspark/broadcast.py"
 run_test "-m doctest pyspark/accumulators.py"
 run_test "-m doctest pyspark/serializers.py"
diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run
index 47bb654baf44784565dc6130e0f0b534ff24c905..3a6f22f41fca5406a615f57b9d1c96187ce6d928 100755
--- a/repl-bin/src/deb/bin/run
+++ b/repl-bin/src/deb/bin/run
@@ -48,8 +48,7 @@ fi
 export JAVA_OPTS
 
 # Build up classpath
-CLASSPATH="$SPARK_CLASSPATH"
-CLASSPATH+=":$FWDIR/conf"
+CLASSPATH=":$FWDIR/conf"
 for jar in `find $FWDIR -name '*jar'`; do
   CLASSPATH+=":$jar"
 done
diff --git a/repl/pom.xml b/repl/pom.xml
index b0e7877bbbac4a11b9af25ac0ca0ba6a1426d4aa..2dfe7ac900b831854b087e36f21db4d4cdc74f78 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -127,7 +127,6 @@
           <environmentVariables>
             <SPARK_HOME>${basedir}/..</SPARK_HOME>
             <SPARK_TESTING>1</SPARK_TESTING>
-            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
           </environmentVariables>
         </configuration>
       </plugin>
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
new file mode 100644
index 0000000000000000000000000000000000000000..05816941b54b3f9d789fcdcf0c9cd9b57328919b
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.repl
+
+import scala.tools.nsc.{Settings, CompilerCommand}
+import scala.Predef._
+
+/**
+ * Command class enabling Spark-specific command line options (provided by
+ * <i>org.apache.spark.repl.SparkRunnerSettings</i>).
+ */
+class SparkCommandLine(args: List[String], override val settings: Settings)
+    extends CompilerCommand(args, settings) {
+
+  def this(args: List[String], error: String => Unit) {
+    this(args, new SparkRunnerSettings(error))
+  }
+
+  def this(args: List[String]) {
+    this(args, str => Console.println("Error: " + str))
+  }
+}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 523fd1222dd71dff1770f1488c21ceb73617b9f6..87d94d51be199a0c9e6b27b856695715a9096d65 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -35,6 +35,7 @@ import java.lang.{Class => jClass}
 import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse}
 
 import org.apache.spark.Logging
+import org.apache.spark.SparkConf
 import org.apache.spark.SparkContext
 
 /** The Scala interactive shell.  It provides a read-eval-print loop
@@ -834,7 +835,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
 
   // runs :load `file` on any files passed via -i
   def loadFiles(settings: Settings) = settings match {
-    case settings: GenericRunnerSettings =>
+    case settings: SparkRunnerSettings =>
       for (filename <- settings.loadfiles.value) {
         val cmd = ":load " + filename
         command(cmd)
@@ -901,7 +902,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     addThunk(printWelcome())
     addThunk(initializeSpark())
 
-    loadFiles(settings)
     // it is broken on startup; go ahead and exit
     if (intp.reporter.hasErrors)
       return false
@@ -921,6 +921,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     }
     // printWelcome()
 
+    loadFiles(settings)
+
     try loop()
     catch AbstractOrMissingHandler()
     finally closeInterpreter()
@@ -929,10 +931,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
   }
 
   def createSparkContext(): SparkContext = {
-    val uri = System.getenv("SPARK_EXECUTOR_URI")
-    if (uri != null) {
-      System.setProperty("spark.executor.uri", uri)
-    }
+    val execUri = System.getenv("SPARK_EXECUTOR_URI")
     val master = this.master match {
       case Some(m) => m
       case None => {
@@ -941,14 +940,25 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     }
     val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath)
-    sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars)
+    val conf = new SparkConf()
+      .setMaster(master)
+      .setAppName("Spark shell")
+      .setJars(jars)
+      .set("spark.repl.class.uri", intp.classServer.uri)
+    if (execUri != null) {
+      conf.set("spark.executor.uri", execUri)
+    }
+    if (System.getenv("SPARK_HOME") != null) {
+      conf.setSparkHome(System.getenv("SPARK_HOME"))
+    }
+    sparkContext = new SparkContext(conf)
     echo("Created spark context..")
     sparkContext
   }
 
   /** process command-line arguments and do as they request */
   def process(args: Array[String]): Boolean = {
-    val command = new CommandLine(args.toList, echo)
+    val command = new SparkCommandLine(args.toList, msg => echo(msg))
     def neededHelp(): String =
       (if (command.settings.help.value) command.usageMsg + "\n" else "") +
       (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "")
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index e1455ef8a110a14317a03aab916254e41b4f8511..59fdb0b37a7663de7469767cb791322c0a31bbae 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -34,10 +34,8 @@ import scala.tools.reflect.StdRuntimeTags._
 import scala.util.control.ControlThrowable
 import util.stackTraceString
 
-import org.apache.spark.HttpServer
+import org.apache.spark.{HttpServer, SparkConf, Logging}
 import org.apache.spark.util.Utils
-import org.apache.spark.SparkEnv
-import org.apache.spark.Logging
 
 // /** directory to save .class files to */
 // private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) {
@@ -91,7 +89,7 @@ import org.apache.spark.Logging
       /** Local directory to save .class files too */
       val outputDir = {
         val tmp = System.getProperty("java.io.tmpdir")
-        val rootDir = System.getProperty("spark.repl.classdir", tmp)
+        val rootDir = new SparkConf().get("spark.repl.classdir",  tmp)
         Utils.createTempDir(rootDir)
       }
       if (SPARK_DEBUG_REPL) {
@@ -112,7 +110,6 @@ import org.apache.spark.Logging
         // Start the classServer and store its URI in a spark system property
     // (which will be passed to executors so that they can connect to it)
       classServer.start()
-      System.setProperty("spark.repl.class.uri", classServer.uri)
       if (SPARK_DEBUG_REPL) {
         echo("Class server started, URI = " + classServer.uri)
       }
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7fd5fbb42468cd31af41b9545a5412ce8789644e
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.repl
+
+import scala.tools.nsc.Settings
+
+/**
+ * <i>scala.tools.nsc.Settings</i> implementation adding Spark-specific REPL
+ * command line options.
+ */
+class SparkRunnerSettings(error: String => Unit) extends Settings(error){
+
+  val loadfiles = MultiStringSetting(
+      "-i",
+      "file",
+      "load a file (assumes the code is given interactively)")
+}
diff --git a/bin/slaves.sh b/sbin/slaves.sh
similarity index 97%
rename from bin/slaves.sh
rename to sbin/slaves.sh
index c367c2fd8ed9ae152607e54eb51b162788521613..a5bc2183d87f3a1a14233bf488e25feef516cbb4 100755
--- a/bin/slaves.sh
+++ b/sbin/slaves.sh
@@ -36,10 +36,10 @@ if [ $# -le 0 ]; then
   exit 1
 fi
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # If the slaves file is specified in the command line,
 # then it takes precedence over the definition in
diff --git a/bin/spark-config.sh b/sbin/spark-config.sh
similarity index 100%
rename from bin/spark-config.sh
rename to sbin/spark-config.sh
diff --git a/bin/spark-daemon.sh b/sbin/spark-daemon.sh
similarity index 96%
rename from bin/spark-daemon.sh
rename to sbin/spark-daemon.sh
index a0c0d44b58d9991f2d94ed16c41269baabe15900..2be2b3d7c09339087fb553a7b4e244f70b392def 100755
--- a/bin/spark-daemon.sh
+++ b/sbin/spark-daemon.sh
@@ -37,10 +37,10 @@ if [ $# -le 1 ]; then
   exit 1
 fi
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # get arguments
 
@@ -147,7 +147,7 @@ case $startStop in
     spark_rotate_log "$log"
     echo starting $command, logging to $log
     cd "$SPARK_PREFIX"
-    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
     newpid=$!
     echo $newpid > $pid
     sleep 2
diff --git a/bin/spark-daemons.sh b/sbin/spark-daemons.sh
similarity index 88%
rename from bin/spark-daemons.sh
rename to sbin/spark-daemons.sh
index 64286cb2da4f5acee66db6dd19ed442190d6466c..5d9f2bb51cae0547b053967fd5521a6ecb53ca04 100755
--- a/bin/spark-daemons.sh
+++ b/sbin/spark-daemons.sh
@@ -27,9 +27,9 @@ if [ $# -le 1 ]; then
   exit 1
 fi
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
-exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/spark-daemon.sh" "$@"
+exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/spark-daemon.sh" "$@"
diff --git a/spark-executor b/sbin/spark-executor
similarity index 88%
rename from spark-executor
rename to sbin/spark-executor
index 2c07c5484338fc9acddf4a4c74130566efba5176..de5bfab563125689b7c5c5a2e1b9f9d68f4b4aa1 100755
--- a/spark-executor
+++ b/sbin/spark-executor
@@ -17,6 +17,7 @@
 # limitations under the License.
 #
 
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
 echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend
+exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend
diff --git a/bin/start-all.sh b/sbin/start-all.sh
similarity index 89%
rename from bin/start-all.sh
rename to sbin/start-all.sh
index 0182f1ab240b744ec097c74dd1065029188c7f99..2daf49db359df4d62c9b3ddee2d9787523801345 100755
--- a/bin/start-all.sh
+++ b/sbin/start-all.sh
@@ -21,14 +21,14 @@
 # Starts the master on this node.
 # Starts a worker on each node specified in conf/slaves
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
 # Load the Spark configuration
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # Start Master
-"$bin"/start-master.sh
+"$sbin"/start-master.sh
 
 # Start Workers
-"$bin"/start-slaves.sh
+"$sbin"/start-slaves.sh
diff --git a/bin/start-master.sh b/sbin/start-master.sh
similarity index 88%
rename from bin/start-master.sh
rename to sbin/start-master.sh
index 648c7ae75fe81ddafbd9394ee6f1f60f00f7c263..3dcf7cc3483a785c21fd3266b16426a40cbdbb54 100755
--- a/bin/start-master.sh
+++ b/sbin/start-master.sh
@@ -19,10 +19,10 @@
 
 # Starts the master on the machine this script is executed on.
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
   . "${SPARK_CONF_DIR}/spark-env.sh"
@@ -49,4 +49,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
diff --git a/bin/start-slave.sh b/sbin/start-slave.sh
similarity index 92%
rename from bin/start-slave.sh
rename to sbin/start-slave.sh
index 4eefa209443385881905ee0c7208ea56fa79e9e5..524be38c629681b8acf1df845feabe020e25ee5c 100755
--- a/bin/start-slave.sh
+++ b/sbin/start-slave.sh
@@ -20,8 +20,8 @@
 # Usage: start-slave.sh <worker#> <master-spark-URL>
 #   where <master-spark-URL> is like "spark://localhost:7077"
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
 # Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI
 if [ "$SPARK_PUBLIC_DNS" = "" ]; then
@@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
diff --git a/bin/start-slaves.sh b/sbin/start-slaves.sh
similarity index 78%
rename from bin/start-slaves.sh
rename to sbin/start-slaves.sh
index 00dc4888b2e95220b2ea8ec4bcc32810a7235bfc..fd5cdeb1e6788ab0e7550621781883b31dc8f3cd 100755
--- a/bin/start-slaves.sh
+++ b/sbin/start-slaves.sh
@@ -17,10 +17,10 @@
 # limitations under the License.
 #
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
   . "${SPARK_CONF_DIR}/spark-env.sh"
@@ -37,12 +37,12 @@ fi
 
 # Launch the slaves
 if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
-  exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
+  exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
 else
   if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then
     SPARK_WORKER_WEBUI_PORT=8081
   fi
   for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
-    "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" $(( $i + 1 ))  spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
+    "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 ))  spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
   done
 fi
diff --git a/bin/stop-all.sh b/sbin/stop-all.sh
similarity index 89%
rename from bin/stop-all.sh
rename to sbin/stop-all.sh
index b6c83a7ba4c7df08aafa58f7e7a772e0d98452c9..60b358d3745658dad80f3a71c58410253d5a7455 100755
--- a/bin/stop-all.sh
+++ b/sbin/stop-all.sh
@@ -21,12 +21,12 @@
 # Run this on the master nde
 
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
 # Load the Spark configuration
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # Stop the slaves, then the master
-"$bin"/stop-slaves.sh
-"$bin"/stop-master.sh
+"$sbin"/stop-slaves.sh
+"$sbin"/stop-master.sh
diff --git a/bin/stop-master.sh b/sbin/stop-master.sh
similarity index 86%
rename from bin/stop-master.sh
rename to sbin/stop-master.sh
index 310e33bedc057194d4b410f2335234f5e2f30613..2adabd426563c7f5bf4f67f73da10cc60e753fc2 100755
--- a/bin/stop-master.sh
+++ b/sbin/stop-master.sh
@@ -19,9 +19,9 @@
 
 # Starts the master on the machine this script is executed on.
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
-"$bin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
+"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
diff --git a/bin/stop-slaves.sh b/sbin/stop-slaves.sh
similarity index 81%
rename from bin/stop-slaves.sh
rename to sbin/stop-slaves.sh
index fcb8555d4e5706a6f4f2ec9670525437a3a2acae..eb803b4900347e833ea0be5d2b4ae11845988de9 100755
--- a/bin/stop-slaves.sh
+++ b/sbin/stop-slaves.sh
@@ -17,19 +17,19 @@
 # limitations under the License.
 #
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
   . "${SPARK_CONF_DIR}/spark-env.sh"
 fi
 
 if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
-  "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
+  "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
 else
   for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
-    "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
+    "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
   done
 fi
diff --git a/sbt/sbt b/sbt/sbt
index 5942280585ba6e94fe0e4b9d4978412a472444ef..7f47d90cf11bbcef036fd4c5d863fcf8fd4e2052 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -1,4 +1,4 @@
-#!/usr/bin/env bash
+#!/bin/bash
 
 #
 # Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,27 +17,45 @@
 # limitations under the License.
 #
 
-cygwin=false
-case "`uname`" in
-    CYGWIN*) cygwin=true;;
-esac
+# This script launches sbt for this project. If present it uses the system 
+# version of sbt. If there is no system version of sbt it attempts to download
+# sbt locally.
+SBT_VERSION=`awk -F "=" '/sbt\\.version/ {print $2}' ./project/build.properties`
+URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar
+URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar
+JAR=sbt/sbt-launch-${SBT_VERSION}.jar
 
-EXTRA_ARGS="-Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m"
-if [ "$MESOS_HOME" != "" ]; then
-  EXTRA_ARGS="$EXTRA_ARGS -Djava.library.path=$MESOS_HOME/lib/java"
-fi
-
-export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd)
-export SPARK_TESTING=1  # To put test classes on classpath
-
-SBT_JAR="$SPARK_HOME"/sbt/sbt-launch-*.jar
-if $cygwin; then
-    SBT_JAR=`cygpath -w $SBT_JAR`
-    export SPARK_HOME=`cygpath -w $SPARK_HOME`
-    EXTRA_ARGS="$EXTRA_ARGS -Djline.terminal=jline.UnixTerminal -Dsbt.cygwin=true"
-    stty -icanon min 1 -echo > /dev/null 2>&1
-    java $EXTRA_ARGS $SBT_OPTS -jar $SBT_JAR "$@"
-    stty icanon echo > /dev/null 2>&1
+printf "Checking for system sbt ["
+if hash sbt 2>/dev/null; then 
+  printf "FOUND]\n"
+  # Use System SBT
+  sbt "$@"
 else
-    java $EXTRA_ARGS $SBT_OPTS -jar $SBT_JAR "$@"
-fi
\ No newline at end of file
+  printf "NOT FOUND]\n"
+  # Download sbt or use already downloaded
+  if [ ! -d .sbtlib ]; then
+    mkdir .sbtlib
+  fi
+  if [ ! -f ${JAR} ]; then
+    # Download
+    printf "Attempting to fetch sbt\n"
+    if hash curl 2>/dev/null; then
+      curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR}
+    elif hash wget 2>/dev/null; then
+      wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR}
+    else
+      printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n"
+      exit -1
+    fi
+  fi
+  if [ ! -f ${JAR} ]; then
+    # We failed to download
+    printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n"
+    exit -1
+  fi
+  printf "Launching sbt from ${JAR}\n"
+  java \
+    -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \
+    -jar ${JAR} \
+    "$@"
+fi
diff --git a/sbt/sbt-launch-0.11.3-2.jar b/sbt/sbt-launch-0.11.3-2.jar
deleted file mode 100644
index 23e5c3f31149bbf2bddbf1ae8d1fd02aba7910ad..0000000000000000000000000000000000000000
Binary files a/sbt/sbt-launch-0.11.3-2.jar and /dev/null differ
diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd
deleted file mode 100644
index 681fe00f9210818221e94afcc11db5256ae54ac3..0000000000000000000000000000000000000000
--- a/sbt/sbt.cmd
+++ /dev/null
@@ -1,25 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-set EXTRA_ARGS=
-if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java
-
-set SPARK_HOME=%~dp0..
-
-java -Xmx1200M -XX:MaxPermSize=200m -XX:ReservedCodeCacheSize=256m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*"
diff --git a/streaming/pom.xml b/streaming/pom.xml
index e3b6fee9b297711c9a4de5279e41da14abc30e9b..459756912dbe5f2668cfb4e0f897fa6f0196eb25 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -60,59 +60,10 @@
       <artifactId>jackson-mapper-asl</artifactId>
       <version>1.9.11</version>
     </dependency>
-    <dependency>
-      <groupId>com.sksamuel.kafka</groupId>
-      <artifactId>kafka_${scala.binary.version}</artifactId>
-      <version>0.8.0-beta1</version>
-      <exclusions>
-        <exclusion>
-          <groupId>com.sun.jmx</groupId>
-          <artifactId>jmxri</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jdmk</groupId>
-          <artifactId>jmxtools</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>net.sf.jopt-simple</groupId>
-          <artifactId>jopt-simple</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.flume</groupId>
-      <artifactId>flume-ng-sdk</artifactId>
-      <version>1.2.0</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.xerial.snappy</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.twitter4j</groupId>
-      <artifactId>twitter4j-stream</artifactId>
-      <version>3.0.3</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
     <dependency>
       <groupId>org.scala-lang</groupId>
       <artifactId>scala-library</artifactId>
     </dependency>
-    <dependency>
-      <groupId>${akka.group}</groupId>
-      <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
-    </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
@@ -137,11 +88,6 @@
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.eclipse.paho</groupId>
-      <artifactId>mqtt-client</artifactId>
-       <version>0.4.0</version>
-    </dependency>
   </dependencies>
   <build>
     <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
@@ -151,6 +97,35 @@
         <groupId>org.scalatest</groupId>
         <artifactId>scalatest-maven-plugin</artifactId>
       </plugin>
+      
+      <!-- 
+           This plugin forces the generation of jar containing streaming test classes, 
+           so that the tests classes of external modules can use them. The two execution profiles
+           are necessary - first one for 'mvn package', second one for 'mvn compile'. Ideally, 
+           'mvn compile' should not compile test classes and therefore should not need this. 
+           However, an open Maven bug (http://jira.codehaus.org/browse/MNG-3559)
+           causes the compilation to fail if streaming test-jar is not generated. Hence, the 
+           second execution profile for 'mvn compile'.
+      -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.2</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>test-jar-on-compile</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index a32e4852c501bed8331f4a2dfd704627b2bef1f9..476ae70bc9447d3166265d6a267120764c0fb7ee 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -24,10 +24,9 @@ import java.util.concurrent.RejectedExecutionException
 import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hadoop.conf.Configuration
 
-import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.{SparkException, SparkConf, Logging}
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.util.MetadataCleaner
-import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.streaming.scheduler.JobGenerator
 
 
@@ -36,14 +35,14 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
   extends Logging with Serializable {
   val master = ssc.sc.master
   val framework = ssc.sc.appName
-  val sparkHome = ssc.sc.sparkHome
+  val sparkHome = ssc.sc.getSparkHome.getOrElse(null)
   val jars = ssc.sc.jars
-  val environment = ssc.sc.environment
   val graph = ssc.graph
   val checkpointDir = ssc.checkpointDir
   val checkpointDuration = ssc.checkpointDuration
   val pendingTimes = ssc.scheduler.getPendingTimes()
-  val delaySeconds = MetadataCleaner.getDelaySeconds
+  val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf)
+  val sparkConf = ssc.conf
 
   def validate() {
     assert(master != null, "Checkpoint.master is null")
@@ -59,11 +58,12 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
  * Convenience class to handle the writing of graph checkpoint to file
  */
 private[streaming]
-class CheckpointWriter(jobGenerator: JobGenerator, checkpointDir: String, hadoopConf: Configuration) extends Logging {
+class CheckpointWriter(jobGenerator: JobGenerator, conf: SparkConf, checkpointDir: String, hadoopConf: Configuration)
+  extends Logging {
   val file = new Path(checkpointDir, "graph")
   val MAX_ATTEMPTS = 3
   val executor = Executors.newFixedThreadPool(1)
-  val compressionCodec = CompressionCodec.createCodec()
+  val compressionCodec = CompressionCodec.createCodec(conf)
   // The file to which we actually write - and then "move" to file
   val writeFile = new Path(file.getParent, file.getName + ".next")
   // The file to which existing checkpoint is backed up (i.e. "moved")
@@ -162,7 +162,7 @@ object CheckpointReader extends Logging {
 
   private val graphFileNames = Seq("graph", "graph.bk")
   
-  def read(checkpointDir: String, hadoopConf: Configuration): Option[Checkpoint] = {
+  def read(checkpointDir: String, conf: SparkConf, hadoopConf: Configuration): Option[Checkpoint] = {
     val checkpointPath = new Path(checkpointDir)
     def fs = checkpointPath.getFileSystem(hadoopConf)
     val existingFiles = graphFileNames.map(new Path(checkpointPath, _)).filter(fs.exists)
@@ -175,7 +175,7 @@ object CheckpointReader extends Logging {
     }
     logInfo("Checkpoint files found: " + existingFiles.mkString(","))
 
-    val compressionCodec = CompressionCodec.createCodec()
+    val compressionCodec = CompressionCodec.createCodec(conf)
     existingFiles.foreach(file => {
       logInfo("Attempting to load checkpoint from file '" + file + "'")
       try {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
index 20074249d74a938ed8dc5fdaf5c7af3f6c0e7f54..2cdd45291d9a4c8359b7b0126166493e8cf71709 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -56,8 +56,6 @@ abstract class DStream[T: ClassTag] (
     @transient protected[streaming] var ssc: StreamingContext
   ) extends Serializable with Logging {
 
-  initLogging()
-
   // =======================================================================
   // Methods that should be implemented by subclasses of DStream
   // =======================================================================
@@ -208,7 +206,7 @@ abstract class DStream[T: ClassTag] (
         checkpointDuration + "). Please set it to higher than " + checkpointDuration + "."
     )
 
-    val metadataCleanerDelay = MetadataCleaner.getDelaySeconds
+    val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf)
     logInfo("metadataCleanupDelay = " + metadataCleanerDelay)
     assert(
       metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
index 1081d3c807ef06ed6f10bb88db78bc6995db154a..1589bc19a242e23187ef9512bb8728fed8b76537 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
@@ -52,7 +52,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T])
     logDebug("Current checkpoint files:\n" + checkpointFiles.toSeq.mkString("\n"))
 
     // Add the checkpoint files to the data to be serialized 
-    if (!currentCheckpointFiles.isEmpty) {
+    if (!checkpointFiles.isEmpty) {
       currentCheckpointFiles.clear()
       currentCheckpointFiles ++= checkpointFiles
       timeToCheckpointFile ++= currentCheckpointFiles
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
index 34919d315cccad1b5fbf5375410fc49db51f8725..0cca6d50e6820ea43d40b32f38ad37b0d868f5a1 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
@@ -24,7 +24,6 @@ import org.apache.spark.Logging
 import org.apache.spark.streaming.scheduler.Job
 
 final private[streaming] class DStreamGraph extends Serializable with Logging {
-  initLogging()
 
   private val inputStreams = new ArrayBuffer[InputDStream[_]]()
   private val outputStreams = new ArrayBuffer[DStream[_]]()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
index 80af96c060a14b994f8b8ae9873512df77dd94c6..56dbcbda23a7029bda765437ce235430136fa4df 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -108,8 +108,9 @@ extends Serializable {
     createCombiner: V => C,
     mergeValue: (C, V) => C,
     mergeCombiner: (C, C) => C,
-    partitioner: Partitioner) : DStream[(K, C)] = {
-    new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner)
+    partitioner: Partitioner,
+    mapSideCombine: Boolean = true): DStream[(K, C)] = {
+    new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine)
   }
 
   /**
@@ -173,7 +174,13 @@ extends Serializable {
       slideDuration: Duration,
       partitioner: Partitioner
     ): DStream[(K, Seq[V])] = {
-    self.window(windowDuration, slideDuration).groupByKey(partitioner)
+    val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v
+    val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v
+    val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2
+    self.groupByKey(partitioner)
+        .window(windowDuration, slideDuration)
+        .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner)
+        .asInstanceOf[DStream[(K, Seq[V])]]
   }
 
   /**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 30deba417e0ddd0c0c1243ef64a51487a16720dc..76be81603c76c08fdeff00873d6ace791be2fed5 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -17,23 +17,6 @@
 
 package org.apache.spark.streaming
 
-import akka.actor.Props
-import akka.actor.SupervisorStrategy
-import akka.zeromq.Subscribe
-import akka.util.ByteString
-
-import org.apache.spark.streaming.dstream._
-import org.apache.spark.streaming.scheduler.StreamingListener
-
-import org.apache.spark._
-import org.apache.spark.rdd.RDD
-import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy
-import org.apache.spark.streaming.receivers.ZeroMQReceiver
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.MetadataCleaner
-import org.apache.spark.streaming.receivers.ActorReceiver
-import org.apache.spark.streaming.scheduler.{JobScheduler, NetworkInputTracker}
-
 import scala.collection.mutable.Queue
 import scala.collection.Map
 import scala.reflect.ClassTag
@@ -41,14 +24,21 @@ import scala.reflect.ClassTag
 import java.io.InputStream
 import java.util.concurrent.atomic.AtomicInteger
 
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
 import org.apache.hadoop.io.LongWritable
 import org.apache.hadoop.io.Text
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
-import org.apache.hadoop.fs.{LocalFileSystem, Path}
+import org.apache.hadoop.fs.Path
 
-import twitter4j.Status
-import twitter4j.auth.Authorization
+import org.apache.spark._
+import org.apache.spark.rdd.RDD
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.MetadataCleaner
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.streaming.receivers._
+import org.apache.spark.streaming.scheduler._
 import org.apache.hadoop.conf.Configuration
 
 /**
@@ -64,18 +54,27 @@ class StreamingContext private (
 
   /**
    * Create a StreamingContext using an existing SparkContext.
-   * @param sparkContext Existing SparkContext
-   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param sparkContext existing SparkContext
+   * @param batchDuration the time interval at which streaming data will be divided into batches
    */
   def this(sparkContext: SparkContext, batchDuration: Duration) = {
     this(sparkContext, null, batchDuration)
   }
 
+  /**
+   * Create a StreamingContext by providing the configuration necessary for a new SparkContext.
+   * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters
+   * @param batchDuration the time interval at which streaming data will be divided into batches
+   */
+  def this(conf: SparkConf, batchDuration: Duration) = {
+    this(StreamingContext.createNewSparkContext(conf), null, batchDuration)
+  }
+
   /**
    * Create a StreamingContext by providing the details necessary for creating a new SparkContext.
-   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param appName A name for your job, to display on the cluster web UI
-   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param master cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+   * @param appName a name for your job, to display on the cluster web UI
+   * @param batchDuration the time interval at which streaming data will be divided into batches
    */
   def this(
       master: String,
@@ -95,20 +94,20 @@ class StreamingContext private (
    *                   HDFS compatible filesystems
    */
   def this(path: String, hadoopConf: Configuration = new Configuration) =
-    this(null, CheckpointReader.read(path, hadoopConf).get, null)
-
-  initLogging()
+    this(null, CheckpointReader.read(path, new SparkConf(), hadoopConf).get, null)
 
   if (sc_ == null && cp_ == null) {
     throw new Exception("Spark Streaming cannot be initialized with " +
       "both SparkContext and checkpoint as null")
   }
 
-  if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds < 0) {
-    MetadataCleaner.setDelaySeconds(cp_.delaySeconds)
+  private val conf_ = Option(sc_).map(_.conf).getOrElse(cp_.sparkConf)
+
+  if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds(conf_) < 0) {
+    MetadataCleaner.setDelaySeconds(conf_, cp_.delaySeconds)
   }
 
-  if (MetadataCleaner.getDelaySeconds < 0) {
+  if (MetadataCleaner.getDelaySeconds(conf_) < 0) {
     throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; "
       + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)")
   }
@@ -117,12 +116,14 @@ class StreamingContext private (
 
   protected[streaming] val sc: SparkContext = {
     if (isCheckpointPresent) {
-      new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment)
+      new SparkContext(cp_.sparkConf)
     } else {
       sc_
     }
   }
 
+  protected[streaming] val conf = sc.conf
+
   protected[streaming] val env = SparkEnv.get
 
   protected[streaming] val graph: DStreamGraph = {
@@ -230,74 +231,6 @@ class StreamingContext private (
     networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy))
   }
 
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
-   *                       and each frame has sequence of byte thus it needs the converter
-   *                       (which might be deserializer of bytes) to translate from sequence
-   *                       of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel RDD storage level. Defaults to memory-only.
-   */
-  def zeroMQStream[T: ClassTag](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
-      storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
-      supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
-    ): DStream[T] = {
-    actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
-        "ZeroMQReceiver", storageLevel, supervisorStrategy)
-  }
-
-  /**
-   * Create an input stream that pulls messages from a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel  Storage level to use for storing the received objects
-   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
-   */
-  def kafkaStream(
-      zkQuorum: String,
-      groupId: String,
-      topics: Map[String, Int],
-      storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
-    ): DStream[(String, String)] = {
-    val kafkaParams = Map[String, String](
-      "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
-      "zookeeper.connection.timeout.ms" -> "10000")
-    kafkaStream[String, String, kafka.serializer.StringDecoder, kafka.serializer.StringDecoder](
-      kafkaParams,
-      topics,
-      storageLevel)
-  }
-
-  /**
-   * Create an input stream that pulls messages from a Kafka Broker.
-   * @param kafkaParams Map of kafka configuration paramaters.
-   *                    See: http://kafka.apache.org/configuration.html
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def kafkaStream[
-    K: ClassTag,
-    V: ClassTag,
-    U <: kafka.serializer.Decoder[_]: Manifest,
-    T <: kafka.serializer.Decoder[_]: Manifest](
-      kafkaParams: Map[String, String],
-      topics: Map[String, Int],
-      storageLevel: StorageLevel
-    ): DStream[(K, V)] = {
-    val inputStream = new KafkaInputDStream[K, V, U, T](this, kafkaParams, topics, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
-
   /**
    * Create a input stream from TCP source hostname:port. Data is received using
    * a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited
@@ -336,22 +269,6 @@ class StreamingContext private (
     inputStream
   }
 
-  /**
-   * Create a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def flumeStream (
-      hostname: String,
-      port: Int,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[SparkFlumeEvent] = {
-    val inputStream = new FlumeInputDStream[SparkFlumeEvent](this, hostname, port, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
-
   /**
    * Create a input stream from network source hostname:port, where data is received
    * as serialized blocks (serialized using the Spark's serializer) that can be directly
@@ -426,24 +343,6 @@ class StreamingContext private (
     fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString)
   }
 
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
-   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
-   *        .consumerSecret, .accessToken and .accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Option[Authorization] = None,
-      filters: Seq[String] = Nil,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[Status] = {
-    val inputStream = new TwitterInputDStream(this, twitterAuth, filters, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
-
   /**
    * Create an input stream from a queue of RDDs. In each batch,
    * it will process either one or all of the RDDs returned by the queue.
@@ -477,21 +376,6 @@ class StreamingContext private (
     inputStream
   }
 
-/**
-   * Create an input stream that receives messages pushed by a mqtt publisher.
-   * @param brokerUrl Url of remote mqtt publisher
-   * @param topic topic name to subscribe to
-   * @param storageLevel RDD storage level. Defaults to memory-only.
-   */
-
-  def mqttStream(
-    brokerUrl: String,
-    topic: String,
-    storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2): DStream[String] = {
-    val inputStream = new MQTTInputDStream[String](this, brokerUrl, topic, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
   /**
    * Create a unified DStream from multiple DStreams of the same type and same slide duration.
    */
@@ -614,7 +498,7 @@ object StreamingContext extends Logging {
     ): StreamingContext = {
 
     try {
-      CheckpointReader.read(checkpointPath, hadoopConf) match {
+      CheckpointReader.read(checkpointPath,  new SparkConf(), hadoopConf) match {
         case Some(checkpoint) =>
           return new StreamingContext(null, checkpoint, null)
         case None =>
@@ -634,18 +518,40 @@ object StreamingContext extends Logging {
     }
   }
 
+  /**
+   * Find the JAR from which a given class was loaded, to make it easy for users to pass
+   * their JARs to StreamingContext.
+   */
+  def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls)
+
+
+  protected[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = {
+    // Set the default cleaner delay to an hour if not already set.
+    // This should be sufficient for even 1 second batch intervals.
+    if (MetadataCleaner.getDelaySeconds(conf) < 0) {
+      MetadataCleaner.setDelaySeconds(conf, 3600)
+    }
+    val sc = new SparkContext(conf)
+    sc
+  }
+
   protected[streaming] def createNewSparkContext(
       master: String,
       appName: String,
       sparkHome: String,
       jars: Seq[String],
-      environment: Map[String, String]): SparkContext = {
+      environment: Map[String, String]
+    ): SparkContext = {
+
+    val conf = SparkContext.updatedConf(
+      new SparkConf(), master, appName, sparkHome, jars, environment)
     // Set the default cleaner delay to an hour if not already set.
-    // This should be sufficient for even 1 second interval.
-    if (MetadataCleaner.getDelaySeconds < 0) {
-      MetadataCleaner.setDelaySeconds(3600)
+    // This should be sufficient for even 1 second batch intervals.
+    if (MetadataCleaner.getDelaySeconds(conf) < 0) {
+      MetadataCleaner.setDelaySeconds(conf, 3600)
     }
-    new SparkContext(master, appName, sparkHome, jars, environment)
+    val sc = new SparkContext(master, appName, sparkHome, jars, environment)
+    sc
   }
 
   protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index dfd6e27c3e9101dbe125c26b0e5a4b99469c4a27..6c3467d4056d721a90e0efc55e903862af12526e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -155,7 +155,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
 
   /**
    * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
-   * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more
+   * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
    * information.
    */
   def combineByKey[C](createCombiner: JFunction[V, C],
@@ -168,6 +168,22 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
     dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner)
   }
 
+  /**
+   * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
+   * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
+   * information.
+   */
+  def combineByKey[C](createCombiner: JFunction[V, C],
+      mergeValue: JFunction2[C, V, C],
+      mergeCombiners: JFunction2[C, C, C],
+      partitioner: Partitioner,
+      mapSideCombine: Boolean
+    ): JavaPairDStream[K, C] = {
+    implicit val cm: ClassTag[C] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
+    dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine)
+  }
+
   /**
    * Return a new DStream by applying `groupByKey` over a sliding window. This is similar to
    * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index f38d1453175bd3b32857a85c9b18bfb64ebc648e..d96e9ac7b7e3ab082b994650fd4b062d3e63974c 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -17,28 +17,23 @@
 
 package org.apache.spark.streaming.api.java
 
-import java.lang.{Integer => JInt}
-import java.io.InputStream
-import java.util.{Map => JMap, List => JList}
 
 import scala.collection.JavaConversions._
 import scala.reflect.ClassTag
 
-import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import twitter4j.Status
-import akka.actor.Props
-import akka.actor.SupervisorStrategy
-import akka.zeromq.Subscribe
-import akka.util.ByteString
+import java.io.InputStream
+import java.lang.{Integer => JInt}
+import java.util.{List => JList, Map => JMap}
 
-import twitter4j.auth.Authorization
+import akka.actor.{Props, SupervisorStrategy}
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
+import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
-import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD}
 import org.apache.spark.streaming._
-import org.apache.spark.streaming.dstream._
 import org.apache.spark.streaming.scheduler.StreamingListener
 import org.apache.hadoop.conf.Configuration
 
@@ -124,6 +119,14 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   def this(sparkContext: JavaSparkContext, batchDuration: Duration) =
     this(new StreamingContext(sparkContext.sc, batchDuration))
 
+  /**
+   * Creates a StreamingContext using an existing SparkContext.
+   * @param conf A Spark application configuration
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(conf: SparkConf, batchDuration: Duration) =
+    this(new StreamingContext(conf, batchDuration))
+
   /**
    * Re-creates a StreamingContext from a checkpoint file.
    * @param path Path to the directory that was specified as the checkpoint directory
@@ -140,81 +143,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   /** The underlying SparkContext */
   val sc: JavaSparkContext = new JavaSparkContext(ssc.sc)
 
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   */
-  def kafkaStream(
-    zkQuorum: String,
-    groupId: String,
-    topics: JMap[String, JInt])
-  : JavaPairDStream[String, String] = {
-    implicit val cmt: ClassTag[String] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
-      StorageLevel.MEMORY_ONLY_SER_2)
-
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel RDD storage level. Defaults to memory-only
-   *
-   */
-  def kafkaStream(
-    zkQuorum: String,
-    groupId: String,
-    topics: JMap[String, JInt],
-    storageLevel: StorageLevel)
-  : JavaPairDStream[String, String] = {
-    implicit val cmt: ClassTag[String] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
-      storageLevel)
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param keyTypeClass Key type of RDD
-   * @param valueTypeClass value type of RDD
-   * @param keyDecoderClass Type of kafka key decoder
-   * @param valueDecoderClass Type of kafka value decoder
-   * @param kafkaParams Map of kafka configuration paramaters.
-   *                    See: http://kafka.apache.org/configuration.html
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   * @param storageLevel RDD storage level. Defaults to memory-only
-   */
-  def kafkaStream[K, V, U <: kafka.serializer.Decoder[_], T <: kafka.serializer.Decoder[_]](
-    keyTypeClass: Class[K],
-    valueTypeClass: Class[V],
-    keyDecoderClass: Class[U],
-    valueDecoderClass: Class[T],
-    kafkaParams: JMap[String, String],
-    topics: JMap[String, JInt],
-    storageLevel: StorageLevel)
-  : JavaPairDStream[K, V] = {
-    implicit val keyCmt: ClassTag[K] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
-    implicit val valueCmt: ClassTag[V] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
-
-    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
-    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
-
-    ssc.kafkaStream[K, V, U, T](
-      kafkaParams.toMap,
-      Map(topics.mapValues(_.intValue()).toSeq: _*),
-      storageLevel)
-  }
-
   /**
    * Create a input stream from network source hostname:port. Data is received using
    * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
@@ -328,98 +256,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     ssc.fileStream[K, V, F](directory)
   }
 
-  /**
-   * Creates a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
-    JavaDStream[SparkFlumeEvent] = {
-    ssc.flumeStream(hostname, port, storageLevel)
-  }
-
-
-  /**
-   * Create a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   */
-  def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
-    ssc.flumeStream(hostname, port)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization object
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String],
-      storageLevel: StorageLevel
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth), filters, storageLevel)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * .consumerSecret, .accessToken and .accessTokenSecret to be set.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      filters: Array[String],
-      storageLevel: StorageLevel
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(None, filters, storageLevel)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String]
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth), filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * .consumerSecret, .accessToken and .accessTokenSecret to be set.
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(
-      filters: Array[String]
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(None, filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   */
-  def twitterStream(
-      twitterAuth: Authorization
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth))
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * .consumerSecret, .accessToken and .accessTokenSecret to be set.
-   */
-  def twitterStream(): JavaDStream[Status] = {
-    ssc.twitterStream()
-  }
-
   /**
    * Create an input stream with any arbitrary user implemented actor receiver.
    * @param props Props object defining creation of the actor
@@ -482,70 +318,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     ssc.actorStream[T](props, name)
   }
 
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def zeroMQStream[T](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
-      storageLevel: StorageLevel,
-      supervisorStrategy: SupervisorStrategy
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel RDD storage level. Defaults to memory-only.
-   */
-  def zeroMQStream[T](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
-      storageLevel: StorageLevel
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   */
-  def zeroMQStream[T](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
-  }
-
   /**
    * Registers an output stream that will be computed every interval
    */
@@ -716,7 +488,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   def stop() = ssc.stop()
 }
 
-
 /**
  * JavaStreamingContext object contains a number of static utility functions.
  */
@@ -787,6 +558,12 @@ object JavaStreamingContext {
     }, hadoopConf, createOnError)
     new JavaStreamingContext(ssc)
   }
+
+  /**
+   * Find the JAR from which a given class was loaded, to make it easy for users to pass
+   * their JARs to StreamingContext.
+   */
+  def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray
 }
 
 /**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
index 5add20871e3fd68b83d63d649af903f2d13f4f51..27d474c0a0459a3aa556ba8e7bea3bb967396dbc 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
@@ -88,8 +88,6 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe
  */
 abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging {
 
-  initLogging()
-
   lazy protected val env = SparkEnv.get
 
   lazy protected val actor = env.actorSystem.actorOf(
@@ -176,8 +174,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
   /** A helper actor that communicates with the NetworkInputTracker */
   private class NetworkReceiverActor extends Actor {
     logInfo("Attempting to register with tracker")
-    val ip = System.getProperty("spark.driver.host", "localhost")
-    val port = System.getProperty("spark.driver.port", "7077").toInt
+    val ip = env.conf.get("spark.driver.host", "localhost")
+    val port = env.conf.get("spark.driver.port", "7077").toInt
     val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
     val tracker = env.actorSystem.actorSelection(url)
     val timeout = 5.seconds
@@ -214,7 +212,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
     case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null)
 
     val clock = new SystemClock()
-    val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong
+    val blockInterval = env.conf.get("spark.streaming.blockInterval", "200").toLong
     val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer)
     val blockStorageLevel = storageLevel
     val blocksForPushing = new ArrayBlockingQueue[Block](1000)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
index e6e00220979981c2f5a5db69d8254e26add226dc..84e69f277b22e97d2ac8303bc5d784faa8b96b06 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
@@ -29,8 +29,9 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
     createCombiner: V => C,
     mergeValue: (C, V) => C,
     mergeCombiner: (C, C) => C,
-    partitioner: Partitioner
-  ) extends DStream [(K,C)] (parent.ssc) {
+    partitioner: Partitioner,
+    mapSideCombine: Boolean = true
+  ) extends DStream[(K,C)] (parent.ssc) {
 
   override def dependencies = List(parent)
 
@@ -38,8 +39,8 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
 
   override def compute(validTime: Time): Option[RDD[(K,C)]] = {
     parent.getOrCompute(validTime) match {
-      case Some(rdd) =>
-        Some(rdd.combineByKey[C](createCombiner, mergeValue, mergeCombiner, partitioner))
+      case Some(rdd) => Some(rdd.combineByKey[C](
+          createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine))
       case None => None
     }
   }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
index 73d959331a3c24974cfe8c42e809783cf56eed28..89c43ff935bb415183abe74e943ed953e4531406 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
@@ -17,10 +17,10 @@
 
 package org.apache.spark.streaming.dstream
 
-import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.rdd.{PartitionerAwareUnionRDD, RDD, UnionRDD}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.Duration
 
 import scala.reflect.ClassTag
 
@@ -51,6 +51,14 @@ class WindowedDStream[T: ClassTag](
 
   override def compute(validTime: Time): Option[RDD[T]] = {
     val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime)
-    Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
+    val rddsInWindow = parent.slice(currentWindow)
+    val windowRDD = if (rddsInWindow.flatMap(_.partitioner).distinct.length == 1) {
+      logDebug("Using partition aware union for windowing at " + validTime)
+      new PartitionerAwareUnionRDD(ssc.sc, rddsInWindow)
+    } else {
+      logDebug("Using normal union for windowing at " + validTime)
+      new UnionRDD(ssc.sc,rddsInWindow)
+    }
+    Some(windowRDD)
   }
 }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index 6fbe6da921792ed0e4158a2b0db43169981ee2f5..917b4c57f62f893c3a89bc764782414c5c3f454a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -37,8 +37,6 @@ private[scheduler] case class ClearCheckpointData(time: Time) extends JobGenerat
 private[streaming]
 class JobGenerator(jobScheduler: JobScheduler) extends Logging {
 
-  initLogging()
-
   val ssc = jobScheduler.ssc
   val graph = ssc.graph
   val eventProcessorActor = ssc.env.actorSystem.actorOf(Props(new Actor {
@@ -49,14 +47,14 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
     }
   }))
   val clock = {
-    val clockClass = System.getProperty(
+    val clockClass = ssc.sc.conf.get(
       "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
     Class.forName(clockClass).newInstance().asInstanceOf[Clock]
   }
   val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
     longTime => eventProcessorActor ! GenerateJobs(new Time(longTime)))
   lazy val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
-    new CheckpointWriter(this, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration)
+    new CheckpointWriter(this, ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration)
   } else {
     null
   }
@@ -68,7 +66,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
       startFirstTime()
     }
   }
-  
+
   def stop() {
     timer.stop()
     if (checkpointWriter != null) checkpointWriter.stop()
@@ -112,7 +110,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
     // or if the property is defined set it to that time
     if (clock.isInstanceOf[ManualClock]) {
       val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds
-      val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong
+      val jumpTime = ssc.sc.conf.get("spark.streaming.manualClock.jump", "0").toLong
       clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
     }
 
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
index 9511ccfbeddd6132b455882bddf65c8bd82bb5e6..9304fc1a9338d6b7336a607ccbd09346fcc6ce9b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@ -25,15 +25,13 @@ import org.apache.spark.streaming._
 
 /**
  * This class schedules jobs to be run on Spark. It uses the JobGenerator to generate
- * the jobs and runs them using a thread pool. Number of threads 
+ * the jobs and runs them using a thread pool. Number of threads
  */
 private[streaming]
 class JobScheduler(val ssc: StreamingContext) extends Logging {
 
-  initLogging()
-
   val jobSets = new ConcurrentHashMap[Time, JobSet]
-  val numConcurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt
+  val numConcurrentJobs = ssc.conf.get("spark.streaming.concurrentJobs", "1").toInt
   val executor = Executors.newFixedThreadPool(numConcurrentJobs)
   val generator = new JobGenerator(this)
   val listenerBus = new StreamingListenerBus()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
index abff55d77c829b5063e7c52ff51dfec4fac53c1a..75f724464348c107f32498d1457b78c9b8f97c61 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
@@ -160,7 +160,9 @@ class NetworkInputTracker(
       }
       // Run the dummy Spark job to ensure that all slaves have registered.
       // This avoids all the receivers to be scheduled on the same node.
-      ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
+      if (!ssc.sparkContext.isLocal) {
+        ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
+      }
 
       // Distribute the receivers and start them
       ssc.sparkContext.runJob(tempRDD, startReceiver)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
index 4a3993e3e3effd0b38b214d4c36142c9e73db3f3..1559f7a9f7ac00a917cc742bbc6f9287270e9e20 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
@@ -39,7 +39,6 @@ import org.apache.hadoop.conf.Configuration
 
 private[streaming]
 object MasterFailureTest extends Logging {
-  initLogging()
 
   @volatile var killed = false
   @volatile var killCount = 0
@@ -331,7 +330,6 @@ class TestOutputStream[T: ClassTag](
  */
 private[streaming]
 class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging {
-  initLogging()
 
   override def run() {
     try {
@@ -366,7 +364,6 @@ class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread
 private[streaming]
 class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
   extends Thread with Logging {
-  initLogging()
 
   override def run() {
     val localTestDir = Files.createTempDir()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index fc8655a0839d0ece4e4cd079e8befc726eb6e3c6..6585d494a6f86e04503a0c4c85283aa3031e6462 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -20,7 +20,7 @@ package org.apache.spark.streaming.util
 import java.nio.ByteBuffer
 import org.apache.spark.util.{RateLimitedOutputStream, IntParam}
 import java.net.ServerSocket
-import org.apache.spark.{Logging}
+import org.apache.spark.{SparkConf, Logging}
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
 import scala.io.Source
 import java.io.IOException
@@ -42,7 +42,7 @@ object RawTextSender extends Logging {
     // Repeat the input data multiple times to fill in a buffer
     val lines = Source.fromFile(file).getLines().toArray
     val bufferStream = new FastByteArrayOutputStream(blockSize + 1000)
-    val ser = new KryoSerializer().newInstance()
+    val ser = new KryoSerializer(new SparkConf()).newInstance()
     val serStream = ser.serializeStream(bufferStream)
     var i = 0
     while (bufferStream.position < blockSize) {
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index daeb99f5b7bf32f24f0db3079768a15876791e23..0d2145da9a1a687794ba79e77cea01b8ed030b9e 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -17,23 +17,19 @@
 
 package org.apache.spark.streaming;
 
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
-
-import kafka.serializer.StringDecoder;
+import scala.Tuple2;
 
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.spark.streaming.api.java.JavaDStreamLike;
 import org.junit.After;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
+import java.io.*;
+import java.util.*;
 
-import scala.Tuple2;
-import twitter4j.Status;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
 
+import org.apache.spark.SparkConf;
 import org.apache.spark.HashPartitioner;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
@@ -43,39 +39,11 @@ import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.streaming.api.java.JavaDStream;
 import org.apache.spark.streaming.api.java.JavaPairDStream;
 import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.apache.spark.streaming.dstream.SparkFlumeEvent;
-import org.apache.spark.streaming.JavaTestUtils;
-import org.apache.spark.streaming.JavaCheckpointTestUtils;
-
-import java.io.*;
-import java.util.*;
-
-import akka.actor.Props;
-import akka.zeromq.Subscribe;
-
 
 // The test suite itself is Serializable so that anonymous Function implementations can be
 // serialized, as an alternative to converting these anonymous classes to static inner classes;
 // see http://stackoverflow.com/questions/758570/.
-public class JavaAPISuite implements Serializable {
-  private transient JavaStreamingContext ssc;
-
-  @Before
-  public void setUp() {
-      System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
-      ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
-    ssc.checkpoint("checkpoint");
-  }
-
-  @After
-  public void tearDown() {
-    ssc.stop();
-    ssc = null;
-
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.driver.port");
-  }
-
+public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable {
   @Test
   public void testCount() {
     List<List<Integer>> inputData = Arrays.asList(
@@ -101,7 +69,7 @@ public class JavaAPISuite implements Serializable {
         Arrays.asList("hello", "world"),
         Arrays.asList("goodnight", "moon"));
 
-   List<List<Integer>> expected = Arrays.asList(
+    List<List<Integer>> expected = Arrays.asList(
         Arrays.asList(5,5),
         Arrays.asList(9,4));
 
@@ -1596,26 +1564,6 @@ public class JavaAPISuite implements Serializable {
   // Input stream tests. These mostly just test that we can instantiate a given InputStream with
   // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the
   // InputStream functionality is deferred to the existing Scala tests.
-  @Test
-  public void testKafkaStream() {
-    HashMap<String, Integer> topics = Maps.newHashMap();
-    JavaPairDStream<String, String> test1 = ssc.kafkaStream("localhost:12345", "group", topics);
-    JavaPairDStream<String, String> test2 = ssc.kafkaStream("localhost:12345", "group", topics,
-      StorageLevel.MEMORY_AND_DISK());
-
-    HashMap<String, String> kafkaParams = Maps.newHashMap();
-    kafkaParams.put("zookeeper.connect","localhost:12345");
-    kafkaParams.put("group.id","consumer-group");
-    JavaPairDStream<String, String> test3 = ssc.kafkaStream(
-      String.class,
-      String.class,
-      StringDecoder.class,
-      StringDecoder.class,
-      kafkaParams,
-      topics,
-      StorageLevel.MEMORY_AND_DISK());
-  }
-
   @Test
   public void testSocketTextStream() {
     JavaDStream<String> test = ssc.socketTextStream("localhost", 12345);
@@ -1654,36 +1602,4 @@ public class JavaAPISuite implements Serializable {
   public void testRawSocketStream() {
     JavaDStream<String> test = ssc.rawSocketStream("localhost", 12345);
   }
-
-  @Test
-  public void testFlumeStream() {
-    JavaDStream<SparkFlumeEvent> test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
-  }
-
-  @Test
-  public void testFileStream() {
-    JavaPairDStream<String, String> foo =
-      ssc.<String, String, SequenceFileInputFormat<String,String>>fileStream("/tmp/foo");
-  }
-
-  @Test
-  public void testTwitterStream() {
-    String[] filters = new String[] { "good", "bad", "ugly" };
-    JavaDStream<Status> test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY());
-  }
-
-  @Test
-  public void testActorStream() {
-    JavaDStream<String> test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY());
-  }
-
-  @Test
-  public void testZeroMQStream() {
-    JavaDStream<String> test = ssc.zeroMQStream("url", (Subscribe) null, new Function<byte[][], Iterable<String>>() {
-      @Override
-      public Iterable<String> call(byte[][] b) throws Exception {
-        return null;
-      }
-    });
-  }
 }
diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
new file mode 100644
index 0000000000000000000000000000000000000000..34bee568859f9736f66e2428305f3b07e1fa5457
--- /dev/null
+++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.streaming;
+
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class LocalJavaStreamingContext {
+
+    protected transient JavaStreamingContext ssc;
+
+    @Before
+    public void setUp() {
+        System.clearProperty("spark.driver.port");
+        System.clearProperty("spark.hostPort");
+        System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+        ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+        ssc.checkpoint("checkpoint");
+    }
+
+    @After
+    public void tearDown() {
+        ssc.stop();
+        ssc = null;
+
+        // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+        System.clearProperty("spark.driver.port");
+        System.clearProperty("spark.hostPort");
+    }
+}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index b35ca00b53d71cbbfbcb513cdf31e77d87205ff2..ee6b433d1f1fa2d05a33991cad2d7cdf0b81a7c5 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -23,9 +23,9 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext._
 
 import util.ManualClock
+import org.apache.spark.{SparkContext, SparkConf}
 
 class BasicOperationsSuite extends TestSuiteBase {
-
   test("map") {
     val input = Seq(1 to 4, 5 to 8, 9 to 12)
     testOperation(
@@ -375,7 +375,11 @@ class BasicOperationsSuite extends TestSuiteBase {
   }
 
   test("slice") {
-    val ssc = new StreamingContext("local[2]", "BasicOperationSuite", Seconds(1))
+    val conf2 = new SparkConf()
+      .setMaster("local[2]")
+      .setAppName("BasicOperationsSuite")
+      .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+    val ssc = new StreamingContext(new SparkContext(conf2), Seconds(1))
     val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
     val stream = new TestInputStream[Int](ssc, input, 2)
     ssc.registerInputStream(stream)
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index 53bc24ff7ac22ae6b51481c0704e3938ae941b0e..6499de98c925e3426ed942114565aef7c161b209 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -57,7 +57,7 @@ class CheckpointSuite extends TestSuiteBase {
 
     assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second")
 
-    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+    conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
 
     val stateStreamCheckpointInterval = Seconds(1)
     val fs = FileSystem.getLocal(new Configuration())
@@ -132,8 +132,9 @@ class CheckpointSuite extends TestSuiteBase {
     assert(!stateStream.generatedRDDs.isEmpty,
       "No restored RDDs in state stream after recovery from second failure")
 
-    // Adjust manual clock time as if it is being restarted after a delay
-    System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
+    // Adjust manual clock time as if it is being restarted after a delay; this is a hack because
+    // we modify the conf object, but it works for this one property
+    ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
     ssc.start()
     advanceTimeWithRealDelay(ssc, 4)
     ssc.stop()
@@ -196,10 +197,6 @@ class CheckpointSuite extends TestSuiteBase {
   // It also tests whether batches, whose processing was incomplete due to the
   // failure, are re-processed or not.
   test("recovery with file input stream") {
-    // Disable manual clock as FileInputDStream does not work with manual clock
-    val clockProperty = System.getProperty("spark.streaming.clock")
-    System.clearProperty("spark.streaming.clock")
-
     // Set up the streaming context and input streams
     val testDir = Files.createTempDir()
     var ssc = new StreamingContext(master, framework, Seconds(1))
@@ -296,10 +293,6 @@ class CheckpointSuite extends TestSuiteBase {
     )
     // To ensure that all the inputs were received correctly
     assert(expectedOutput.last === output.last)
-
-    // Enable manual clock back again for other tests
-    if (clockProperty != null)
-      System.setProperty("spark.streaming.clock", clockProperty)
   }
 
 
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index 5fa14ad7c489a19d5b2ae3cc6726cbcb43ed005f..a8e053278a2b2833c83298fb9fd325b8eb43e0f4 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -23,7 +23,7 @@ import akka.actor.IOManager
 import akka.actor.Props
 import akka.util.ByteString
 
-import org.apache.spark.streaming.dstream.{FileInputDStream, NetworkReceiver, SparkFlumeEvent}
+import org.apache.spark.streaming.dstream.{NetworkReceiver}
 import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket}
 import java.io.{File, BufferedWriter, OutputStreamWriter}
 import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue}
@@ -31,18 +31,11 @@ import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
 import util.ManualClock
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming.receivers.Receiver
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.Logging
 import scala.util.Random
 import org.apache.commons.io.FileUtils
 import org.scalatest.BeforeAndAfter
-import org.apache.flume.source.avro.AvroSourceProtocol
-import org.apache.flume.source.avro.AvroFlumeEvent
-import org.apache.flume.source.avro.Status
-import org.apache.avro.ipc.{specific, NettyTransceiver}
-import org.apache.avro.ipc.specific.SpecificRequestor
-import java.nio.ByteBuffer
 import collection.JavaConversions._
-import java.nio.charset.Charset
 import com.google.common.io.Files
 import java.util.concurrent.atomic.AtomicInteger
 
@@ -56,9 +49,9 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     testServer.start()
 
     // Set up the streaming context and input streams
-    val ssc = new StreamingContext(master, framework, batchDuration)
+    val ssc = new StreamingContext(conf, batchDuration)
     val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
-    val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String  ]]
+    val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
     val outputStream = new TestOutputStream(networkStream, outputBuffer)
     def output = outputBuffer.flatMap(x => x)
     ssc.registerOutputStream(outputStream)
@@ -99,62 +92,13 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
   }
 
 
-  test("flume input stream") {
-    // Set up the streaming context and input streams
-    val ssc = new StreamingContext(master, framework, batchDuration)
-    val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
-    val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
-      with SynchronizedBuffer[Seq[SparkFlumeEvent]]
-    val outputStream = new TestOutputStream(flumeStream, outputBuffer)
-    ssc.registerOutputStream(outputStream)
-    ssc.start()
-
-    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
-    val input = Seq(1, 2, 3, 4, 5)
-    Thread.sleep(1000)
-    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
-    val client = SpecificRequestor.getClient(
-      classOf[AvroSourceProtocol], transceiver)
-
-    for (i <- 0 until input.size) {
-      val event = new AvroFlumeEvent
-      event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
-      event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
-      client.append(event)
-      Thread.sleep(500)
-      clock.addToTime(batchDuration.milliseconds)
-    }
-
-    val startTime = System.currentTimeMillis()
-    while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
-      logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
-      Thread.sleep(100)
-    }
-    Thread.sleep(1000)
-    val timeTaken = System.currentTimeMillis() - startTime
-    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
-    logInfo("Stopping context")
-    ssc.stop()
-
-    val decoder = Charset.forName("UTF-8").newDecoder()
-
-    assert(outputBuffer.size === input.length)
-    for (i <- 0 until outputBuffer.size) {
-      assert(outputBuffer(i).size === 1)
-      val str = decoder.decode(outputBuffer(i).head.event.getBody)
-      assert(str.toString === input(i).toString)
-      assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
-    }
-  }
-
-
   test("file input stream") {
     // Disable manual clock as FileInputDStream does not work with manual clock
-    System.clearProperty("spark.streaming.clock")
+    conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
 
     // Set up the streaming context and input streams
     val testDir = Files.createTempDir()
-    val ssc = new StreamingContext(master, framework, batchDuration)
+    val ssc = new StreamingContext(conf, batchDuration)
     val fileStream = ssc.textFileStream(testDir.toString)
     val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
     def output = outputBuffer.flatMap(x => x)
@@ -195,7 +139,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     FileUtils.deleteDirectory(testDir)
 
     // Enable manual clock back again for other tests
-    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+    conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
   }
 
 
@@ -206,7 +150,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     testServer.start()
 
     // Set up the streaming context and input streams
-    val ssc = new StreamingContext(master, framework, batchDuration)
+    val ssc = new StreamingContext(conf, batchDuration)
     val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor",
       StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope
     val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
@@ -249,20 +193,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     }
   }
 
-  test("kafka input stream") {
-    val ssc = new StreamingContext(master, framework, batchDuration)
-    val topics = Map("my-topic" -> 1)
-    val test1 = ssc.kafkaStream("localhost:12345", "group", topics)
-    val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK)
-
-    // Test specifying decoder
-    val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
-    val test3 = ssc.kafkaStream[
-      String,
-      String,
-      kafka.serializer.StringDecoder,
-      kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK)
-  }
 
   test("multi-thread receiver") {
     // set up the test receiver
@@ -273,7 +203,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     MultiThreadTestReceiver.haveAllThreadsFinished = false
 
     // set up the network stream using the test receiver
-    val ssc = new StreamingContext(master, framework, batchDuration)
+    val ssc = new StreamingContext(conf, batchDuration)
     val networkStream = ssc.networkStream[Int](testReceiver)
     val countStream = networkStream.count
     val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]]
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index e969e91d13e9ad762a92e777b751e8d7abd12560..b20d02f99681e87879c31a59778a9fc24388b6ab 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -28,7 +28,7 @@ import java.io.{ObjectInputStream, IOException}
 
 import org.scalatest.{BeforeAndAfter, FunSuite}
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkContext, SparkConf, Logging}
 import org.apache.spark.rdd.RDD
 
 /**
@@ -133,20 +133,22 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
   // Whether to actually wait in real time before changing manual clock
   def actuallyWait = false
 
+  //// A SparkConf to use in tests. Can be modified before calling setupStreams to configure things.
+  val conf = new SparkConf()
+    .setMaster(master)
+    .setAppName(framework)
+    .set("spark.cleaner.ttl", "3600")
+
   // Default before function for any streaming test suite. Override this
   // if you want to add your stuff to "before" (i.e., don't call before { } )
   def beforeFunction() {
     if (useManualClock) {
-      System.setProperty(
-        "spark.streaming.clock",
-        "org.apache.spark.streaming.util.ManualClock"
-      )
+      logInfo("Using manual clock")
+      conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
     } else {
-      System.clearProperty("spark.streaming.clock")
+      logInfo("Using real clock")
+      conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
     }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.driver.port")
-    System.clearProperty("spark.hostPort")
   }
 
   // Default after function for any streaming test suite. Override this
@@ -169,9 +171,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
       operation: DStream[U] => DStream[V],
       numPartitions: Int = numInputPartitions
     ): StreamingContext = {
-
     // Create StreamingContext
-    val ssc = new StreamingContext(master, framework, batchDuration)
+    val ssc = new StreamingContext(conf, batchDuration)
     if (checkpointDir != null) {
       ssc.checkpoint(checkpointDir)
     }
@@ -195,9 +196,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
       input2: Seq[Seq[V]],
       operation: (DStream[U], DStream[V]) => DStream[W]
     ): StreamingContext = {
-
     // Create StreamingContext
-    val ssc = new StreamingContext(master, framework, batchDuration)
+    val ssc = new StreamingContext(conf, batchDuration)
     if (checkpointDir != null) {
       ssc.checkpoint(checkpointDir)
     }
@@ -273,7 +273,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
       val startTime = System.currentTimeMillis()
       while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
         logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput)
-        Thread.sleep(100)
+        Thread.sleep(10)
       }
       val timeTaken = System.currentTimeMillis() - startTime
 
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
index 6b4aaefcdf90f02ffaa26e5ed2faa6c5e1f565b0..c39abfc21b3ba219a08c6bf6e514db052dbcd7f0 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
@@ -18,7 +18,6 @@
 package org.apache.spark.streaming
 
 import org.apache.spark.streaming.StreamingContext._
-import collection.mutable.ArrayBuffer
 
 class WindowOperationsSuite extends TestSuiteBase {
 
@@ -225,9 +224,7 @@ class WindowOperationsSuite extends TestSuiteBase {
     val slideDuration = Seconds(1)
     val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
     val operation = (s: DStream[(String, Int)]) => {
-      s.groupByKeyAndWindow(windowDuration, slideDuration)
-       .map(x => (x._1, x._2.toSet))
-       .persist()
+      s.groupByKeyAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toSet))
     }
     testOperation(input, operation, expectedOutput, numBatches, true)
   }
diff --git a/yarn/README.md b/yarn/README.md
new file mode 100644
index 0000000000000000000000000000000000000000..65ee85447e04a7eca0843e18a3a4896476ddcb8c
--- /dev/null
+++ b/yarn/README.md
@@ -0,0 +1,12 @@
+# YARN DIRECTORY LAYOUT
+
+Hadoop Yarn related codes are organized in separate directories to minimize duplicated code.
+
+ * common : Common codes that do not depending on specific version of Hadoop.
+
+ * alpha / stable : Codes that involve specific version of Hadoop YARN API.
+
+  alpha represents 0.23 and 2.0.x
+  stable represents 2.2 and later, until the API changes again.
+
+alpha / stable will build together with common dir into a single jar
diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..8291e9e7a36ce8ab42d742bd3e3e13ffde99e7b8
--- /dev/null
+++ b/yarn/alpha/pom.xml
@@ -0,0 +1,32 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>yarn-parent_2.10</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-yarn-alpha_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project YARN Alpha API</name>
+
+</project>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
similarity index 88%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 240ed8b32ae93b7099b7d83adb2fcbf600a444b8..2bb11e54c549af037fc75a2dd92300e3588828b4 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -36,15 +36,18 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
 
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.util.Utils
 
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
+                        sparkConf: SparkConf) extends Logging {
 
-class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
-  
-  private var rpc: YarnRPC = YarnRPC.create(conf)
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
+
+  private val rpc: YarnRPC = YarnRPC.create(conf)
   private var resourceManager: AMRMProtocol = _
   private var appAttemptId: ApplicationAttemptId = _
   private var userThread: Thread = _
@@ -57,18 +60,23 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
     YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES)
   private var isLastAMRetry: Boolean = true
-  // default to numWorkers * 2, with minimum of 3
-  private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures",
-    math.max(args.numWorkers * 2, 3).toString()).toInt
+
+  // Default to numWorkers * 2, with minimum of 3
+  private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures",
+    math.max(args.numWorkers * 2, 3))
 
   def run() {
     // Setup the directories so things go to yarn approved directories rather
     // then user specified and /tmp.
     System.setProperty("spark.local.dir", getLocalDirs())
 
+    // set the web ui port to be ephemeral for yarn so we don't conflict with
+    // other spark processes running on the same box
+    System.setProperty("spark.ui.port", "0")
+
     // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using.
     ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
-    
+
     appAttemptId = getApplicationAttemptId()
     isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
     resourceManager = registerWithResourceManager()
@@ -92,24 +100,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     //  }
     //}
     // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
-    
+
     ApplicationMaster.register(this)
     // Start the user's JAR
     userThread = startUserClass()
-    
+
     // This a bit hacky, but we need to wait until the spark.driver.port property has
     // been set by the Thread executing the user class.
-    waitForSparkMaster()
-
     waitForSparkContextInitialized()
 
     // Do this after spark master is up and SparkContext is created so that we can register UI Url
     val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
-    
+
     // Allocate all containers
     allocateWorkers()
-    
-    // Wait for the user class to Finish     
+
+    // Wait for the user class to Finish
     userThread.join()
 
     System.exit(0)
@@ -122,14 +128,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
     val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
       .getOrElse(Option(System.getenv("LOCAL_DIRS"))
-        .getOrElse(""))
+      .getOrElse(""))
 
     if (localDirs.isEmpty()) {
       throw new Exception("Yarn Local dirs can't be empty")
     }
     localDirs
   }
-  
+
   private def getApplicationAttemptId(): ApplicationAttemptId = {
     val envs = System.getenv()
     val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV)
@@ -138,7 +144,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     logInfo("ApplicationAttemptId: " + appAttemptId)
     appAttemptId
   }
-  
+
   private def registerWithResourceManager(): AMRMProtocol = {
     val rmAddress = NetUtils.createSocketAddr(yarnConf.get(
       YarnConfiguration.RM_SCHEDULER_ADDRESS,
@@ -146,50 +152,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     logInfo("Connecting to ResourceManager at " + rmAddress)
     rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
   }
-  
+
   private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
     logInfo("Registering the ApplicationMaster")
     val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest])
       .asInstanceOf[RegisterApplicationMasterRequest]
     appMasterRequest.setApplicationAttemptId(appAttemptId)
     // Setting this to master host,port - so that the ApplicationReport at client has some
-    // sensible info. 
+    // sensible info.
     // Users can then monitor stderr/stdout on that node if required.
     appMasterRequest.setHost(Utils.localHostName())
     appMasterRequest.setRpcPort(0)
     appMasterRequest.setTrackingUrl(uiAddress)
     resourceManager.registerApplicationMaster(appMasterRequest)
   }
-  
-  private def waitForSparkMaster() {
-    logInfo("Waiting for spark driver to be reachable.")
-    var driverUp = false
-    var tries = 0
-    val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt
-    while(!driverUp && tries < numTries) {
-      val driverHost = System.getProperty("spark.driver.host")
-      val driverPort = System.getProperty("spark.driver.port")
-      try {
-        val socket = new Socket(driverHost, driverPort.toInt)
-        socket.close()
-        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
-        driverUp = true
-      } catch {
-        case e: Exception => {
-          logWarning("Failed to connect to driver at %s:%s, retrying ...".
-            format(driverHost, driverPort))
-          Thread.sleep(100)
-          tries = tries + 1
-        }
-      }
-    }
-  }
 
-  private def startUserClass(): Thread  = {
+  private def startUserClass(): Thread = {
     logInfo("Starting the user JAR in a separate Thread")
     val mainMethod = Class.forName(
       args.userClass,
-      false /* initialize */,
+      false /* initialize */ ,
       Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
     val t = new Thread {
       override def run() {
@@ -226,7 +208,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       ApplicationMaster.sparkContextRef.synchronized {
         var count = 0
         val waitTime = 10000L
-        val numTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt
+        val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10)
         while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) {
           logInfo("Waiting for spark context initialization ... " + count)
           count = count + 1
@@ -241,8 +223,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
             yarnConf,
             resourceManager,
             appAttemptId,
-            args, 
-            sparkContext.preferredNodeLocationData) 
+            args,
+            sparkContext.preferredNodeLocationData,
+            sparkContext.getConf)
         } else {
           logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".
             format(count * waitTime, numTries))
@@ -250,7 +233,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
             yarnConf,
             resourceManager,
             appAttemptId,
-            args)
+            args,
+            sparkContext.getConf)
         }
       }
     } finally {
@@ -294,7 +278,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
       // we want to be reasonably responsive without causing too many requests to RM.
       val schedulerInterval =
-        System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong
+        sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000)
 
       // must be <= timeoutInterval / 2.
       val interval = math.min(timeoutInterval / 2, schedulerInterval)
@@ -304,7 +288,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
 
   private def launchReporterThread(_sleepTime: Long): Thread = {
-    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+    val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
 
     val t = new Thread {
       override def run() {
@@ -342,11 +326,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     for (container <- containers) {
       logInfo("Launching shell command on a new container."
         + ", containerId=" + container.getId()
-        + ", containerNode=" + container.getNodeId().getHost() 
+        + ", containerNode=" + container.getNodeId().getHost()
         + ":" + container.getNodeId().getPort()
         + ", containerNodeURI=" + container.getNodeHttpAddress()
         + ", containerState" + container.getState()
-        + ", containerResourceMemory"  
+        + ", containerResourceMemory"
         + container.getResource().getMemory())
     }
   }
@@ -372,12 +356,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
 
   /**
-   * Clean up the staging directory. 
+   * Clean up the staging directory.
    */
-  private def cleanupStagingDir() { 
+  private def cleanupStagingDir() {
     var stagingDirPath: Path = null
     try {
-      val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+      val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean
       if (!preserveFiles) {
         stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
         if (stagingDirPath == null) {
@@ -393,7 +377,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
   }
 
-  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM.
   class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
 
     def run() {
@@ -403,6 +387,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
     }
   }
+
 }
 
 object ApplicationMaster {
@@ -412,6 +397,7 @@ object ApplicationMaster {
   // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
   // optimal as more containers are available. Might need to handle this better.
   private val ALLOCATOR_LOOP_WAIT_COUNT = 30
+
   def incrementAllocatorLoop(by: Int) {
     val count = yarnAllocatorLoop.getAndAdd(by)
     if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
@@ -446,19 +432,20 @@ object ApplicationMaster {
     // Note that this will unfortunately not properly clean up the staging files because it gets
     // called too late, after the filesystem is already shutdown.
     if (modified) {
-      Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
+      Runtime.getRuntime().addShutdownHook(new Thread with Logging {
         // This is not only logs, but also ensures that log system is initialized for this instance
         // when we are actually 'run'-ing.
         logInfo("Adding shutdown hook for context " + sc)
-        override def run() { 
-          logInfo("Invoking sc stop from shutdown hook") 
-          sc.stop() 
+
+        override def run() {
+          logInfo("Invoking sc stop from shutdown hook")
+          sc.stop()
           // Best case ...
           for (master <- applicationMasters) {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
-        } 
-      } )
+        }
+      })
     }
 
     // Wait for initialization to complete and atleast 'some' nodes can get allocated.
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
similarity index 91%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 79dd03806523bcea066fcafd05160fe732032d22..6abb4d501779fd3193c3050d9eb1991576db0178 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -40,14 +40,18 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, Records}
 
-import org.apache.spark.Logging 
+import org.apache.spark.{Logging, SparkConf}
 import org.apache.spark.util.Utils
 import org.apache.spark.deploy.SparkHadoopUtil
 
 
-class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
+class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
+  extends YarnClientImpl with Logging {
 
-  def this(args: ClientArguments) = this(new Configuration(), args)
+  def this(args: ClientArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ClientArguments) = this(args, new SparkConf())
 
   var rpc: YarnRPC = YarnRPC.create(conf)
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
@@ -59,7 +63,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short)
 
   // App files are world-wide readable and owner writable -> rw-r--r--
-  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) 
+  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short)
 
   // for client user who want to monitor app status by itself.
   def runApp() = {
@@ -103,7 +107,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
       (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " +
         "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD)
-    ).foreach { case(cond, errStr) => 
+    ).foreach { case(cond, errStr) =>
       if (cond) {
         logError(errStr)
         args.printUsageAndExit(1)
@@ -121,7 +125,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       clusterMetrics.getNumNodeManagers)
 
     val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
-    logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s,
+    logInfo( """Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s,
       queueApplicationCount = %s, queueChildQueueCount = %s""".format(
         queueInfo.getQueueName,
         queueInfo.getCurrentCapacity,
@@ -130,7 +134,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         queueInfo.getChildQueues.size))
   }
 
-  def verifyClusterResources(app: GetNewApplicationResponse) = { 
+  def verifyClusterResources(app: GetNewApplicationResponse) = {
     val maxMem = app.getMaximumResourceCapability().getMemory()
     logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
 
@@ -141,12 +145,12 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     }
     val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
     if (amMem > maxMem) {
-      logError("AM size is to large to run on this cluster "  + amMem)
+      logError("AM size is to large to run on this cluster " + amMem)
       System.exit(1)
     }
 
     // We could add checks to make sure the entire cluster has enough resources but that involves
-    // getting all the node reports and computing ourselves 
+    // getting all the node reports and computing ourselves
   }
 
   def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
@@ -207,7 +211,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
       fs.setReplication(newPath, replication)
       if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
-    } 
+    }
     // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
     // version shows the specific version in the distributed cache configuration
     val qualPath = fs.makeQualified(newPath)
@@ -230,7 +234,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       }
     }
     val dst = new Path(fs.getHomeDirectory(), appStagingDir)
-    val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
+    val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort
 
     if (UserGroupInformation.isSecurityEnabled()) {
       val dstFs = dst.getFileSystem(conf)
@@ -241,7 +245,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
 
-    Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, 
+    Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
       Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF"))
     .foreach { case(destName, _localPath) =>
       val localPath: String = if (_localPath != null) _localPath.trim() else ""
@@ -253,7 +257,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         }
         val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
         val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           destName, statCache)
       }
     }
@@ -265,7 +269,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache, true)
       }
     }
@@ -277,7 +281,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache)
       }
     }
@@ -289,7 +293,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
           linkname, statCache)
       }
     }
@@ -299,14 +303,14 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   }
 
   def setupLaunchEnv(
-      localResources: HashMap[String, LocalResource], 
+      localResources: HashMap[String, LocalResource],
       stagingDir: String): HashMap[String, String] = {
     logInfo("Setting up the launch environment")
     val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
 
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_STAGING_DIR") = stagingDir
 
@@ -326,7 +330,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     val prefix = " --args "
     val args = clientArgs.userArgs
     val retval = new StringBuilder()
-    for (arg <- args){
+    for (arg <- args) {
       retval.append(prefix).append(" '").append(arg).append("' ")
     }
     retval.toString
@@ -354,7 +358,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     // Add Xmx for am memory
     JAVA_OPTS += "-Xmx" + amMemory + "m "
 
-    JAVA_OPTS += " -Djava.io.tmpdir=" + 
+    JAVA_OPTS += " -Djava.io.tmpdir=" +
       new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
 
     // Commenting it out for now - so that people can refer to the properties if required. Remove
@@ -387,11 +391,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
     }
 
-    val commands = List[String](javaCommand + 
+    val commands = List[String](javaCommand +
       " -server " +
       JAVA_OPTS +
       " " + args.amClass +
-      " --class " + args.userClass + 
+      " --class " + args.userClass +
       " --jar " + args.userJar +
       userArgsToString(args) +
       " --worker-memory " + args.workerMemory +
@@ -421,9 +425,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     super.submitApplication(appContext)
   }
 
-  def monitorApplication(appId: ApplicationId): Boolean = {  
+  def monitorApplication(appId: ApplicationId): Boolean = {
+    val interval = sparkConf.getLong("spark.yarn.report.interval", 1000)
+
     while (true) {
-      Thread.sleep(1000)
+      Thread.sleep(interval)
       val report = super.getApplicationReport(appId)
 
       logInfo("Application report from ASM: \n" +
@@ -443,7 +449,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
       val state = report.getYarnApplicationState()
       val dsStatus = report.getFinalApplicationStatus()
-      if (state == YarnApplicationState.FINISHED || 
+      if (state == YarnApplicationState.FINISHED ||
         state == YarnApplicationState.FAILED ||
         state == YarnApplicationState.KILLED) {
         return true
@@ -463,9 +469,10 @@ object Client {
     // Note that anything with SPARK prefix gets propagated to all (remote) processes
     System.setProperty("SPARK_YARN_MODE", "true")
 
-    val args = new ClientArguments(argStrings)
+    val sparkConf = new SparkConf
+    val args = new ClientArguments(argStrings, sparkConf)
 
-    new Client(args).run
+    new Client(args, sparkConf).run
   }
 
   // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
@@ -475,29 +482,28 @@ object Client {
     }
   }
 
-  def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) {
+  def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
     Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
     // If log4j present, ensure ours overrides all others
     if (addLog4j) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + LOG4J_PROP)
     }
     // Normally the users app.jar is last in case conflicts with spark jars
-    val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false")
-      .toBoolean
+    val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean
     if (userClasspathFirst) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + SPARK_JAR)
     Client.populateHadoopClasspath(conf, env)
 
     if (!userClasspathFirst) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + "*")
   }
 }
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
similarity index 89%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
index 69038844bbe4324d3f28cb1a902c99a7c147b087..ddfec1a4ac6728e8dd0030a6b5fcf164a1dff42c 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -29,36 +29,41 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
 import akka.actor._
 import akka.remote._
 import akka.actor.Terminated
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.util.{Utils, AkkaUtils}
 import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
 import org.apache.spark.scheduler.SplitInfo
 
-class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf)
+  extends Logging {
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf)
+
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
 
   private val rpc: YarnRPC = YarnRPC.create(conf)
-  private var resourceManager: AMRMProtocol = null
-  private var appAttemptId: ApplicationAttemptId = null
-  private var reporterThread: Thread = null
+  private var resourceManager: AMRMProtocol = _
+  private var appAttemptId: ApplicationAttemptId = _
+  private var reporterThread: Thread = _
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
 
-  private var yarnAllocator: YarnAllocationHandler = null
+  private var yarnAllocator: YarnAllocationHandler = _
   private var driverClosed:Boolean = false
 
-  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1
-  var actor: ActorRef = null
+  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0,
+    conf = sparkConf)._1
+  var actor: ActorRef = _
 
   // This actor just working as a monitor to watch on Driver Actor.
   class MonitorActor(driverUrl: String) extends Actor {
 
-    var driver: ActorSelection = null
+    var driver: ActorSelection = _
 
     override def preStart() {
       logInfo("Listen to driver: " + driverUrl)
       driver = context.actorSelection(driverUrl)
-      driver ! "hello"
+      // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events.
+      driver ! "Hello"
       context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
     }
 
@@ -162,8 +167,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
         Thread.sleep(100)
       }
     }
-    System.setProperty("spark.driver.host", driverHost)
-    System.setProperty("spark.driver.port", driverPort.toString)
+    sparkConf.set("spark.driver.host",  driverHost)
+    sparkConf.set("spark.driver.port",  driverPort.toString)
 
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
       driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
@@ -175,9 +180,11 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
   private def allocateWorkers() {
 
     // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now.
-    val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = scala.collection.immutable.Map()
+    val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
+      scala.collection.immutable.Map()
 
-    yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, preferredNodeLocationData)
+    yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId,
+      args, preferredNodeLocationData, sparkConf)
 
     logInfo("Allocating " + args.numWorkers + " workers.")
     // Wait until all containers have finished
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
similarity index 97%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index 6a90cc51cfbaf71ef5683703dd84fa6d5b872ccb..132630e5ef04cbb09bc400dacef37a1ea2b0d28d 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -37,12 +37,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils}
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 
 class WorkerRunnable(
     container: Container,
     conf: Configuration,
+    sparkConf: SparkConf,
     masterAddress: String,
     slaveId: String,
     hostname: String,
@@ -51,7 +52,7 @@ class WorkerRunnable(
   extends Runnable with Logging {
 
   var rpc: YarnRPC = YarnRPC.create(conf)
-  var cm: ContainerManager = null
+  var cm: ContainerManager = _
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
 
   def run = {
@@ -200,7 +201,7 @@ class WorkerRunnable(
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
 
     // Allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
similarity index 95%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 9ab20735299c0e35a1e1c13fcd11485ee6ee87d0..e91257be8ed008585c0491fc0c1ce13a6258cd15 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -26,9 +26,9 @@ import scala.collection
 import scala.collection.JavaConversions._
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 
-import org.apache.spark.Logging
-import org.apache.spark.scheduler.SplitInfo
-import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
 import org.apache.spark.util.Utils
 
 import org.apache.hadoop.conf.Configuration
@@ -62,7 +62,8 @@ private[yarn] class YarnAllocationHandler(
     val workerMemory: Int,
     val workerCores: Int,
     val preferredHostToCount: Map[String, Int], 
-    val preferredRackToCount: Map[String, Int])
+    val preferredRackToCount: Map[String, Int],
+    val sparkConf: SparkConf)
   extends Logging {
   // These three are locked on allocatedHostToContainersMap. Complementary data structures
   // allocatedHostToContainersMap : containers which are running : host, Set<containerid>
@@ -214,9 +215,9 @@ private[yarn] class YarnAllocationHandler(
       // host if there are sufficiently large number of hosts/containers.
 
       val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size)
-      allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
-      allocatedContainers ++= ClusterScheduler.prioritizeContainers(rackLocalContainers)
-      allocatedContainers ++= ClusterScheduler.prioritizeContainers(offRackContainers)
+      allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers)
+      allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers)
+      allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers)
 
       // Run each of the allocated containers
       for (container <- allocatedContainers) {
@@ -239,7 +240,7 @@ private[yarn] class YarnAllocationHandler(
           // (workerIdCounter)
           val workerId = workerIdCounter.incrementAndGet().toString
           val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
-            System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+            sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"),
             CoarseGrainedSchedulerBackend.ACTOR_NAME)
 
           logInfo("launching container on " + containerId + " host " + workerHostname)
@@ -260,7 +261,7 @@ private[yarn] class YarnAllocationHandler(
           }
 
           new Thread(
-            new WorkerRunnable(container, conf, driverUrl, workerId,
+            new WorkerRunnable(container, conf, sparkConf, driverUrl, workerId,
               workerHostname, workerMemory, workerCores)
           ).start()
         }
@@ -552,7 +553,8 @@ object YarnAllocationHandler {
     conf: Configuration,
     resourceManager: AMRMProtocol,
     appAttemptId: ApplicationAttemptId,
-    args: ApplicationMasterArguments): YarnAllocationHandler = {
+    args: ApplicationMasterArguments,
+    sparkConf: SparkConf): YarnAllocationHandler = {
 
     new YarnAllocationHandler(
       conf,
@@ -562,7 +564,8 @@ object YarnAllocationHandler {
       args.workerMemory,
       args.workerCores,
       Map[String, Int](),
-      Map[String, Int]())
+      Map[String, Int](),
+      sparkConf)
   }
 
   def newAllocator(
@@ -571,7 +574,8 @@ object YarnAllocationHandler {
     appAttemptId: ApplicationAttemptId,
     args: ApplicationMasterArguments,
     map: collection.Map[String,
-    collection.Set[SplitInfo]]): YarnAllocationHandler = {
+    collection.Set[SplitInfo]],
+    sparkConf: SparkConf): YarnAllocationHandler = {
 
     val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
     new YarnAllocationHandler(
@@ -582,7 +586,8 @@ object YarnAllocationHandler {
       args.workerMemory,
       args.workerCores,
       hostToCount,
-      rackToCount)
+      rackToCount,
+      sparkConf)
   }
 
   def newAllocator(
@@ -592,7 +597,8 @@ object YarnAllocationHandler {
     maxWorkers: Int,
     workerMemory: Int,
     workerCores: Int,
-    map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
+    map: collection.Map[String, collection.Set[SplitInfo]],
+    sparkConf: SparkConf): YarnAllocationHandler = {
 
     val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
 
@@ -604,7 +610,8 @@ object YarnAllocationHandler {
       workerMemory,
       workerCores,
       hostToCount,
-      rackToCount)
+      rackToCount,
+      sparkConf)
   }
 
   // A simple method to copy the split info map.
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
similarity index 97%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 70be15d0a3999021636d93ef13f1683b9e1945cb..1419f215c78e5831e7134adbb6e10d5fb6420960 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -19,13 +19,14 @@ package org.apache.spark.deploy.yarn
 
 import scala.collection.mutable.{ArrayBuffer, HashMap}
 
+import org.apache.spark.SparkConf
 import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
 import org.apache.spark.util.IntParam
 import org.apache.spark.util.MemoryParam
 
 
 // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware !
-class ClientArguments(val args: Array[String]) {
+class ClientArguments(val args: Array[String], val sparkConf: SparkConf) {
   var addJars: String = null
   var files: String = null
   var archives: String = null
@@ -35,7 +36,7 @@ class ClientArguments(val args: Array[String]) {
   var workerMemory = 1024 // MB
   var workerCores = 1
   var numWorkers = 2
-  var amQueue = System.getProperty("QUEUE", "default")
+  var amQueue = sparkConf.get("QUEUE", "default")
   var amMemory: Int = 512 // MB
   var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
   var appName: String = "Spark"
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
similarity index 94%
rename from yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
index 63a0449e5a0730085554d2b8ae86067135fa8dba..522e0a9ad7eeb50f4c2b6b781a68ea998639b30a 100644
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
@@ -20,13 +20,14 @@ package org.apache.spark.scheduler.cluster
 import org.apache.spark._
 import org.apache.hadoop.conf.Configuration
 import org.apache.spark.deploy.yarn.YarnAllocationHandler
+import org.apache.spark.scheduler.TaskSchedulerImpl
 import org.apache.spark.util.Utils
 
 /**
  *
  * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM.
  */
-private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
+private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) {
 
   def this(sc: SparkContext) = this(sc, new Configuration())
 
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
similarity index 92%
rename from yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
index b206780c7806e15c84944db05876f89c8f848040..324ef4616fe268b4005a2ec014fb0a8636d4e2e2 100644
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
@@ -20,9 +20,10 @@ package org.apache.spark.scheduler.cluster
 import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState}
 import org.apache.spark.{SparkException, Logging, SparkContext}
 import org.apache.spark.deploy.yarn.{Client, ClientArguments}
+import org.apache.spark.scheduler.TaskSchedulerImpl
 
 private[spark] class YarnClientSchedulerBackend(
-    scheduler: ClusterScheduler,
+    scheduler: TaskSchedulerImpl,
     sc: SparkContext)
   extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
   with Logging {
@@ -52,8 +53,8 @@ private[spark] class YarnClientSchedulerBackend(
     if (workerNumber == null)
       workerNumber = defaultWorkerNumber
 
-    val driverHost = System.getProperty("spark.driver.host")
-    val driverPort = System.getProperty("spark.driver.port")
+    val driverHost = conf.get("spark.driver.host")
+    val driverPort = conf.get("spark.driver.port")
     val hostport = driverHost + ":" + driverPort
 
     val argsArray = Array[String](
@@ -66,8 +67,8 @@ private[spark] class YarnClientSchedulerBackend(
       "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher"
     )
 
-    val args = new ClientArguments(argsArray)
-    client = new Client(args)
+    val args = new ClientArguments(argsArray, conf)
+    client = new Client(args, conf)
     appId = client.runApp()
     waitForApp()
   }
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
similarity index 95%
rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
index 29b3f22e13697b38bc501e2f914d8fc0a202d722..a4638cc863611c0e152be72f47aed0c222aba5f8 100644
--- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -19,6 +19,7 @@ package org.apache.spark.scheduler.cluster
 
 import org.apache.spark._
 import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
+import org.apache.spark.scheduler.TaskSchedulerImpl
 import org.apache.spark.util.Utils
 import org.apache.hadoop.conf.Configuration
 
@@ -26,7 +27,7 @@ import org.apache.hadoop.conf.Configuration
  *
  * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of ApplicationMaster, etc is done
  */
-private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
+private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) {
 
   logInfo("Created YarnClusterScheduler")
 
diff --git a/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
similarity index 100%
rename from new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
rename to yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
diff --git a/yarn/pom.xml b/yarn/pom.xml
index bc64a190fd284cebc6ab993ea6de6254841402c0..aea8b0cddefa29d82dc6ceff60ec1d74b4de1100 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -25,11 +25,10 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-yarn_2.10</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project YARN Support</name>
-  <url>http://spark.incubator.apache.org/</url>
-
+  <artifactId>yarn-parent_2.10</artifactId>
+  <packaging>pom</packaging>
+  <name>Spark Project YARN Parent POM</name>
+  
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
@@ -73,45 +72,52 @@
     </dependency>
   </dependencies>
 
+  <profiles>
+    <profile>
+      <id>yarn-alpha</id>
+      <modules>
+        <module>alpha</module>
+      </modules>
+    </profile>
+
+    <profile>
+      <id>yarn</id>
+      <modules>
+        <module>stable</module>
+      </modules>
+    </profile>
+  </profiles>
+
   <build>
-    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <configuration>
-          <shadedArtifactAttached>false</shadedArtifactAttached>
-          <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.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>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
         <executions>
           <execution>
-            <phase>package</phase>
+           <id>add-scala-sources</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/main/scala</source>
+                <source>../common/src/main/scala</source>
+              </sources>
+            </configuration>
+          </execution>
+          <execution>
+            <id>add-scala-test-sources</id>
+            <phase>generate-test-sources</phase>
             <goals>
-              <goal>shade</goal>
+              <goal>add-test-source</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>
+              <sources>
+                <source>src/test/scala</source>
+                <source>../common/src/test/scala</source>
+              </sources>
             </configuration>
           </execution>
         </executions>
@@ -150,12 +156,16 @@
         <artifactId>scalatest-maven-plugin</artifactId>
         <configuration>
           <environmentVariables>
-            <SPARK_HOME>${basedir}/..</SPARK_HOME>
+            <SPARK_HOME>${basedir}/../..</SPARK_HOME>
             <SPARK_TESTING>1</SPARK_TESTING>
             <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
           </environmentVariables>
         </configuration>
       </plugin>
     </plugins>
+
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
   </build>
+
 </project>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
deleted file mode 100644
index f76a5ddd39e90d4998d7712113dd0949823d5e18..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.deploy.yarn
-
-import org.apache.spark.util.IntParam
-import collection.mutable.ArrayBuffer
-
-class ApplicationMasterArguments(val args: Array[String]) {
-  var userJar: String = null
-  var userClass: String = null
-  var userArgs: Seq[String] = Seq[String]()
-  var workerMemory = 1024
-  var workerCores = 1
-  var numWorkers = 2
-
-  parseArgs(args.toList)
-  
-  private def parseArgs(inputArgs: List[String]): Unit = {
-    val userArgsBuffer = new ArrayBuffer[String]()
-
-    var args = inputArgs
-
-    while (! args.isEmpty) {
-
-      args match {
-        case ("--jar") :: value :: tail =>
-          userJar = value
-          args = tail
-
-        case ("--class") :: value :: tail =>
-          userClass = value
-          args = tail
-
-        case ("--args") :: value :: tail =>
-          userArgsBuffer += value
-          args = tail
-
-        case ("--num-workers") :: IntParam(value) :: tail =>
-          numWorkers = value
-          args = tail
-
-        case ("--worker-memory") :: IntParam(value) :: tail =>
-          workerMemory = value
-          args = tail
-
-        case ("--worker-cores") :: IntParam(value) :: tail =>
-          workerCores = value
-          args = tail
-
-        case Nil =>
-          if (userJar == null || userClass == null) {
-            printUsageAndExit(1)
-          }
-
-        case _ =>
-          printUsageAndExit(1, args)
-      }
-    }
-
-    userArgs = userArgsBuffer.readOnly
-  }
-  
-  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
-    if (unknownParam != null) {
-      System.err.println("Unknown/unsupported param " + unknownParam)
-    }
-    System.err.println(
-      "Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] \n" +
-      "Options:\n" +
-      "  --jar JAR_PATH       Path to your application's JAR file (required)\n" +
-      "  --class CLASS_NAME   Name of your application's main class (required)\n" +
-      "  --args ARGS          Arguments to be passed to your application's main class.\n" +
-      "                       Mutliple invocations are possible, each will be passed in order.\n" +
-      "  --num-workers NUM    Number of workers to start (Default: 2)\n" +
-      "  --worker-cores NUM   Number of cores for the workers (Default: 1)\n" +
-      "  --worker-memory MEM  Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n")
-    System.exit(exitCode)
-  }
-}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
deleted file mode 100644
index b3a7886d9322688966a2722a33b3bd8f786d9949..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.deploy.yarn
-
-import org.apache.spark.util.MemoryParam
-import org.apache.spark.util.IntParam
-import collection.mutable.{ArrayBuffer, HashMap}
-import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
-
-// TODO: Add code and support for ensuring that yarn resource 'asks' are location aware !
-class ClientArguments(val args: Array[String]) {
-  var addJars: String = null
-  var files: String = null
-  var archives: String = null
-  var userJar: String = null
-  var userClass: String = null
-  var userArgs: Seq[String] = Seq[String]()
-  var workerMemory = 1024
-  var workerCores = 1
-  var numWorkers = 2
-  var amQueue = System.getProperty("QUEUE", "default")
-  var amMemory: Int = 512
-  var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
-  var appName: String = "Spark"
-  // TODO
-  var inputFormatInfo: List[InputFormatInfo] = null
-
-  parseArgs(args.toList)
-
-  private def parseArgs(inputArgs: List[String]): Unit = {
-    val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]()
-    val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]()
-
-    var args = inputArgs
-
-    while (! args.isEmpty) {
-
-      args match {
-        case ("--jar") :: value :: tail =>
-          userJar = value
-          args = tail
-
-        case ("--class") :: value :: tail =>
-          userClass = value
-          args = tail
-
-        case ("--args") :: value :: tail =>
-          userArgsBuffer += value
-          args = tail
-
-        case ("--master-class") :: value :: tail =>
-          amClass = value
-          args = tail
-
-        case ("--master-memory") :: MemoryParam(value) :: tail =>
-          amMemory = value
-          args = tail
-
-        case ("--worker-memory") :: MemoryParam(value) :: tail =>
-          workerMemory = value
-          args = tail
-
-        case ("--num-workers") :: IntParam(value) :: tail =>
-          numWorkers = value
-          args = tail
-
-        case ("--worker-cores") :: IntParam(value) :: tail =>
-          workerCores = value
-          args = tail
-
-        case ("--queue") :: value :: tail =>
-          amQueue = value
-          args = tail
-
-        case ("--name") :: value :: tail =>
-          appName = value
-          args = tail
-
-        case ("--addJars") :: value :: tail =>
-          addJars = value
-          args = tail
-
-        case ("--files") :: value :: tail =>
-          files = value
-          args = tail
-
-        case ("--archives") :: value :: tail =>
-          archives = value
-          args = tail
-
-        case Nil =>
-          if (userJar == null || userClass == null) {
-            printUsageAndExit(1)
-          }
-
-        case _ =>
-          printUsageAndExit(1, args)
-      }
-    }
-
-    userArgs = userArgsBuffer.readOnly
-    inputFormatInfo = inputFormatMap.values.toList
-  }
-
-
-  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
-    if (unknownParam != null) {
-      System.err.println("Unknown/unsupported param " + unknownParam)
-    }
-    System.err.println(
-      "Usage: org.apache.spark.deploy.yarn.Client [options] \n" +
-      "Options:\n" +
-      "  --jar JAR_PATH             Path to your application's JAR file (required)\n" +
-      "  --class CLASS_NAME         Name of your application's main class (required)\n" +
-      "  --args ARGS                Arguments to be passed to your application's main class.\n" +
-      "                             Mutliple invocations are possible, each will be passed in order.\n" +
-      "  --num-workers NUM          Number of workers to start (Default: 2)\n" +
-      "  --worker-cores NUM         Number of cores for the workers (Default: 1). This is unsused right now.\n" +
-      "  --master-class CLASS_NAME  Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" +
-      "  --master-memory MEM        Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
-      "  --worker-memory MEM        Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
-      "  --name NAME                The name of your application (Default: Spark)\n" +
-      "  --queue QUEUE              The hadoop queue to use for allocation requests (Default: 'default')\n" +
-      "  --addJars jars             Comma separated list of local jars that want SparkContext.addJar to work with.\n" +
-      "  --files files              Comma separated list of files to be distributed with the job.\n" +
-      "  --archives archives        Comma separated list of archives to be distributed with the job."
-      )
-    System.exit(exitCode)
-  }
-
-}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
deleted file mode 100644
index 5f159b073f5372dc9e1f73347f88ae8aa48e0cbb..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.deploy.yarn
-
-import java.net.URI
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileStatus
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.fs.permission.FsAction
-import org.apache.hadoop.yarn.api.records.LocalResource
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
-import org.apache.hadoop.yarn.api.records.LocalResourceType
-import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
-
-import org.apache.spark.Logging 
-
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.LinkedHashMap
-import scala.collection.mutable.Map
-
-
-/** Client side methods to setup the Hadoop distributed cache */
-class ClientDistributedCacheManager() extends Logging {
-  private val distCacheFiles: Map[String, Tuple3[String, String, String]] = 
-    LinkedHashMap[String, Tuple3[String, String, String]]()
-  private val distCacheArchives: Map[String, Tuple3[String, String, String]] = 
-    LinkedHashMap[String, Tuple3[String, String, String]]()
-
-
-  /**
-   * Add a resource to the list of distributed cache resources. This list can
-   * be sent to the ApplicationMaster and possibly the workers so that it can 
-   * be downloaded into the Hadoop distributed cache for use by this application.
-   * Adds the LocalResource to the localResources HashMap passed in and saves 
-   * the stats of the resources to they can be sent to the workers and verified.
-   *
-   * @param fs FileSystem
-   * @param conf Configuration
-   * @param destPath path to the resource
-   * @param localResources localResource hashMap to insert the resource into
-   * @param resourceType LocalResourceType 
-   * @param link link presented in the distributed cache to the destination
-   * @param statCache cache to store the file/directory stats 
-   * @param appMasterOnly Whether to only add the resource to the app master
-   */
-  def addResource(
-      fs: FileSystem,
-      conf: Configuration,
-      destPath: Path, 
-      localResources: HashMap[String, LocalResource],
-      resourceType: LocalResourceType,
-      link: String,
-      statCache: Map[URI, FileStatus],
-      appMasterOnly: Boolean = false) = {
-    val destStatus = fs.getFileStatus(destPath)
-    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
-    amJarRsrc.setType(resourceType)
-    val visibility = getVisibility(conf, destPath.toUri(), statCache)
-    amJarRsrc.setVisibility(visibility)
-    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath))
-    amJarRsrc.setTimestamp(destStatus.getModificationTime())
-    amJarRsrc.setSize(destStatus.getLen())
-    if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name")
-    localResources(link) = amJarRsrc
-    
-    if (appMasterOnly == false) {
-      val uri = destPath.toUri()
-      val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link)
-      if (resourceType == LocalResourceType.FILE) {
-        distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), 
-          destStatus.getModificationTime().toString(), visibility.name())
-      } else {
-        distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), 
-          destStatus.getModificationTime().toString(), visibility.name())
-      }
-    }
-  }
-
-  /**
-   * Adds the necessary cache file env variables to the env passed in
-   * @param env
-   */
-  def setDistFilesEnv(env: Map[String, String]) = {
-    val (keys, tupleValues) = distCacheFiles.unzip
-    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
-
-    if (keys.size > 0) {
-      env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = 
-        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = 
-        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = 
-        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
-    }
-  }
-
-  /**
-   * Adds the necessary cache archive env variables to the env passed in
-   * @param env
-   */
-  def setDistArchivesEnv(env: Map[String, String]) = {
-    val (keys, tupleValues) = distCacheArchives.unzip
-    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
-
-    if (keys.size > 0) {
-      env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = 
-        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") =
-        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = 
-        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
-    }
-  }
-
-  /**
-   * Returns the local resource visibility depending on the cache file permissions
-   * @param conf
-   * @param uri
-   * @param statCache
-   * @return LocalResourceVisibility
-   */
-  def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]):
-      LocalResourceVisibility = {
-    if (isPublic(conf, uri, statCache)) {
-      return LocalResourceVisibility.PUBLIC 
-    } 
-    return LocalResourceVisibility.PRIVATE
-  }
-
-  /**
-   * Returns a boolean to denote whether a cache file is visible to all(public)
-   * or not
-   * @param conf
-   * @param uri
-   * @param statCache
-   * @return true if the path in the uri is visible to all, false otherwise
-   */
-  def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = {
-    val fs = FileSystem.get(uri, conf)
-    val current = new Path(uri.getPath())
-    //the leaf level file should be readable by others
-    if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) {
-      return false
-    }
-    return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache)
-  }
-
-  /**
-   * Returns true if all ancestors of the specified path have the 'execute'
-   * permission set for all users (i.e. that other users can traverse
-   * the directory heirarchy to the given path)
-   * @param fs
-   * @param path
-   * @param statCache
-   * @return true if all ancestors have the 'execute' permission set for all users
-   */
-  def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, 
-      statCache: Map[URI, FileStatus]): Boolean =  {
-    var current = path
-    while (current != null) {
-      //the subdirs in the path should have execute permissions for others
-      if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) {
-        return false
-      }
-      current = current.getParent()
-    }
-    return true
-  }
-
-  /**
-   * Checks for a given path whether the Other permissions on it 
-   * imply the permission in the passed FsAction
-   * @param fs
-   * @param path
-   * @param action
-   * @param statCache
-   * @return true if the path in the uri is visible to all, false otherwise
-   */
-  def checkPermissionOfOther(fs: FileSystem, path: Path,
-      action: FsAction, statCache: Map[URI, FileStatus]): Boolean = {
-    val status = getFileStatus(fs, path.toUri(), statCache)
-    val perms = status.getPermission()
-    val otherAction = perms.getOtherAction()
-    if (otherAction.implies(action)) {
-      return true
-    }
-    return false
-  }
-
-  /**
-   * Checks to see if the given uri exists in the cache, if it does it 
-   * returns the existing FileStatus, otherwise it stats the uri, stores
-   * it in the cache, and returns the FileStatus.
-   * @param fs
-   * @param uri
-   * @param statCache
-   * @return FileStatus
-   */
-  def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = {
-    val stat = statCache.get(uri) match {
-      case Some(existstat) => existstat
-      case None => 
-        val newStat = fs.getFileStatus(new Path(uri))
-        statCache.put(uri, newStat)
-        newStat
-    }
-    return stat
-  }
-}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
deleted file mode 100644
index 2ba2366ead17113c764663ed934a5221f0eeb0ee..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.deploy.yarn
-
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.security.UserGroupInformation
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.conf.Configuration
-
-/**
- * Contains util methods to interact with Hadoop from spark.
- */
-class YarnSparkHadoopUtil extends SparkHadoopUtil {
-
-  // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true.
-  override def isYarnMode(): Boolean = { true }
-
-  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
-  // Always create a new config, dont reuse yarnConf.
-  override def newConfiguration(): Configuration = new YarnConfiguration(new Configuration())
-
-  // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
-  override def addCredentials(conf: JobConf) {
-    val jobCreds = conf.getCredentials()
-    jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
-  }
-}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
deleted file mode 100644
index 4e988b8017600e09b3300fe1a3ec36e82025a73d..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.scheduler.cluster
-
-import org.apache.hadoop.conf.Configuration
-
-import org.apache.spark._
-import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
-import org.apache.spark.scheduler.ClusterScheduler
-import org.apache.spark.util.Utils
-
-/**
- *
- * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of
- * ApplicationMaster, etc. is done
- */
-private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration)
-  extends ClusterScheduler(sc) {
-
-  logInfo("Created YarnClusterScheduler")
-
-  def this(sc: SparkContext) = this(sc, new Configuration())
-
-  // Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate
-  // Note that only the first creation of SparkContext influences (and ideally, there must be only one SparkContext, right ?)
-  // Subsequent creations are ignored - since nodes are already allocated by then.
-
-
-  // By default, rack is unknown
-  override def getRackForHost(hostPort: String): Option[String] = {
-    val host = Utils.parseHostPort(hostPort)._1
-    val retval = YarnAllocationHandler.lookupRack(conf, host)
-    if (retval != null) Some(retval) else None
-  }
-
-  override def postStartHook() {
-    val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc)
-    if (sparkContextInitialized){
-      // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
-      Thread.sleep(3000L)
-    }
-    logInfo("YarnClusterScheduler.postStartHook done")
-  }
-}
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
deleted file mode 100644
index 2941356bc55f9f85ca176c3ef0d23a1a08c6a8e5..0000000000000000000000000000000000000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.spark.deploy.yarn
-
-import java.net.URI
-
-import org.scalatest.FunSuite
-import org.scalatest.mock.MockitoSugar
-import org.mockito.Mockito.when
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileStatus
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.fs.permission.FsAction
-import org.apache.hadoop.yarn.api.records.LocalResource
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
-import org.apache.hadoop.yarn.api.records.LocalResourceType
-import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
-
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
-
-
-class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar {
-
-  class MockClientDistributedCacheManager extends ClientDistributedCacheManager {
-    override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): 
-        LocalResourceVisibility = {
-      return LocalResourceVisibility.PRIVATE
-    }
-  }
-  
-  test("test getFileStatus empty") {
-    val distMgr = new ClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val uri = new URI("/tmp/testing")
-    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val stat = distMgr.getFileStatus(fs, uri, statCache)
-    assert(stat.getPath() === null)
-  }
-
-  test("test getFileStatus cached") {
-    val distMgr = new ClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val uri = new URI("/tmp/testing")
-    val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", 
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus)
-    val stat = distMgr.getFileStatus(fs, uri, statCache)
-    assert(stat.getPath().toString() === "/tmp/testing")
-  }
-
-  test("test addResource") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", 
-      statCache, false)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 0)
-    assert(resource.getSize() === 0)
-    assert(resource.getType() === LocalResourceType.FILE)
-
-    val env = new HashMap[String, String]()
-    distMgr.setDistFilesEnv(env)
-    assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link")
-    assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0")
-    assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0")
-    assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
-
-    distMgr.setDistArchivesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
-
-    //add another one and verify both there and order correct
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
-      null, new Path("/tmp/testing2"))
-    val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2")
-    when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus)
-    distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", 
-      statCache, false)
-    val resource2 = localResources("link2")
-    assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2)
-    assert(resource2.getTimestamp() === 10)
-    assert(resource2.getSize() === 20)
-    assert(resource2.getType() === LocalResourceType.FILE)
-
-    val env2 = new HashMap[String, String]()
-    distMgr.setDistFilesEnv(env2)
-    val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
-    val files = env2("SPARK_YARN_CACHE_FILES").split(',') 
-    val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
-    val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',')
-    assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link")
-    assert(timestamps(0)  === "0")
-    assert(sizes(0)  === "0")
-    assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name())
-
-    assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2")
-    assert(timestamps(1)  === "10")
-    assert(sizes(1)  === "20")
-    assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name())
-  }
-
-  test("test addResource link null") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
-
-    intercept[Exception] {
-      distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, 
-        statCache, false)
-    }
-    assert(localResources.get("link") === None)
-    assert(localResources.size === 0)
-  }
-
-  test("test addResource appmaster only") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
-      statCache, true)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 10)
-    assert(resource.getSize() === 20)
-    assert(resource.getType() === LocalResourceType.ARCHIVE)
-
-    val env = new HashMap[String, String]()
-    distMgr.setDistFilesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
-
-    distMgr.setDistArchivesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
-  }
-
-  test("test addResource archive") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
-      statCache, false)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 10)
-    assert(resource.getSize() === 20)
-    assert(resource.getType() === LocalResourceType.ARCHIVE)
-
-    val env = new HashMap[String, String]()
-
-    distMgr.setDistArchivesEnv(env)
-    assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link")
-    assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10")
-    assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20")
-    assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
-
-    distMgr.setDistFilesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
-  }
-
-
-}
diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..62fe3e274250f2ad13d37adfa9f353666b0d9ac2
--- /dev/null
+++ b/yarn/stable/pom.xml
@@ -0,0 +1,32 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>yarn-parent_2.10</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-yarn_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project YARN Stable API</name>
+
+</project>
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
similarity index 87%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index eeeca3ea8a33e4f77562383dfef50a7507628894..69ae14ce8385cfb3b978481d90b2681d02a2ae80 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -38,15 +38,18 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
 
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.util.Utils
 
 
-class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
+                        sparkConf: SparkConf) extends Logging {
+
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
-  
-  private var rpc: YarnRPC = YarnRPC.create(conf)
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
   private var appAttemptId: ApplicationAttemptId = _
   private var userThread: Thread = _
@@ -61,14 +64,18 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   private var amClient: AMRMClient[ContainerRequest] = _
 
   // Default to numWorkers * 2, with minimum of 3
-  private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures",
-    math.max(args.numWorkers * 2, 3).toString()).toInt
+  private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures",
+    math.max(args.numWorkers * 2, 3))
 
   def run() {
     // Setup the directories so things go to YARN approved directories rather
     // than user specified and /tmp.
     System.setProperty("spark.local.dir", getLocalDirs())
 
+    // set the web ui port to be ephemeral for yarn so we don't conflict with
+    // other spark processes running on the same box
+    System.setProperty("spark.ui.port", "0")
+
     // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using.
     ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
 
@@ -81,16 +88,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // Workaround until hadoop moves to something which has
     // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line)
     // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
-    
+
     ApplicationMaster.register(this)
 
     // Start the user's JAR
     userThread = startUserClass()
-    
+
     // This a bit hacky, but we need to wait until the spark.driver.port property has
     // been set by the Thread executing the user class.
-    waitForSparkMaster()
-
     waitForSparkContextInitialized()
 
     // Do this after Spark master is up and SparkContext is created so that we can register UI Url.
@@ -99,7 +104,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // Allocate all containers
     allocateWorkers()
 
-    // Wait for the user class to Finish     
+    // Wait for the user class to Finish
     userThread.join()
 
     System.exit(0)
@@ -112,14 +117,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
     val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
       .getOrElse(Option(System.getenv("LOCAL_DIRS"))
-        .getOrElse(""))
+      .getOrElse(""))
 
     if (localDirs.isEmpty()) {
       throw new Exception("Yarn Local dirs can't be empty")
     }
     localDirs
   }
-  
+
   private def getApplicationAttemptId(): ApplicationAttemptId = {
     val envs = System.getenv()
     val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
@@ -128,41 +133,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     logInfo("ApplicationAttemptId: " + appAttemptId)
     appAttemptId
   }
-  
+
   private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
     logInfo("Registering the ApplicationMaster")
     amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
   }
-  
-  private def waitForSparkMaster() {
-    logInfo("Waiting for Spark driver to be reachable.")
-    var driverUp = false
-    var tries = 0
-    val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt
-    while (!driverUp && tries < numTries) {
-      val driverHost = System.getProperty("spark.driver.host")
-      val driverPort = System.getProperty("spark.driver.port")
-      try {
-        val socket = new Socket(driverHost, driverPort.toInt)
-        socket.close()
-        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
-        driverUp = true
-      } catch {
-        case e: Exception => {
-          logWarning("Failed to connect to driver at %s:%s, retrying ...".
-            format(driverHost, driverPort))
-          Thread.sleep(100)
-          tries = tries + 1
-        }
-      }
-    }
-  }
 
-  private def startUserClass(): Thread  = {
+  private def startUserClass(): Thread = {
     logInfo("Starting the user JAR in a separate Thread")
     val mainMethod = Class.forName(
       args.userClass,
-      false /* initialize */,
+      false /* initialize */ ,
       Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
     val t = new Thread {
       override def run() {
@@ -199,7 +180,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       ApplicationMaster.sparkContextRef.synchronized {
         var numTries = 0
         val waitTime = 10000L
-        val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt
+        val maxNumTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10)
         while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) {
           logInfo("Waiting for Spark context initialization ... " + numTries)
           numTries = numTries + 1
@@ -214,8 +195,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
             yarnConf,
             amClient,
             appAttemptId,
-            args, 
-            sparkContext.preferredNodeLocationData)
+            args,
+            sparkContext.preferredNodeLocationData,
+            sparkContext.getConf)
         } else {
           logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d".
             format(numTries * waitTime, maxNumTries))
@@ -223,7 +205,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
             yarnConf,
             amClient,
             appAttemptId,
-            args)
+            args,
+            sparkContext.getConf)
         }
       }
     } finally {
@@ -265,7 +248,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
       // we want to be reasonably responsive without causing too many requests to RM.
       val schedulerInterval =
-        System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong
+        sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000)
+
 
       // must be <= timeoutInterval / 2.
       val interval = math.min(timeoutInterval / 2, schedulerInterval)
@@ -275,7 +259,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
 
   private def launchReporterThread(_sleepTime: Long): Thread = {
-    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+    val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
 
     val t = new Thread {
       override def run() {
@@ -314,11 +298,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     for (container <- containers) {
       logInfo("Launching shell command on a new container."
         + ", containerId=" + container.getId()
-        + ", containerNode=" + container.getNodeId().getHost() 
+        + ", containerNode=" + container.getNodeId().getHost()
         + ":" + container.getNodeId().getPort()
         + ", containerNodeURI=" + container.getNodeHttpAddress()
         + ", containerState" + container.getState()
-        + ", containerResourceMemory"  
+        + ", containerResourceMemory"
         + container.getResource().getMemory())
     }
   }
@@ -334,16 +318,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     logInfo("finishApplicationMaster with " + status)
     // Set tracking URL to empty since we don't have a history server.
-    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */)
   }
 
   /**
-   * Clean up the staging directory. 
+   * Clean up the staging directory.
    */
-  private def cleanupStagingDir() { 
+  private def cleanupStagingDir() {
     var stagingDirPath: Path = null
     try {
-      val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+      val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean
       if (!preserveFiles) {
         stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
         if (stagingDirPath == null) {
@@ -359,7 +343,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
   }
 
-  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM.
   class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
 
     def run() {
@@ -369,6 +353,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
     }
   }
+
 }
 
 object ApplicationMaster {
@@ -415,19 +400,20 @@ object ApplicationMaster {
     // Note that this will unfortunately not properly clean up the staging files because it gets
     // called too late, after the filesystem is already shutdown.
     if (modified) {
-      Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
+      Runtime.getRuntime().addShutdownHook(new Thread with Logging {
         // This is not only logs, but also ensures that log system is initialized for this instance
         // when we are actually 'run'-ing.
         logInfo("Adding shutdown hook for context " + sc)
-        override def run() { 
-          logInfo("Invoking sc stop from shutdown hook") 
-          sc.stop() 
+
+        override def run() {
+          logInfo("Invoking sc stop from shutdown hook")
+          sc.stop()
           // Best case ...
           for (master <- applicationMasters) {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
-        } 
-      } )
+        }
+      })
     }
 
     // Wait for initialization to complete and atleast 'some' nodes can get allocated.
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
similarity index 92%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 94678815e806ad66680880e59ac4f7c01d1fd67c..440ad5cde54eb4c8812bbb510329a2928c0a051e 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, Records}
 
-import org.apache.spark.Logging 
+import org.apache.spark.{Logging, SparkConf}
 import org.apache.spark.util.Utils
 import org.apache.spark.deploy.SparkHadoopUtil
 
@@ -50,7 +50,13 @@ import org.apache.spark.deploy.SparkHadoopUtil
  * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster,
  * which will launch a Spark master process and negotiate resources throughout its duration.
  */
-class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
+class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
+  extends YarnClientImpl with Logging {
+
+  def this(args: ClientArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ClientArguments) = this(args, new SparkConf())
 
   var rpc: YarnRPC = YarnRPC.create(conf)
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
@@ -63,8 +69,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   // App files are world-wide readable and owner writable -> rw-r--r--
   val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short)
 
-  def this(args: ClientArguments) = this(new Configuration(), args)
-
   def runApp(): ApplicationId = {
     validateArgs()
     // Initialize and start the client service.
@@ -141,7 +145,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       clusterMetrics.getNumNodeManagers)
 
     val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
-    logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
+    logInfo( """Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
       queueApplicationCount = %s, queueChildQueueCount = %s""".format(
         queueInfo.getQueueName,
         queueInfo.getCurrentCapacity,
@@ -150,7 +154,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         queueInfo.getChildQueues.size))
   }
 
-  def verifyClusterResources(app: GetNewApplicationResponse) = { 
+  def verifyClusterResources(app: GetNewApplicationResponse) = {
     val maxMem = app.getMaximumResourceCapability().getMemory()
     logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
 
@@ -221,7 +225,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
       fs.setReplication(newPath, replication)
       if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
-    } 
+    }
     // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
     // version shows the specific version in the distributed cache configuration
     val qualPath = fs.makeQualified(newPath)
@@ -244,7 +248,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       }
     }
     val dst = new Path(fs.getHomeDirectory(), appStagingDir)
-    val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
+    val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort
 
     if (UserGroupInformation.isSecurityEnabled()) {
       val dstFs = dst.getFileSystem(conf)
@@ -269,7 +273,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         }
         val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
         val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           destName, statCache)
       }
     }
@@ -283,7 +287,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val destPath = copyRemoteFile(dst, localPath, replication)
         // Only add the resource to the Spark ApplicationMaster.
         val appMasterOnly = true
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache, appMasterOnly)
       }
     }
@@ -295,7 +299,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache)
       }
     }
@@ -307,7 +311,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
           linkname, statCache)
       }
     }
@@ -317,14 +321,14 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   }
 
   def setupLaunchEnv(
-      localResources: HashMap[String, LocalResource], 
+      localResources: HashMap[String, LocalResource],
       stagingDir: String): HashMap[String, String] = {
     logInfo("Setting up the launch environment")
     val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
 
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_STAGING_DIR") = stagingDir
 
@@ -345,7 +349,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     val prefix = " --args "
     val args = clientArgs.userArgs
     val retval = new StringBuilder()
-    for (arg <- args){
+    for (arg <- args) {
       retval.append(prefix).append(" '").append(arg).append("' ")
     }
     retval.toString
@@ -406,11 +410,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     }
 
     val commands = List[String](
-      javaCommand + 
+      javaCommand +
       " -server " +
       JAVA_OPTS +
       " " + args.amClass +
-      " --class " + args.userClass + 
+      " --class " + args.userClass +
       " --jar " + args.userJar +
       userArgsToString(args) +
       " --worker-memory " + args.workerMemory +
@@ -436,9 +440,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     super.submitApplication(appContext)
   }
 
-  def monitorApplication(appId: ApplicationId): Boolean = {  
+  def monitorApplication(appId: ApplicationId): Boolean = {
+    val interval = sparkConf.getLong("spark.yarn.report.interval", 1000)
+
     while (true) {
-      Thread.sleep(1000)
+      Thread.sleep(interval)
       val report = super.getApplicationReport(appId)
 
       logInfo("Application report from ASM: \n" +
@@ -458,7 +464,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
       val state = report.getYarnApplicationState()
       val dsStatus = report.getFinalApplicationStatus()
-      if (state == YarnApplicationState.FINISHED || 
+      if (state == YarnApplicationState.FINISHED ||
         state == YarnApplicationState.FAILED ||
         state == YarnApplicationState.KILLED) {
         return true
@@ -478,10 +484,10 @@ object Client {
     // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -
     // see Client#setupLaunchEnv().
     System.setProperty("SPARK_YARN_MODE", "true")
+    val sparkConf = new SparkConf()
+    val args = new ClientArguments(argStrings, sparkConf)
 
-    val args = new ClientArguments(argStrings)
-
-    (new Client(args)).run()
+    new Client(args, sparkConf).run()
   }
 
   // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
@@ -491,29 +497,29 @@ object Client {
     }
   }
 
-  def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) {
+  def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
     Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
     // If log4j present, ensure ours overrides all others
     if (addLog4j) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + LOG4J_PROP)
     }
     // Normally the users app.jar is last in case conflicts with spark jars
-    val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false")
+    val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false")
       .toBoolean
     if (userClasspathFirst) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + SPARK_JAR)
     Client.populateHadoopClasspath(conf, env)
 
     if (!userClasspathFirst) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + "*")
   }
 }
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
similarity index 87%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
index bc31bb2eb09013651f23b2fa52ce017ea1f82bdc..49248a8516b9cef6941524500965354d9ac30d08 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -28,16 +28,20 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
 import akka.actor._
 import akka.remote._
 import akka.actor.Terminated
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.util.{Utils, AkkaUtils}
 import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
 import org.apache.spark.scheduler.SplitInfo
 import org.apache.hadoop.yarn.client.api.AMRMClient
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
 
-class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf)
+  extends Logging {
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
 
   private var appAttemptId: ApplicationAttemptId = _
   private var reporterThread: Thread = _
@@ -48,23 +52,26 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
 
   private var amClient: AMRMClient[ContainerRequest] = _
 
-  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1
+  val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0,
+    conf = sparkConf)._1
   var actor: ActorRef = _
 
   // This actor just working as a monitor to watch on Driver Actor.
   class MonitorActor(driverUrl: String) extends Actor {
 
-    var driver: ActorSelection = null
+    var driver: ActorSelection = _
 
     override def preStart() {
       logInfo("Listen to driver: " + driverUrl)
       driver = context.actorSelection(driverUrl)
+      // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events.
+      driver ! "Hello"
       context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
     }
 
     override def receive = {
       case x: DisassociatedEvent =>
-        logInfo("Driver terminated or disconnected! Shutting down.")
+        logInfo(s"Driver terminated or disconnected! Shutting down. $x")
         driverClosed = true
     }
   }
@@ -90,7 +97,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
     // must be <= timeoutInterval/ 2.
     // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM.
     // so atleast 1 minute or timeoutInterval / 10 - whichever is higher.
-    val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L))
+    val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval / 10, 60000L))
     reporterThread = launchReporterThread(interval)
 
     // Wait for the reporter thread to Finish.
@@ -136,8 +143,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
         Thread.sleep(100)
       }
     }
-    System.setProperty("spark.driver.host", driverHost)
-    System.setProperty("spark.driver.port", driverPort.toString)
+    sparkConf.set("spark.driver.host", driverHost)
+    sparkConf.set("spark.driver.port", driverPort.toString)
 
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
       driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
@@ -157,7 +164,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
       amClient,
       appAttemptId,
       args,
-      preferredNodeLocationData)
+      preferredNodeLocationData,
+      sparkConf)
 
     logInfo("Allocating " + args.numWorkers + " workers.")
     // Wait until all containers have finished
@@ -165,7 +173,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
     // TODO: Handle container failure
 
     yarnAllocator.addResourceRequests(args.numWorkers)
-    while(yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+    while (yarnAllocator.getNumWorkersRunning < args.numWorkers) {
       yarnAllocator.allocateResources()
       Thread.sleep(100)
     }
@@ -176,7 +184,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
 
   // TODO: We might want to extend this to allocate more containers in case they die !
   private def launchReporterThread(_sleepTime: Long): Thread = {
-    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+    val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
 
     val t = new Thread {
       override def run() {
@@ -208,7 +216,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
 
   def finishApplicationMaster(status: FinalApplicationStatus) {
     logInfo("finish ApplicationMaster with " + status)
-    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */)
   }
 
 }
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
similarity index 97%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index 9f5523c4b97a8811c14a19f9e34c6cd1e32028e7..b7699050bbaeb0f8dbf63b2f9a942eb3b1ab1bab 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -39,12 +39,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 
 class WorkerRunnable(
     container: Container,
     conf: Configuration,
+    sparkConf: SparkConf,
     masterAddress: String,
     slaveId: String,
     hostname: String,
@@ -197,7 +198,7 @@ class WorkerRunnable(
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
 
     // Allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
similarity index 96%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index c27257cda4e55c361b4e160da670f9afa664d87c..738ff986d85a58152e4b874a698f380b2b645977 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -26,9 +26,9 @@ import scala.collection
 import scala.collection.JavaConversions._
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 
-import org.apache.spark.Logging
-import org.apache.spark.scheduler.SplitInfo
-import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
 import org.apache.spark.util.Utils
 
 import org.apache.hadoop.conf.Configuration
@@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
 import org.apache.hadoop.yarn.util.{RackResolver, Records}
 
 
-object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
+object AllocationType extends Enumeration {
   type AllocationType = Value
   val HOST, RACK, ANY = Value
 }
@@ -64,7 +64,8 @@ private[yarn] class YarnAllocationHandler(
     val workerMemory: Int,
     val workerCores: Int,
     val preferredHostToCount: Map[String, Int], 
-    val preferredRackToCount: Map[String, Int])
+    val preferredRackToCount: Map[String, Int],
+    val sparkConf: SparkConf)
   extends Logging {
   // These three are locked on allocatedHostToContainersMap. Complementary data structures
   // allocatedHostToContainersMap : containers which are running : host, Set<containerid>
@@ -233,9 +234,9 @@ private[yarn] class YarnAllocationHandler(
       // Note that the list we create below tries to ensure that not all containers end up within
       // a host if there is a sufficiently large number of hosts/containers.
       val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size)
-      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
-      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(rackLocalContainers)
-      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(offRackContainers)
+      allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers)
+      allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers)
+      allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers)
 
       // Run each of the allocated containers.
       for (container <- allocatedContainersToProcess) {
@@ -254,8 +255,8 @@ private[yarn] class YarnAllocationHandler(
         } else {
           val workerId = workerIdCounter.incrementAndGet().toString
           val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
-            System.getProperty("spark.driver.host"),
-            System.getProperty("spark.driver.port"),
+            sparkConf.get("spark.driver.host"),
+            sparkConf.get("spark.driver.port"),
             CoarseGrainedSchedulerBackend.ACTOR_NAME)
 
           logInfo("Launching container %s for on host %s".format(containerId, workerHostname))
@@ -279,6 +280,7 @@ private[yarn] class YarnAllocationHandler(
           val workerRunnable = new WorkerRunnable(
             container,
             conf,
+            sparkConf,
             driverUrl,
             workerId,
             workerHostname,
@@ -565,7 +567,8 @@ object YarnAllocationHandler {
       conf: Configuration,
       amClient: AMRMClient[ContainerRequest],
       appAttemptId: ApplicationAttemptId,
-      args: ApplicationMasterArguments
+      args: ApplicationMasterArguments,
+      sparkConf: SparkConf
     ): YarnAllocationHandler = {
     new YarnAllocationHandler(
       conf,
@@ -575,7 +578,8 @@ object YarnAllocationHandler {
       args.workerMemory,
       args.workerCores,
       Map[String, Int](),
-      Map[String, Int]())
+      Map[String, Int](),
+      sparkConf)
   }
 
   def newAllocator(
@@ -584,7 +588,8 @@ object YarnAllocationHandler {
       appAttemptId: ApplicationAttemptId,
       args: ApplicationMasterArguments,
       map: collection.Map[String,
-      collection.Set[SplitInfo]]
+      collection.Set[SplitInfo]],
+      sparkConf: SparkConf
     ): YarnAllocationHandler = {
     val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map)
     new YarnAllocationHandler(
@@ -595,7 +600,8 @@ object YarnAllocationHandler {
       args.workerMemory,
       args.workerCores,
       hostToSplitCount,
-      rackToSplitCount)
+      rackToSplitCount,
+      sparkConf)
   }
 
   def newAllocator(
@@ -605,7 +611,8 @@ object YarnAllocationHandler {
       maxWorkers: Int,
       workerMemory: Int,
       workerCores: Int,
-      map: collection.Map[String, collection.Set[SplitInfo]]
+      map: collection.Map[String, collection.Set[SplitInfo]],
+      sparkConf: SparkConf
     ): YarnAllocationHandler = {
     val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
     new YarnAllocationHandler(
@@ -616,7 +623,8 @@ object YarnAllocationHandler {
       workerMemory,
       workerCores,
       hostToCount,
-      rackToCount)
+      rackToCount,
+      sparkConf)
   }
 
   // A simple method to copy the split info map.