Move zero-length partition testing from JavaAPISuite.java to PartitioningSuite.scala

This commit is contained in:
Christopher Nguyen 2013-06-16 01:23:48 -07:00
parent 479442a9b9
commit 5c886194e4
2 changed files with 19 additions and 24 deletions

View file

@ -314,28 +314,6 @@ public class JavaAPISuite implements Serializable {
List<Double> take = rdd.take(5);
}
@Test
public void zeroLengthPartitions() {
// Create RDD with some consecutive empty partitions (including the "first" one)
JavaDoubleRDD rdd = sc
.parallelizeDoubles(Arrays.asList(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8)
.filter(new Function<Double, Boolean>() {
@Override
public Boolean call(Double x) {
return x > 0.0;
}
});
// Run the partitions, including the consecutive empty ones, through StatCounter
StatCounter stats = rdd.stats();
Assert.assertEquals(6.0, stats.sum(), 0.01);
Assert.assertEquals(6.0/2, rdd.mean(), 0.01);
Assert.assertEquals(1.0, rdd.variance(), 0.01);
Assert.assertEquals(1.0, rdd.stdev(), 0.01);
// Add other tests here for classes that should be able to handle empty partitions correctly
}
@Test
public void map() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));

View file

@ -1,10 +1,10 @@
package spark
import org.scalatest.FunSuite
import scala.collection.mutable.ArrayBuffer
import SparkContext._
import spark.util.StatCounter
import scala.math._
class PartitioningSuite extends FunSuite with LocalSparkContext {
@ -120,4 +120,21 @@ class PartitioningSuite extends FunSuite with LocalSparkContext {
assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array"))
assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array"))
}
test("Zero-length partitions should be correctly handled") {
// Create RDD with some consecutive empty partitions (including the "first" one)
sc = new SparkContext("local", "test")
val rdd: RDD[Double] = sc
.parallelize(Array(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8)
.filter(_ >= 0.0)
// Run the partitions, including the consecutive empty ones, through StatCounter
val stats: StatCounter = rdd.stats();
assert(abs(6.0 - stats.sum) < 0.01);
assert(abs(6.0/2 - rdd.mean) < 0.01);
assert(abs(1.0 - rdd.variance) < 0.01);
assert(abs(1.0 - rdd.stdev) < 0.01);
// Add other tests here for classes that should be able to handle empty partitions correctly
}
}