diff --git a/R/run-tests.sh b/R/run-tests.sh
index e82ad0ba2cd0690c7c0b861ba293d810a6ac6a04..18a1e13bdc65556e564f1ff74f1984330e394704 100755
--- a/R/run-tests.sh
+++ b/R/run-tests.sh
@@ -23,7 +23,7 @@ FAILED=0
 LOGFILE=$FWDIR/unit-tests.out
 rm -f $LOGFILE
 
-SPARK_TESTING=1 $FWDIR/../bin/sparkR --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
+SPARK_TESTING=1 $FWDIR/../bin/sparkR --conf spark.buffer.pageSize=4m --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
 FAILED=$((PIPESTATUS[0]||$FAILED))
 
 if [[ $FAILED != 0 ]]; then
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index c21990f4e477865e5485c9d89f22b1a1d1b19273..866e0b41515771574baa498aa54ab0c7b715f403 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -20,6 +20,9 @@ package org.apache.spark.util.collection.unsafe.sort;
 import java.io.IOException;
 import java.util.LinkedList;
 
+import scala.runtime.AbstractFunction0;
+import scala.runtime.BoxedUnit;
+
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -90,6 +93,17 @@ public final class UnsafeExternalSorter {
     this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
     this.pageSizeBytes = conf.getSizeAsBytes("spark.buffer.pageSize", "64m");
     initializeForWriting();
+
+    // Register a cleanup task with TaskContext to ensure that memory is guaranteed to be freed at
+    // the end of the task. This is necessary to avoid memory leaks in when the downstream operator
+    // does not fully consume the sorter's output (e.g. sort followed by limit).
+    taskContext.addOnCompleteCallback(new AbstractFunction0<BoxedUnit>() {
+      @Override
+      public BoxedUnit apply() {
+        freeMemory();
+        return null;
+      }
+    });
   }
 
   // TODO: metrics tracking + integration with shuffle write metrics
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 90cd342a6cf7fc210944d1a288a723e43afb3055..60be85e53e2aa6211599547519d9307be4071027 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -52,7 +52,11 @@ def launch_gateway():
         script = "./bin/spark-submit.cmd" if on_windows else "./bin/spark-submit"
         submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell")
         if os.environ.get("SPARK_TESTING"):
-            submit_args = "--conf spark.ui.enabled=false " + submit_args
+            submit_args = ' '.join([
+                "--conf spark.ui.enabled=false",
+                "--conf spark.buffer.pageSize=4mb",
+                submit_args
+            ])
         command = [os.path.join(SPARK_HOME, script)] + shlex.split(submit_args)
 
         # Start a socket that will be used by PythonGatewayServer to communicate its port to us
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
index 2564bbd2077bf232f070ae75be93278079615ee7..6644e85d4a037d488f12a565d0c830bc0a314a35 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
@@ -229,7 +229,7 @@ private[spark] object SQLConf {
       " a specific query.")
 
   val UNSAFE_ENABLED = booleanConf("spark.sql.unsafe.enabled",
-    defaultValue = Some(false),
+    defaultValue = Some(true),
     doc = "When true, use the new optimized Tungsten physical execution backend.")
 
   val DIALECT = stringConf(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
index 41a0c519ba52760f1df4ae973c07c6205a3b14e8..70e5031fb63c0c6646ea1286dfcebb0aeb1ea9b5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
@@ -47,7 +47,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
 
   override def canProcessSafeRows: Boolean = true
 
-  override def canProcessUnsafeRows: Boolean = true
+  override def canProcessUnsafeRows: Boolean = {
+    // Do not use the Unsafe path if we are using a RangePartitioning, since this may lead to
+    // an interpreted RowOrdering being applied to an UnsafeRow, which will lead to
+    // ClassCastExceptions at runtime. This check can be removed after SPARK-9054 is fixed.
+    !newPartitioning.isInstanceOf[RangePartitioning]
+  }
 
   /**
    * Determines whether records must be defensively copied before being sent to the shuffle.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
index 5c1102410879ab7ca12bbb68e64f8832cbe4a023..eb64684ae0fd96c5d374b8c4a0ce13372116c157 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql
 
 import org.scalatest.Matchers._
 
-import org.apache.spark.sql.execution.Project
+import org.apache.spark.sql.execution.{Project, TungstenProject}
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.test.SQLTestUtils
@@ -538,6 +538,7 @@ class ColumnExpressionSuite extends QueryTest with SQLTestUtils {
     def checkNumProjects(df: DataFrame, expectedNumProjects: Int): Unit = {
       val projects = df.queryExecution.executedPlan.collect {
         case project: Project => project
+        case tungstenProject: TungstenProject => tungstenProject
       }
       assert(projects.size === expectedNumProjects)
     }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeExternalSortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeExternalSortSuite.scala
index 7a4baa9e4a49d6bc21a618496f20310433cf9cb6..138636b0c65b822b5c761c0c04d49198e06540c5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeExternalSortSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeExternalSortSuite.scala
@@ -36,10 +36,7 @@ class UnsafeExternalSortSuite extends SparkPlanTest with BeforeAndAfterAll {
     TestSQLContext.conf.setConf(SQLConf.CODEGEN_ENABLED, SQLConf.CODEGEN_ENABLED.defaultValue.get)
   }
 
-  ignore("sort followed by limit should not leak memory") {
-    // TODO: this test is going to fail until we implement a proper iterator interface
-    // with a close() method.
-    TestSQLContext.sparkContext.conf.set("spark.unsafe.exceptionOnMemoryLeak", "false")
+  test("sort followed by limit") {
     checkThatPlansAgree(
       (1 to 100).map(v => Tuple1(v)).toDF("a"),
       (child: SparkPlan) => Limit(10, UnsafeExternalSort('a.asc :: Nil, true, child)),
@@ -48,21 +45,6 @@ class UnsafeExternalSortSuite extends SparkPlanTest with BeforeAndAfterAll {
     )
   }
 
-  test("sort followed by limit") {
-    TestSQLContext.sparkContext.conf.set("spark.unsafe.exceptionOnMemoryLeak", "false")
-    try {
-      checkThatPlansAgree(
-        (1 to 100).map(v => Tuple1(v)).toDF("a"),
-        (child: SparkPlan) => Limit(10, UnsafeExternalSort('a.asc :: Nil, true, child)),
-        (child: SparkPlan) => Limit(10, Sort('a.asc :: Nil, global = true, child)),
-        sortAnswers = false
-      )
-    } finally {
-      TestSQLContext.sparkContext.conf.set("spark.unsafe.exceptionOnMemoryLeak", "false")
-
-    }
-  }
-
   test("sorting does not crash for large inputs") {
     val sortOrder = 'a.asc :: Nil
     val stringLength = 1024 * 1024 * 2