diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java
index 93bb69b41c7907cf3645e3b310d5200b1d9a3808..3190a43e734866b532831699b2c2d061b101a99c 100644
--- a/core/src/test/scala/spark/JavaAPISuite.java
+++ b/core/src/test/scala/spark/JavaAPISuite.java
@@ -314,6 +314,28 @@ 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));
diff --git a/project/plugins.sbt b/project/plugins.sbt
index d4f244287236343a0d475849bb3d1ff4c3b6d152..25b812a28da35d945575db6807a6bdebf69869ee 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -16,3 +16,5 @@ addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1")
 //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns)
 
 //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6")
+
+libraryDependencies += "com.novocode" % "junit-interface" % "0.10-M4" % "test"