aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorShivaram Venkataraman <shivaram@eecs.berkeley.edu>2012-08-11 16:42:35 -0700
committerShivaram Venkataraman <shivaram@eecs.berkeley.edu>2012-08-11 16:42:35 -0700
commit73452cc64989752c4aa8e3a05abed314a5b6b985 (patch)
tree0ff7f900d9f70a50bdd3641ee524164ad3334ea5
parent64b8fd62f0de1f789c5a48ef8b60fdbb2c875704 (diff)
downloadspark-73452cc64989752c4aa8e3a05abed314a5b6b985.tar.gz
spark-73452cc64989752c4aa8e3a05abed314a5b6b985.tar.bz2
spark-73452cc64989752c4aa8e3a05abed314a5b6b985.zip
Update test cases to match the new size estimates. Uses 64-bit and compressed
oops setting to get deterministic results
-rw-r--r--core/src/test/scala/spark/BoundedMemoryCacheSuite.scala26
-rw-r--r--core/src/test/scala/spark/SizeEstimatorSuite.scala86
2 files changed, 77 insertions, 35 deletions
diff --git a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
index 024ce0b8d1..745c86a0d0 100644
--- a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
+++ b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
@@ -4,28 +4,44 @@ import org.scalatest.FunSuite
class BoundedMemoryCacheSuite extends FunSuite {
test("constructor test") {
- val cache = new BoundedMemoryCache(40)
- expect(40)(cache.getCapacity)
+ val cache = new BoundedMemoryCache(60)
+ expect(60)(cache.getCapacity)
}
test("caching") {
- val cache = new BoundedMemoryCache(40) {
+ // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
+ val oldArch = System.setProperty("os.arch", "amd64")
+ val oldOops = System.setProperty("spark.test.useCompressedOops", "true")
+
+ val cache = new BoundedMemoryCache(60) {
//TODO sorry about this, but there is not better way how to skip 'cacheTracker.dropEntry'
override protected def reportEntryDropped(datasetId: Any, partition: Int, entry: Entry) {
logInfo("Dropping key (%s, %d) of size %d to make space".format(datasetId, partition, entry.size))
}
}
//should be OK
- expect(CachePutSuccess(30))(cache.put("1", 0, "Meh"))
+ expect(CachePutSuccess(56))(cache.put("1", 0, "Meh"))
//we cannot add this to cache (there is not enough space in cache) & we cannot evict the only value from
//cache because it's from the same dataset
expect(CachePutFailure())(cache.put("1", 1, "Meh"))
//should be OK, dataset '1' can be evicted from cache
- expect(CachePutSuccess(30))(cache.put("2", 0, "Meh"))
+ expect(CachePutSuccess(56))(cache.put("2", 0, "Meh"))
//should fail, cache should obey it's capacity
expect(CachePutFailure())(cache.put("3", 0, "Very_long_and_useless_string"))
+
+ if (oldArch != null) {
+ System.setProperty("os.arch", oldArch)
+ } else {
+ System.clearProperty("os.arch")
+ }
+
+ if (oldOops != null) {
+ System.setProperty("spark.test.useCompressedOops", oldOops)
+ } else {
+ System.clearProperty("spark.test.useCompressedOops")
+ }
}
}
diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala
index 63bc951858..9c45b3c287 100644
--- a/core/src/test/scala/spark/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala
@@ -1,6 +1,7 @@
package spark
import org.scalatest.FunSuite
+import org.scalatest.BeforeAndAfterAll
class DummyClass1 {}
@@ -17,61 +18,86 @@ class DummyClass4(val d: DummyClass3) {
val x: Int = 0
}
-class SizeEstimatorSuite extends FunSuite {
+class SizeEstimatorSuite extends FunSuite with BeforeAndAfterAll {
+ var oldArch: String = _
+ var oldOops: String = _
+
+ override def beforeAll() {
+ // 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")
+ }
+
+ override def afterAll() {
+ if (oldArch != null) {
+ System.setProperty("os.arch", oldArch)
+ } else {
+ System.clearProperty("os.arch")
+ }
+
+ if (oldOops != null) {
+ System.setProperty("spark.test.useCompressedOops", oldOops)
+ } else {
+ System.clearProperty("spark.test.useCompressedOops")
+ }
+ }
+
test("simple classes") {
- expect(8)(SizeEstimator.estimate(new DummyClass1))
- expect(12)(SizeEstimator.estimate(new DummyClass2))
- expect(20)(SizeEstimator.estimate(new DummyClass3))
- expect(16)(SizeEstimator.estimate(new DummyClass4(null)))
- expect(36)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
+ expect(16)(SizeEstimator.estimate(new DummyClass1))
+ expect(16)(SizeEstimator.estimate(new DummyClass2))
+ expect(24)(SizeEstimator.estimate(new DummyClass3))
+ expect(24)(SizeEstimator.estimate(new DummyClass4(null)))
+ expect(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
}
test("strings") {
- expect(24)(SizeEstimator.estimate(""))
- expect(26)(SizeEstimator.estimate("a"))
- expect(28)(SizeEstimator.estimate("ab"))
- expect(40)(SizeEstimator.estimate("abcdefgh"))
+ expect(48)(SizeEstimator.estimate(""))
+ expect(56)(SizeEstimator.estimate("a"))
+ expect(56)(SizeEstimator.estimate("ab"))
+ expect(64)(SizeEstimator.estimate("abcdefgh"))
}
test("primitive arrays") {
- expect(10)(SizeEstimator.estimate(new Array[Byte](10)))
- expect(20)(SizeEstimator.estimate(new Array[Char](10)))
- expect(20)(SizeEstimator.estimate(new Array[Short](10)))
- expect(40)(SizeEstimator.estimate(new Array[Int](10)))
- expect(80)(SizeEstimator.estimate(new Array[Long](10)))
- expect(40)(SizeEstimator.estimate(new Array[Float](10)))
- expect(80)(SizeEstimator.estimate(new Array[Double](10)))
- expect(4000)(SizeEstimator.estimate(new Array[Int](1000)))
- expect(8000)(SizeEstimator.estimate(new Array[Long](1000)))
+ expect(32)(SizeEstimator.estimate(new Array[Byte](10)))
+ expect(40)(SizeEstimator.estimate(new Array[Char](10)))
+ expect(40)(SizeEstimator.estimate(new Array[Short](10)))
+ expect(56)(SizeEstimator.estimate(new Array[Int](10)))
+ expect(96)(SizeEstimator.estimate(new Array[Long](10)))
+ expect(56)(SizeEstimator.estimate(new Array[Float](10)))
+ expect(96)(SizeEstimator.estimate(new Array[Double](10)))
+ expect(4016)(SizeEstimator.estimate(new Array[Int](1000)))
+ expect(8016)(SizeEstimator.estimate(new Array[Long](1000)))
}
test("object arrays") {
// Arrays containing nulls should just have one pointer per element
- expect(40)(SizeEstimator.estimate(new Array[String](10)))
- expect(40)(SizeEstimator.estimate(new Array[AnyRef](10)))
+ expect(56)(SizeEstimator.estimate(new Array[String](10)))
+ expect(56)(SizeEstimator.estimate(new Array[AnyRef](10)))
// For object arrays with non-null elements, each object should take one pointer plus
// however many bytes that class takes. (Note that Array.fill calls the code in its
// second parameter separately for each object, so we get distinct objects.)
- expect(120)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)))
- expect(160)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)))
- expect(240)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)))
- expect(12 + 16)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)))
+ expect(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)))
+ expect(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)))
+ expect(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)))
+ expect(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)))
// Past size 100, our samples 100 elements, but we should still get the right size.
- expect(24000)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)))
+ expect(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)))
// If an array contains the *same* element many times, we should only count it once.
val d1 = new DummyClass1
- expect(48)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object
- expect(408)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object
+ expect(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object
+ expect(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object
// Same thing with huge array containing the same element many times. Note that this won't
- // return exactly 4008 because it can't tell that *all* the elements will equal the first
+ // return exactly 4032 because it can't tell that *all* the elements will equal the first
// one it samples, but it should be close to that.
+
+ // TODO: If we sample 100 elements, this should always be 4176 ?
val estimatedSize = SizeEstimator.estimate(Array.fill(1000)(d1))
assert(estimatedSize >= 4000, "Estimated size " + estimatedSize + " should be more than 4000")
- assert(estimatedSize <= 4100, "Estimated size " + estimatedSize + " should be less than 4100")
+ assert(estimatedSize <= 4200, "Estimated size " + estimatedSize + " should be less than 4100")
}
}