diff --git a/README.md b/README.md
index 22e7ab824577ae8d5f0b445a9d1c57670c4055bb..8f686743f042ba8dbb6b9d2b672357b83ad9c864 100644
--- a/README.md
+++ b/README.md
@@ -13,7 +13,7 @@ This README file only contains basic setup instructions.
 ## Building
 
 Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT),
-which can be obtained from [here](http://www.scala-sbt.org/release/docs/Getting-Started/Setup.html). To build Spark and its example programs, run:
+which can be obtained from [here](http://www.scala-sbt.org). To build Spark and its example programs, run:
 
     sbt assembly
 
@@ -38,19 +38,22 @@ locally with one thread, or "local[N]" to run locally with N threads.
 
 ## Running tests
 
-### With sbt. (you need sbt installed)
-Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows on *nix based systems using sbt.
+### With sbt (Much faster to run compared to maven)
+Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows using sbt.
 
-`SPARK_HOME=$(pwd) SPARK_TESTING=1 sbt test`
- 
-TODO: figure out instructions for windows.
+`sbt test`
  
 ### With maven.
+1. Export these necessary environment variables as follows.
+
+ `export SCALA_HOME=<scala distribution>`
+
+ `export MAVEN_OPTS="-Xmx1512m -XX:MaxPermSize=512m"`
 
-1. Build assembly by
+2. Build assembly by
 `mvn package -DskipTests`
 
-2. Run tests
+3. Run tests
 `mvn test`
 
 ## A Note About Hadoop Versions
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 4d6a97e255d348c2d229f2b6acf373e50b0d4764..c6f6883b011527d17917625c16967318835b8a62 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -169,10 +169,15 @@ class SparkContext(
   // 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");
+  for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS");
       value <- Option(System.getenv(key))) {
     executorEnvs(key) = value
   }
+  // A workaround for SPARK_TESTING and SPARK_HOME
+  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
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index 6d1695eae74befd8a652e7b5f26d4dc3e61bbacd..89c5631ad8f07a49fc39e85820bb9b5d800cde42 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -30,13 +30,16 @@ 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 = Option(System.getenv("SPARK_HOME"))
+      .orElse(Option(System.getProperty("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("./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/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
index 4cb4ddc9cd66ae9385aa981618ebdd4baf23cbf3..894a72284b3fe840893d0e55b29a59be8294b940 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,13 +18,19 @@
 package org.apache.spark.deploy.worker
 
 import java.io.File
+
+import scala.util.Try
+
 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 sparkHome = Try(sys.env("SPARK_HOME")).toOption
+      .orElse(Option(System.getProperty("spark.home"))).get
     val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()),
       sparkHome, "appUiUrl")
     val appId = "12345-worker321-9876"
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 13d5fd3685bfbaec238155320575f87ab0822469..aded6430b376742ac4a1d76491c469f7bad836bf 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 assembly
+    SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true 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`.
@@ -72,7 +72,7 @@ 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 assembly
+    $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly
 
     # Configure logging
     $ cp conf/log4j.properties.template conf/log4j.properties
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index b335b5a20af8072f472cfefa71852dd3b6f1d45a..8290e7cf438c743e28674ad9fe3b2a4a84f4d723 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._
 
@@ -68,10 +69,12 @@ 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")
+  System.setProperty("spark.home", sparkHome)
+  System.setProperty("spark.testing", "1")
   // 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,7 +82,7 @@ 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
   }
@@ -112,8 +115,9 @@ object SparkBuild extends Build {
 
     // Fork new JVMs for tests and set Java options for those
     fork := true,
+    javaOptions += "-Dspark.home=" + sparkHome,
+    javaOptions += "-Dspark.testing=1",
     javaOptions += "-Xmx3g",
-
     // Show full stack trace and duration in test cases.
     testOptions in Test += Tests.Argument("-oDF"),
 
diff --git a/python/lib/py4j-0.8.1.zip b/python/lib/py4j-0.8.1.zip
new file mode 100644
index 0000000000000000000000000000000000000000..3231e31164e51daad4254e8f09a49336205563b8
Binary files /dev/null and b/python/lib/py4j-0.8.1.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/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index f108c70f215f5d7655f105388c8ff515e83d6a42..7e54421b52f82ba62903d0926a6b89782d44b9ec 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -942,12 +942,14 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     val conf = new SparkConf()
       .setMaster(master)
       .setAppName("Spark shell")
-      .setSparkHome(System.getenv("SPARK_HOME"))
       .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