Added a shuffle test with negative hash codes for some keys (this was a bug earlier)

This commit is contained in:
Matei Zaharia 2010-11-12 16:18:45 -08:00
parent d0a9966555
commit f8966ffc11

View file

@ -31,6 +31,17 @@ class ShuffleSuite extends FunSuite {
assert(valuesFor2.toList.sorted === List(1))
}
test("groupByKey with negative key hash codes") {
val sc = new SparkContext("local", "test")
val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1)))
val groups = pairs.groupByKey().collect()
assert(groups.size === 2)
val valuesForMinus1 = groups.find(_._1 == -1).get._2
assert(valuesForMinus1.toList.sorted === List(1, 2, 3))
val valuesFor2 = groups.find(_._1 == 2).get._2
assert(valuesFor2.toList.sorted === List(1))
}
test("groupByKey with many output partitions") {
val sc = new SparkContext("local", "test")
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)))