diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala
index 76b088448125bedc293b35886f3a7e648bf18fc1..2ea38d4a5d2392b2e37fa9be3bc58dfa64171ddc 100644
--- a/core/src/test/scala/spark/DistributedSuite.scala
+++ b/core/src/test/scala/spark/DistributedSuite.scala
@@ -158,12 +158,28 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
     assert(data.count() === 1000)
   }
 
-  test("compute without caching with low memory") {
+  test("compute without caching when no partitions fit in memory") {
     System.setProperty("spark.storage.memoryFraction", "0.0001")
     sc = new SparkContext(clusterUrl, "test")
-    val data = sc.parallelize(1 to 4000000, 2).persist(StorageLevel.MEMORY_ONLY)
+    // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache
+    // to only 50 KB (0.0001 of 512 MB), so no partitions should fit in memory
+    val data = sc.parallelize(1 to 4000000, 2).persist(StorageLevel.MEMORY_ONLY_SER)
     assert(data.count() === 4000000)
     assert(data.count() === 4000000)
     assert(data.count() === 4000000)
+    System.clearProperty("spark.storage.memoryFraction")
+  }
+
+  test("compute when only some partitions fit in memory") {
+    System.setProperty("spark.storage.memoryFraction", "0.01")
+    sc = new SparkContext(clusterUrl, "test")
+    // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache
+    // to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions
+    // to make sure that *some* of them do fit though
+    val data = sc.parallelize(1 to 4000000, 20).persist(StorageLevel.MEMORY_ONLY_SER)
+    assert(data.count() === 4000000)
+    assert(data.count() === 4000000)
+    assert(data.count() === 4000000)
+    System.clearProperty("spark.storage.memoryFraction")
   }
 }