Skip to content
Closed
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Fix DistributedSuite
  • Loading branch information
Andrew Or committed Oct 14, 2015
commit 7226933d2a40896b9c4d606eb2c0ab6437507431
39 changes: 26 additions & 13 deletions core/src/test/scala/org/apache/spark/DistributedSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -203,22 +203,35 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
}

test("compute without caching when no partitions fit in memory") {
sc = new SparkContext(clusterUrl, "test")
// 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)
val size = 10000
val conf = new SparkConf()
.set("spark.storage.unrollMemoryThreshold", "1024")
.set("spark.testing.memory", (size / 2).toString)
sc = new SparkContext(clusterUrl, "test", conf)
val data = sc.parallelize(1 to size, 2).persist(StorageLevel.MEMORY_ONLY)
assert(data.count() === size)
assert(data.count() === size)
assert(data.count() === size)
// ensure only a subset of partitions were cached
val rddBlocks = sc.env.blockManager.master.getMatchingBlockIds(_.isRDD, askSlaves = true)
assert(rddBlocks.size === 0, s"expected no RDD blocks, found ${rddBlocks.size}")
}

test("compute when only some partitions fit in memory") {
sc = new SparkContext(clusterUrl, "test", new SparkConf)
// TODO: verify that only a subset of partitions fit in memory (SPARK-11078)
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)
val size = 10000
val numPartitions = 10
val conf = new SparkConf()
.set("spark.storage.unrollMemoryThreshold", "1024")
.set("spark.testing.memory", (size * numPartitions).toString)
sc = new SparkContext(clusterUrl, "test", conf)
val data = sc.parallelize(1 to size, numPartitions).persist(StorageLevel.MEMORY_ONLY)
assert(data.count() === size)
assert(data.count() === size)
assert(data.count() === size)
// ensure only a subset of partitions were cached
val rddBlocks = sc.env.blockManager.master.getMatchingBlockIds(_.isRDD, askSlaves = true)
assert(rddBlocks.size > 0, "no RDD blocks found")
assert(rddBlocks.size < numPartitions, s"too many RDD blocks found, expected <$numPartitions")
}

test("passing environment variables to cluster") {
Expand Down