You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by sr...@apache.org on 2017/02/16 12:32:52 UTC

[1/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Repository: spark
Updated Branches:
  refs/heads/master 3871d94a6 -> 0e2405490


http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java
new file mode 100644
index 0000000..8d24104
--- /dev/null
+++ b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java
@@ -0,0 +1,2008 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark.streaming;
+
+import java.io.*;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.JavaCheckpointTestUtils;
+import org.apache.spark.streaming.JavaTestUtils;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.Seconds;
+import org.apache.spark.streaming.StreamingContextState;
+import org.apache.spark.streaming.StreamingContextSuite;
+import org.apache.spark.streaming.Time;
+import scala.Tuple2;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.io.Files;
+import com.google.common.collect.Sets;
+
+import org.apache.spark.HashPartitioner;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.Optional;
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.api.java.*;
+import org.apache.spark.util.LongAccumulator;
+import org.apache.spark.util.Utils;
+
+// The test suite itself is Serializable so that anonymous Function implementations can be
+// serialized, as an alternative to converting these anonymous classes to static inner classes;
+// see http://stackoverflow.com/questions/758570/.
+public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable {
+
+  public static void equalIterator(Iterator<?> a, Iterator<?> b) {
+    while (a.hasNext() && b.hasNext()) {
+      Assert.assertEquals(a.next(), b.next());
+    }
+    Assert.assertEquals(a.hasNext(), b.hasNext());
+  }
+
+  public static void equalIterable(Iterable<?> a, Iterable<?> b) {
+      equalIterator(a.iterator(), b.iterator());
+  }
+
+  @Test
+  public void testInitialization() {
+    Assert.assertNotNull(ssc.sparkContext());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testContextState() {
+    List<List<Integer>> inputData = Arrays.asList(Arrays.asList(1, 2, 3, 4));
+    Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState());
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaTestUtils.attachTestOutputStream(stream);
+    Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState());
+    ssc.start();
+    Assert.assertEquals(StreamingContextState.ACTIVE, ssc.getState());
+    ssc.stop();
+    Assert.assertEquals(StreamingContextState.STOPPED, ssc.getState());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCount() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3,4),
+        Arrays.asList(3,4,5),
+        Arrays.asList(3));
+
+    List<List<Long>> expected = Arrays.asList(
+        Arrays.asList(4L),
+        Arrays.asList(3L),
+        Arrays.asList(1L));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Long> count = stream.count();
+    JavaTestUtils.attachTestOutputStream(count);
+    List<List<Long>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testMap() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("hello", "world"),
+        Arrays.asList("goodnight", "moon"));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(5,5),
+        Arrays.asList(9,4));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
+        @Override
+        public Integer call(String s) {
+          return s.length();
+        }
+    });
+    JavaTestUtils.attachTestOutputStream(letterCount);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testWindow() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6,1,2,3),
+        Arrays.asList(7,8,9,4,5,6),
+        Arrays.asList(7,8,9));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> windowed = stream.window(new Duration(2000));
+    JavaTestUtils.attachTestOutputStream(windowed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testWindowWithSlideDuration() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9),
+        Arrays.asList(10,11,12),
+        Arrays.asList(13,14,15),
+        Arrays.asList(16,17,18));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,2,3,4,5,6),
+        Arrays.asList(1,2,3,4,5,6,7,8,9,10,11,12),
+        Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18),
+        Arrays.asList(13,14,15,16,17,18));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> windowed = stream.window(new Duration(4000), new Duration(2000));
+    JavaTestUtils.attachTestOutputStream(windowed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 8, 4);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testFilter() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red sox"));
+
+    List<List<String>> expected = Arrays.asList(
+        Arrays.asList("giants"),
+        Arrays.asList("yankees"));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<String> filtered = stream.filter(new Function<String, Boolean>() {
+      @Override
+      public Boolean call(String s) {
+        return s.contains("a");
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(filtered);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRepartitionMorePartitions() {
+    List<List<Integer>> inputData = Arrays.asList(
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
+    JavaDStream<Integer> stream =
+        JavaTestUtils.attachTestInputStream(ssc, inputData, 2);
+    JavaDStreamLike<Integer,JavaDStream<Integer>,JavaRDD<Integer>> repartitioned =
+        stream.repartition(4);
+    JavaTestUtils.attachTestOutputStream(repartitioned);
+    List<List<List<Integer>>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2);
+    Assert.assertEquals(2, result.size());
+    for (List<List<Integer>> rdd : result) {
+      Assert.assertEquals(4, rdd.size());
+      Assert.assertEquals(
+        10, rdd.get(0).size() + rdd.get(1).size() + rdd.get(2).size() + rdd.get(3).size());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRepartitionFewerPartitions() {
+    List<List<Integer>> inputData = Arrays.asList(
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
+      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
+    JavaDStream<Integer> stream =
+        JavaTestUtils.attachTestInputStream(ssc, inputData, 4);
+    JavaDStreamLike<Integer,JavaDStream<Integer>,JavaRDD<Integer>> repartitioned =
+        stream.repartition(2);
+    JavaTestUtils.attachTestOutputStream(repartitioned);
+    List<List<List<Integer>>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2);
+    Assert.assertEquals(2, result.size());
+    for (List<List<Integer>> rdd : result) {
+      Assert.assertEquals(2, rdd.size());
+      Assert.assertEquals(10, rdd.get(0).size() + rdd.get(1).size());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testGlom() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red sox"));
+
+    List<List<List<String>>> expected = Arrays.asList(
+        Arrays.asList(Arrays.asList("giants", "dodgers")),
+        Arrays.asList(Arrays.asList("yankees", "red sox")));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<List<String>> glommed = stream.glom();
+    JavaTestUtils.attachTestOutputStream(glommed);
+    List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testMapPartitions() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red sox"));
+
+    List<List<String>> expected = Arrays.asList(
+        Arrays.asList("GIANTSDODGERS"),
+        Arrays.asList("YANKEESRED SOX"));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<String> mapped = stream.mapPartitions(
+        new FlatMapFunction<Iterator<String>, String>() {
+          @Override
+          public Iterator<String> call(Iterator<String> in) {
+            StringBuilder out = new StringBuilder();
+            while (in.hasNext()) {
+              out.append(in.next().toUpperCase(Locale.ENGLISH));
+            }
+            return Arrays.asList(out.toString()).iterator();
+          }
+        });
+    JavaTestUtils.attachTestOutputStream(mapped);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  private static class IntegerSum implements Function2<Integer, Integer, Integer> {
+    @Override
+    public Integer call(Integer i1, Integer i2) {
+      return i1 + i2;
+    }
+  }
+
+  private static class IntegerDifference implements Function2<Integer, Integer, Integer> {
+    @Override
+    public Integer call(Integer i1, Integer i2) {
+      return i1 - i2;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testReduce() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(6),
+        Arrays.asList(15),
+        Arrays.asList(24));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> reduced = stream.reduce(new IntegerSum());
+    JavaTestUtils.attachTestOutputStream(reduced);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testReduceByWindowWithInverse() {
+    testReduceByWindow(true);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testReduceByWindowWithoutInverse() {
+    testReduceByWindow(false);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void testReduceByWindow(boolean withInverse) {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(6),
+        Arrays.asList(21),
+        Arrays.asList(39),
+        Arrays.asList(24));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> reducedWindowed;
+    if (withInverse) {
+      reducedWindowed = stream.reduceByWindow(new IntegerSum(),
+                                              new IntegerDifference(),
+                                              new Duration(2000),
+                                              new Duration(1000));
+    } else {
+      reducedWindowed = stream.reduceByWindow(new IntegerSum(),
+                                              new Duration(2000), new Duration(1000));
+    }
+    JavaTestUtils.attachTestOutputStream(reducedWindowed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testQueueStream() {
+    ssc.stop();
+    // Create a new JavaStreamingContext without checkpointing
+    SparkConf conf = new SparkConf()
+        .setMaster("local[2]")
+        .setAppName("test")
+        .set("spark.streaming.clock", "org.apache.spark.util.ManualClock");
+    ssc = new JavaStreamingContext(conf, new Duration(1000));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc());
+    JavaRDD<Integer> rdd1 = jsc.parallelize(Arrays.asList(1, 2, 3));
+    JavaRDD<Integer> rdd2 = jsc.parallelize(Arrays.asList(4, 5, 6));
+    JavaRDD<Integer> rdd3 = jsc.parallelize(Arrays.asList(7,8,9));
+
+    Queue<JavaRDD<Integer>> rdds = new LinkedList<>();
+    rdds.add(rdd1);
+    rdds.add(rdd2);
+    rdds.add(rdd3);
+
+    JavaDStream<Integer> stream = ssc.queueStream(rdds);
+    JavaTestUtils.attachTestOutputStream(stream);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testTransform() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(3,4,5),
+        Arrays.asList(6,7,8),
+        Arrays.asList(9,10,11));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> transformed = stream.transform(
+      new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
+        @Override
+        public JavaRDD<Integer> call(JavaRDD<Integer> in) {
+          return in.map(new Function<Integer, Integer>() {
+            @Override
+            public Integer call(Integer i) {
+              return i + 2;
+            }
+          });
+        }
+      });
+
+    JavaTestUtils.attachTestOutputStream(transformed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testVariousTransform() {
+    // tests whether all variations of transform can be called from Java
+
+    List<List<Integer>> inputData = Arrays.asList(Arrays.asList(1));
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+
+    List<List<Tuple2<String, Integer>>> pairInputData =
+        Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1));
+
+    stream.transform(
+        new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
+          @Override
+          public JavaRDD<Integer> call(JavaRDD<Integer> in) {
+            return null;
+          }
+        }
+    );
+
+    stream.transform(
+      new Function2<JavaRDD<Integer>, Time, JavaRDD<Integer>>() {
+        @Override public JavaRDD<Integer> call(JavaRDD<Integer> in, Time time) {
+          return null;
+        }
+      }
+    );
+
+    stream.transformToPair(
+        new Function<JavaRDD<Integer>, JavaPairRDD<String, Integer>>() {
+          @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in) {
+            return null;
+          }
+        }
+    );
+
+    stream.transformToPair(
+        new Function2<JavaRDD<Integer>, Time, JavaPairRDD<String, Integer>>() {
+          @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in, Time time) {
+            return null;
+          }
+        }
+    );
+
+    pairStream.transform(
+        new Function<JavaPairRDD<String, Integer>, JavaRDD<Integer>>() {
+          @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in) {
+            return null;
+          }
+        }
+    );
+
+    pairStream.transform(
+        new Function2<JavaPairRDD<String, Integer>, Time, JavaRDD<Integer>>() {
+          @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in, Time time) {
+            return null;
+          }
+        }
+    );
+
+    pairStream.transformToPair(
+        new Function<JavaPairRDD<String, Integer>, JavaPairRDD<String, String>>() {
+          @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in) {
+            return null;
+          }
+        }
+    );
+
+    pairStream.transformToPair(
+        new Function2<JavaPairRDD<String, Integer>, Time, JavaPairRDD<String, String>>() {
+          @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in,
+                                                            Time time) {
+            return null;
+          }
+        }
+    );
+
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testTransformWith() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("california", "dodgers"),
+            new Tuple2<>("new york", "yankees")),
+        Arrays.asList(
+            new Tuple2<>("california", "sharks"),
+            new Tuple2<>("new york", "rangers")));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("california", "giants"),
+            new Tuple2<>("new york", "mets")),
+        Arrays.asList(
+            new Tuple2<>("california", "ducks"),
+            new Tuple2<>("new york", "islanders")));
+
+
+    List<HashSet<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
+        Sets.newHashSet(
+            new Tuple2<>("california",
+                         new Tuple2<>("dodgers", "giants")),
+            new Tuple2<>("new york",
+                         new Tuple2<>("yankees", "mets"))),
+        Sets.newHashSet(
+            new Tuple2<>("california",
+                         new Tuple2<>("sharks", "ducks")),
+            new Tuple2<>("new york",
+                         new Tuple2<>("rangers", "islanders"))));
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.transformWithToPair(
+        pairStream2,
+        new Function3<
+            JavaPairRDD<String, String>,
+            JavaPairRDD<String, String>,
+            Time,
+            JavaPairRDD<String, Tuple2<String, String>>>() {
+          @Override
+          public JavaPairRDD<String, Tuple2<String, String>> call(
+              JavaPairRDD<String, String> rdd1,
+              JavaPairRDD<String, String> rdd2,
+              Time time) {
+            return rdd1.join(rdd2);
+          }
+        }
+    );
+
+    JavaTestUtils.attachTestOutputStream(joined);
+    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+    List<HashSet<Tuple2<String, Tuple2<String, String>>>> unorderedResult = new ArrayList<>();
+    for (List<Tuple2<String, Tuple2<String, String>>> res: result) {
+      unorderedResult.add(Sets.newHashSet(res));
+    }
+
+    Assert.assertEquals(expected, unorderedResult);
+  }
+
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testVariousTransformWith() {
+    // tests whether all variations of transformWith can be called from Java
+
+    List<List<Integer>> inputData1 = Arrays.asList(Arrays.asList(1));
+    List<List<String>> inputData2 = Arrays.asList(Arrays.asList("x"));
+    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1);
+    JavaDStream<String> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1);
+
+    List<List<Tuple2<String, Integer>>> pairInputData1 =
+        Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
+    List<List<Tuple2<Double, Character>>> pairInputData2 =
+        Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x')));
+    JavaPairDStream<String, Integer> pairStream1 = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1));
+    JavaPairDStream<Double, Character> pairStream2 = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1));
+
+    stream1.transformWith(
+        stream2,
+        new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2, Time time) {
+            return null;
+          }
+        }
+    );
+
+    stream1.transformWith(
+        pairStream1,
+        new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time, JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaPairRDD<String, Integer> rdd2,
+                                      Time time) {
+            return null;
+          }
+        }
+    );
+
+    stream1.transformWithToPair(
+        stream2,
+        new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2,
+                                                  Time time) {
+            return null;
+          }
+        }
+    );
+
+    stream1.transformWithToPair(
+        pairStream1,
+        new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time,
+          JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1,
+                                                  JavaPairRDD<String, Integer> rdd2,
+                                                  Time time) {
+            return null;
+          }
+        }
+    );
+
+    pairStream1.transformWith(
+        stream2,
+        new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, JavaRDD<String> rdd2,
+                                      Time time) {
+            return null;
+          }
+        }
+    );
+
+    pairStream1.transformWith(
+        pairStream1,
+        new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<String, Integer>, Time,
+          JavaRDD<Double>>() {
+          @Override
+          public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1,
+                                      JavaPairRDD<String, Integer> rdd2,
+                                      Time time) {
+            return null;
+          }
+        }
+    );
+
+    pairStream1.transformWithToPair(
+        stream2,
+        new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time,
+          JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1,
+                                                  JavaRDD<String> rdd2,
+                                                  Time time) {
+            return null;
+          }
+        }
+    );
+
+    pairStream1.transformWithToPair(
+        pairStream2,
+        new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<Double, Character>, Time,
+          JavaPairRDD<Double, Double>>() {
+          @Override
+          public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1,
+                                                  JavaPairRDD<Double, Character> rdd2,
+                                                  Time time) {
+            return null;
+          }
+        }
+    );
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testStreamingContextTransform(){
+    List<List<Integer>> stream1input = Arrays.asList(
+        Arrays.asList(1),
+        Arrays.asList(2)
+    );
+
+    List<List<Integer>> stream2input = Arrays.asList(
+        Arrays.asList(3),
+        Arrays.asList(4)
+    );
+
+    List<List<Tuple2<Integer, String>>> pairStream1input = Arrays.asList(
+        Arrays.asList(new Tuple2<>(1, "x")),
+        Arrays.asList(new Tuple2<>(2, "y"))
+    );
+
+    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))),
+        Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y")))
+    );
+
+    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1);
+    JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1);
+    JavaPairDStream<Integer, String> pairStream1 = JavaPairDStream.fromJavaDStream(
+        JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1));
+
+    List<JavaDStream<?>> listOfDStreams1 = Arrays.<JavaDStream<?>>asList(stream1, stream2);
+
+    // This is just to test whether this transform to JavaStream compiles
+    ssc.transform(
+      listOfDStreams1,
+      new Function2<List<JavaRDD<?>>, Time, JavaRDD<Long>>() {
+        @Override
+        public JavaRDD<Long> call(List<JavaRDD<?>> listOfRDDs, Time time) {
+          Assert.assertEquals(2, listOfRDDs.size());
+          return null;
+        }
+      }
+    );
+
+    List<JavaDStream<?>> listOfDStreams2 =
+        Arrays.<JavaDStream<?>>asList(stream1, stream2, pairStream1.toJavaDStream());
+
+    JavaPairDStream<Integer, Tuple2<Integer, String>> transformed2 = ssc.transformToPair(
+      listOfDStreams2,
+      new Function2<List<JavaRDD<?>>, Time, JavaPairRDD<Integer, Tuple2<Integer, String>>>() {
+        @Override
+        public JavaPairRDD<Integer, Tuple2<Integer, String>> call(List<JavaRDD<?>> listOfRDDs,
+                                                                  Time time) {
+          Assert.assertEquals(3, listOfRDDs.size());
+          JavaRDD<Integer> rdd1 = (JavaRDD<Integer>)listOfRDDs.get(0);
+          JavaRDD<Integer> rdd2 = (JavaRDD<Integer>)listOfRDDs.get(1);
+          JavaRDD<Tuple2<Integer, String>> rdd3 =
+            (JavaRDD<Tuple2<Integer, String>>)listOfRDDs.get(2);
+          JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3);
+          PairFunction<Integer, Integer, Integer> mapToTuple =
+            new PairFunction<Integer, Integer, Integer>() {
+            @Override
+            public Tuple2<Integer, Integer> call(Integer i) {
+              return new Tuple2<>(i, i);
+            }
+          };
+          return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3);
+        }
+      }
+    );
+    JavaTestUtils.attachTestOutputStream(transformed2);
+    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> result =
+      JavaTestUtils.runStreams(ssc, 2, 2);
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testFlatMap() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("go", "giants"),
+        Arrays.asList("boo", "dodgers"),
+        Arrays.asList("athletics"));
+
+    List<List<String>> expected = Arrays.asList(
+        Arrays.asList("g","o","g","i","a","n","t","s"),
+        Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"),
+        Arrays.asList("a","t","h","l","e","t","i","c","s"));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<String> flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
+      @Override
+      public Iterator<String> call(String x) {
+        return Arrays.asList(x.split("(?!^)")).iterator();
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testForeachRDD() {
+    final LongAccumulator accumRdd = ssc.sparkContext().sc().longAccumulator();
+    final LongAccumulator accumEle = ssc.sparkContext().sc().longAccumulator();
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,1,1),
+        Arrays.asList(1,1,1));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaTestUtils.attachTestOutputStream(stream.count()); // dummy output
+
+    stream.foreachRDD(new VoidFunction<JavaRDD<Integer>>() {
+      @Override
+      public void call(JavaRDD<Integer> rdd) {
+        accumRdd.add(1);
+        rdd.foreach(new VoidFunction<Integer>() {
+          @Override
+          public void call(Integer i) {
+            accumEle.add(1);
+          }
+        });
+      }
+    });
+
+    // This is a test to make sure foreachRDD(VoidFunction2) can be called from Java
+    stream.foreachRDD(new VoidFunction2<JavaRDD<Integer>, Time>() {
+      @Override
+      public void call(JavaRDD<Integer> rdd, Time time) {
+      }
+    });
+
+    JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(2, accumRdd.value().intValue());
+    Assert.assertEquals(6, accumEle.value().intValue());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairFlatMap() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants"),
+        Arrays.asList("dodgers"),
+        Arrays.asList("athletics"));
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>(6, "g"),
+            new Tuple2<>(6, "i"),
+            new Tuple2<>(6, "a"),
+            new Tuple2<>(6, "n"),
+            new Tuple2<>(6, "t"),
+            new Tuple2<>(6, "s")),
+        Arrays.asList(
+            new Tuple2<>(7, "d"),
+            new Tuple2<>(7, "o"),
+            new Tuple2<>(7, "d"),
+            new Tuple2<>(7, "g"),
+            new Tuple2<>(7, "e"),
+            new Tuple2<>(7, "r"),
+            new Tuple2<>(7, "s")),
+        Arrays.asList(
+            new Tuple2<>(9, "a"),
+            new Tuple2<>(9, "t"),
+            new Tuple2<>(9, "h"),
+            new Tuple2<>(9, "l"),
+            new Tuple2<>(9, "e"),
+            new Tuple2<>(9, "t"),
+            new Tuple2<>(9, "i"),
+            new Tuple2<>(9, "c"),
+            new Tuple2<>(9, "s")));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<Integer, String> flatMapped = stream.flatMapToPair(
+      new PairFlatMapFunction<String, Integer, String>() {
+        @Override
+        public Iterator<Tuple2<Integer, String>> call(String in) {
+          List<Tuple2<Integer, String>> out = new ArrayList<>();
+          for (String letter: in.split("(?!^)")) {
+            out.add(new Tuple2<>(in.length(), letter));
+          }
+          return out.iterator();
+        }
+      });
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testUnion() {
+    List<List<Integer>> inputData1 = Arrays.asList(
+        Arrays.asList(1,1),
+        Arrays.asList(2,2),
+        Arrays.asList(3,3));
+
+    List<List<Integer>> inputData2 = Arrays.asList(
+        Arrays.asList(4,4),
+        Arrays.asList(5,5),
+        Arrays.asList(6,6));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,1,4,4),
+        Arrays.asList(2,2,5,5),
+        Arrays.asList(3,3,6,6));
+
+    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
+    JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
+
+    JavaDStream<Integer> unioned = stream1.union(stream2);
+    JavaTestUtils.attachTestOutputStream(unioned);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  /*
+   * Performs an order-invariant comparison of lists representing two RDD streams. This allows
+   * us to account for ordering variation within individual RDD's which occurs during windowing.
+   */
+  public static <T> void assertOrderInvariantEquals(
+      List<List<T>> expected, List<List<T>> actual) {
+    List<Set<T>> expectedSets = new ArrayList<>();
+    for (List<T> list: expected) {
+      expectedSets.add(Collections.unmodifiableSet(new HashSet<>(list)));
+    }
+    List<Set<T>> actualSets = new ArrayList<>();
+    for (List<T> list: actual) {
+      actualSets.add(Collections.unmodifiableSet(new HashSet<>(list)));
+    }
+    Assert.assertEquals(expectedSets, actualSets);
+  }
+
+
+  // PairDStream Functions
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairFilter() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red sox"));
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>("giants", 6)),
+        Arrays.asList(new Tuple2<>("yankees", 7)));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = stream.mapToPair(
+        new PairFunction<String, String, Integer>() {
+          @Override
+          public Tuple2<String, Integer> call(String in) {
+            return new Tuple2<>(in, in.length());
+          }
+        });
+
+    JavaPairDStream<String, Integer> filtered = pairStream.filter(
+        new Function<Tuple2<String, Integer>, Boolean>() {
+      @Override
+      public Boolean call(Tuple2<String, Integer> in) {
+        return in._1().contains("a");
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(filtered);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  private final List<List<Tuple2<String, String>>> stringStringKVStream = Arrays.asList(
+      Arrays.asList(new Tuple2<>("california", "dodgers"),
+                    new Tuple2<>("california", "giants"),
+                    new Tuple2<>("new york", "yankees"),
+                    new Tuple2<>("new york", "mets")),
+      Arrays.asList(new Tuple2<>("california", "sharks"),
+                    new Tuple2<>("california", "ducks"),
+                    new Tuple2<>("new york", "rangers"),
+                    new Tuple2<>("new york", "islanders")));
+
+  @SuppressWarnings("unchecked")
+  private final List<List<Tuple2<String, Integer>>> stringIntKVStream = Arrays.asList(
+      Arrays.asList(
+          new Tuple2<>("california", 1),
+          new Tuple2<>("california", 3),
+          new Tuple2<>("new york", 4),
+          new Tuple2<>("new york", 1)),
+      Arrays.asList(
+          new Tuple2<>("california", 5),
+          new Tuple2<>("california", 5),
+          new Tuple2<>("new york", 3),
+          new Tuple2<>("new york", 1)));
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairMap() { // Maps pair -> pair of different type
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>(1, "california"),
+            new Tuple2<>(3, "california"),
+            new Tuple2<>(4, "new york"),
+            new Tuple2<>(1, "new york")),
+        Arrays.asList(
+            new Tuple2<>(5, "california"),
+            new Tuple2<>(5, "california"),
+            new Tuple2<>(3, "new york"),
+            new Tuple2<>(1, "new york")));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> reversed = pairStream.mapToPair(
+        new PairFunction<Tuple2<String, Integer>, Integer, String>() {
+          @Override
+          public Tuple2<Integer, String> call(Tuple2<String, Integer> in) {
+            return in.swap();
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairMapPartitions() { // Maps pair -> pair of different type
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>(1, "california"),
+            new Tuple2<>(3, "california"),
+            new Tuple2<>(4, "new york"),
+            new Tuple2<>(1, "new york")),
+        Arrays.asList(
+            new Tuple2<>(5, "california"),
+            new Tuple2<>(5, "california"),
+            new Tuple2<>(3, "new york"),
+            new Tuple2<>(1, "new york")));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> reversed = pairStream.mapPartitionsToPair(
+        new PairFlatMapFunction<Iterator<Tuple2<String, Integer>>, Integer, String>() {
+          @Override
+          public Iterator<Tuple2<Integer, String>> call(Iterator<Tuple2<String, Integer>> in) {
+            List<Tuple2<Integer, String>> out = new LinkedList<>();
+            while (in.hasNext()) {
+              Tuple2<String, Integer> next = in.next();
+              out.add(next.swap());
+            }
+            return out.iterator();
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairMap2() { // Maps pair -> single
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Integer>> expected = Arrays.asList(
+            Arrays.asList(1, 3, 4, 1),
+            Arrays.asList(5, 5, 3, 1));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaDStream<Integer> reversed = pairStream.map(
+        new Function<Tuple2<String, Integer>, Integer>() {
+          @Override
+          public Integer call(Tuple2<String, Integer> in) {
+            return in._2();
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair
+    List<List<Tuple2<String, Integer>>> inputData = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("hi", 1),
+            new Tuple2<>("ho", 2)),
+        Arrays.asList(
+            new Tuple2<>("hi", 1),
+            new Tuple2<>("ho", 2)));
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>(1, "h"),
+            new Tuple2<>(1, "i"),
+            new Tuple2<>(2, "h"),
+            new Tuple2<>(2, "o")),
+        Arrays.asList(
+            new Tuple2<>(1, "h"),
+            new Tuple2<>(1, "i"),
+            new Tuple2<>(2, "h"),
+            new Tuple2<>(2, "o")));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+        JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> flatMapped = pairStream.flatMapToPair(
+        new PairFlatMapFunction<Tuple2<String, Integer>, Integer, String>() {
+          @Override
+          public Iterator<Tuple2<Integer, String>> call(Tuple2<String, Integer> in) {
+            List<Tuple2<Integer, String>> out = new LinkedList<>();
+            for (Character s : in._1().toCharArray()) {
+              out.add(new Tuple2<>(in._2(), s.toString()));
+            }
+            return out.iterator();
+          }
+        });
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairGroupByKey() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, List<String>>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("california", Arrays.asList("dodgers", "giants")),
+            new Tuple2<>("new york", Arrays.asList("yankees", "mets"))),
+        Arrays.asList(
+            new Tuple2<>("california", Arrays.asList("sharks", "ducks")),
+            new Tuple2<>("new york", Arrays.asList("rangers", "islanders"))));
+
+    JavaDStream<Tuple2<String, String>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Iterable<String>> grouped = pairStream.groupByKey();
+    JavaTestUtils.attachTestOutputStream(grouped);
+    List<List<Tuple2<String, Iterable<String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected.size(), result.size());
+    Iterator<List<Tuple2<String, Iterable<String>>>> resultItr = result.iterator();
+    Iterator<List<Tuple2<String, List<String>>>> expectedItr = expected.iterator();
+    while (resultItr.hasNext() && expectedItr.hasNext()) {
+      Iterator<Tuple2<String, Iterable<String>>> resultElements = resultItr.next().iterator();
+      Iterator<Tuple2<String, List<String>>> expectedElements = expectedItr.next().iterator();
+      while (resultElements.hasNext() && expectedElements.hasNext()) {
+        Tuple2<String, Iterable<String>> resultElement = resultElements.next();
+        Tuple2<String, List<String>> expectedElement = expectedElements.next();
+        Assert.assertEquals(expectedElement._1(), resultElement._1());
+        equalIterable(expectedElement._2(), resultElement._2());
+      }
+      Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairReduceByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("california", 4),
+            new Tuple2<>("new york", 5)),
+        Arrays.asList(
+            new Tuple2<>("california", 10),
+            new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduced = pairStream.reduceByKey(new IntegerSum());
+
+    JavaTestUtils.attachTestOutputStream(reduced);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCombineByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("california", 4),
+            new Tuple2<>("new york", 5)),
+        Arrays.asList(
+            new Tuple2<>("california", 10),
+            new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> combined = pairStream.combineByKey(
+        new Function<Integer, Integer>() {
+          @Override
+          public Integer call(Integer i) {
+            return i;
+          }
+        }, new IntegerSum(), new IntegerSum(), new HashPartitioner(2));
+
+    JavaTestUtils.attachTestOutputStream(combined);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCountByValue() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("hello", "world"),
+      Arrays.asList("hello", "moon"),
+      Arrays.asList("hello"));
+
+    List<List<Tuple2<String, Long>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("hello", 1L),
+            new Tuple2<>("world", 1L)),
+        Arrays.asList(
+            new Tuple2<>("hello", 1L),
+            new Tuple2<>("moon", 1L)),
+        Arrays.asList(
+            new Tuple2<>("hello", 1L)));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Long> counted = stream.countByValue();
+    JavaTestUtils.attachTestOutputStream(counted);
+    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testGroupByKeyAndWindow() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, List<Integer>>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>("california", Arrays.asList(1, 3)),
+        new Tuple2<>("new york", Arrays.asList(1, 4))
+      ),
+      Arrays.asList(
+        new Tuple2<>("california", Arrays.asList(1, 3, 5, 5)),
+        new Tuple2<>("new york", Arrays.asList(1, 1, 3, 4))
+      ),
+      Arrays.asList(
+        new Tuple2<>("california", Arrays.asList(5, 5)),
+        new Tuple2<>("new york", Arrays.asList(1, 3))
+      )
+    );
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Iterable<Integer>> groupWindowed =
+        pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(groupWindowed);
+    List<List<Tuple2<String, List<Integer>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected.size(), result.size());
+    for (int i = 0; i < result.size(); i++) {
+      Assert.assertEquals(convert(expected.get(i)), convert(result.get(i)));
+    }
+  }
+
+  private static Set<Tuple2<String, HashSet<Integer>>>
+    convert(List<Tuple2<String, List<Integer>>> listOfTuples) {
+    List<Tuple2<String, HashSet<Integer>>> newListOfTuples = new ArrayList<>();
+    for (Tuple2<String, List<Integer>> tuple: listOfTuples) {
+      newListOfTuples.add(convert(tuple));
+    }
+    return new HashSet<>(newListOfTuples);
+  }
+
+  private static Tuple2<String, HashSet<Integer>> convert(Tuple2<String, List<Integer>> tuple) {
+    return new Tuple2<>(tuple._1(), new HashSet<>(tuple._2()));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testReduceByKeyAndWindow() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", 4),
+                      new Tuple2<>("new york", 5)),
+        Arrays.asList(new Tuple2<>("california", 14),
+                      new Tuple2<>("new york", 9)),
+        Arrays.asList(new Tuple2<>("california", 10),
+                      new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduceWindowed =
+        pairStream.reduceByKeyAndWindow(new IntegerSum(), new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reduceWindowed);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testUpdateStateByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", 4),
+                      new Tuple2<>("new york", 5)),
+        Arrays.asList(new Tuple2<>("california", 14),
+                      new Tuple2<>("new york", 9)),
+        Arrays.asList(new Tuple2<>("california", 14),
+                      new Tuple2<>("new york", 9)));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey(
+        new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
+          @Override
+          public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
+            int out = 0;
+            if (state.isPresent()) {
+              out += state.get();
+            }
+            for (Integer v : values) {
+              out += v;
+            }
+            return Optional.of(out);
+          }
+        });
+    JavaTestUtils.attachTestOutputStream(updated);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testUpdateStateByKeyWithInitial() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<Tuple2<String, Integer>> initial = Arrays.asList(
+        new Tuple2<>("california", 1),
+            new Tuple2<>("new york", 2));
+
+    JavaRDD<Tuple2<String, Integer>> tmpRDD = ssc.sparkContext().parallelize(initial);
+    JavaPairRDD<String, Integer> initialRDD = JavaPairRDD.fromJavaRDD(tmpRDD);
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", 5),
+                      new Tuple2<>("new york", 7)),
+        Arrays.asList(new Tuple2<>("california", 15),
+                      new Tuple2<>("new york", 11)),
+        Arrays.asList(new Tuple2<>("california", 15),
+                      new Tuple2<>("new york", 11)));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey(
+        new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
+          @Override
+          public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
+            int out = 0;
+            if (state.isPresent()) {
+              out += state.get();
+            }
+            for (Integer v : values) {
+              out += v;
+            }
+            return Optional.of(out);
+          }
+        }, new HashPartitioner(1), initialRDD);
+    JavaTestUtils.attachTestOutputStream(updated);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testReduceByKeyAndWindowWithInverse() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", 4),
+                      new Tuple2<>("new york", 5)),
+        Arrays.asList(new Tuple2<>("california", 14),
+                      new Tuple2<>("new york", 9)),
+        Arrays.asList(new Tuple2<>("california", 10),
+                      new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduceWindowed =
+        pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(),
+                                        new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reduceWindowed);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCountByValueAndWindow() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("hello", "world"),
+        Arrays.asList("hello", "moon"),
+        Arrays.asList("hello"));
+
+    List<HashSet<Tuple2<String, Long>>> expected = Arrays.asList(
+        Sets.newHashSet(
+            new Tuple2<>("hello", 1L),
+            new Tuple2<>("world", 1L)),
+        Sets.newHashSet(
+            new Tuple2<>("hello", 2L),
+            new Tuple2<>("world", 1L),
+            new Tuple2<>("moon", 1L)),
+        Sets.newHashSet(
+            new Tuple2<>("hello", 2L),
+            new Tuple2<>("moon", 1L)));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, Long> counted =
+      stream.countByValueAndWindow(new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(counted);
+    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+    List<Set<Tuple2<String, Long>>> unorderedResult = new ArrayList<>();
+    for (List<Tuple2<String, Long>> res: result) {
+      unorderedResult.add(Sets.newHashSet(res));
+    }
+
+    Assert.assertEquals(expected, unorderedResult);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairTransform() {
+    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>(3, 5),
+            new Tuple2<>(1, 5),
+            new Tuple2<>(4, 5),
+            new Tuple2<>(2, 5)),
+        Arrays.asList(
+            new Tuple2<>(2, 5),
+            new Tuple2<>(3, 5),
+            new Tuple2<>(4, 5),
+            new Tuple2<>(1, 5)));
+
+    List<List<Tuple2<Integer, Integer>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>(1, 5),
+            new Tuple2<>(2, 5),
+            new Tuple2<>(3, 5),
+            new Tuple2<>(4, 5)),
+        Arrays.asList(
+            new Tuple2<>(1, 5),
+            new Tuple2<>(2, 5),
+            new Tuple2<>(3, 5),
+            new Tuple2<>(4, 5)));
+
+    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<Integer, Integer> sorted = pairStream.transformToPair(
+        new Function<JavaPairRDD<Integer, Integer>, JavaPairRDD<Integer, Integer>>() {
+          @Override
+          public JavaPairRDD<Integer, Integer> call(JavaPairRDD<Integer, Integer> in) {
+            return in.sortByKey();
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(sorted);
+    List<List<Tuple2<Integer, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPairToNormalRDDTransform() {
+    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>(3, 5),
+            new Tuple2<>(1, 5),
+            new Tuple2<>(4, 5),
+            new Tuple2<>(2, 5)),
+        Arrays.asList(
+            new Tuple2<>(2, 5),
+            new Tuple2<>(3, 5),
+            new Tuple2<>(4, 5),
+            new Tuple2<>(1, 5)));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(3,1,4,2),
+        Arrays.asList(2,3,4,1));
+
+    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaDStream<Integer> firstParts = pairStream.transform(
+        new Function<JavaPairRDD<Integer, Integer>, JavaRDD<Integer>>() {
+          @Override
+          public JavaRDD<Integer> call(JavaPairRDD<Integer, Integer> in) {
+            return in.map(new Function<Tuple2<Integer, Integer>, Integer>() {
+              @Override
+              public Integer call(Tuple2<Integer, Integer> in2) {
+                return in2._1();
+              }
+            });
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(firstParts);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testMapValues() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, String>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "DODGERS"),
+                      new Tuple2<>("california", "GIANTS"),
+                      new Tuple2<>("new york", "YANKEES"),
+                      new Tuple2<>("new york", "METS")),
+        Arrays.asList(new Tuple2<>("california", "SHARKS"),
+                      new Tuple2<>("california", "DUCKS"),
+                      new Tuple2<>("new york", "RANGERS"),
+                      new Tuple2<>("new york", "ISLANDERS")));
+
+    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, String> mapped = pairStream.mapValues(new Function<String, String>() {
+      @Override
+      public String call(String s) {
+        return s.toUpperCase(Locale.ENGLISH);
+      }
+    });
+
+    JavaTestUtils.attachTestOutputStream(mapped);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testFlatMapValues() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, String>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "dodgers1"),
+                      new Tuple2<>("california", "dodgers2"),
+                      new Tuple2<>("california", "giants1"),
+                      new Tuple2<>("california", "giants2"),
+                      new Tuple2<>("new york", "yankees1"),
+                      new Tuple2<>("new york", "yankees2"),
+                      new Tuple2<>("new york", "mets1"),
+                      new Tuple2<>("new york", "mets2")),
+        Arrays.asList(new Tuple2<>("california", "sharks1"),
+                      new Tuple2<>("california", "sharks2"),
+                      new Tuple2<>("california", "ducks1"),
+                      new Tuple2<>("california", "ducks2"),
+                      new Tuple2<>("new york", "rangers1"),
+                      new Tuple2<>("new york", "rangers2"),
+                      new Tuple2<>("new york", "islanders1"),
+                      new Tuple2<>("new york", "islanders2")));
+
+    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+
+    JavaPairDStream<String, String> flatMapped = pairStream.flatMapValues(
+        new Function<String, Iterable<String>>() {
+          @Override
+          public Iterable<String> call(String in) {
+            List<String> out = new ArrayList<>();
+            out.add(in + "1");
+            out.add(in + "2");
+            return out;
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCoGroup() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "dodgers"),
+                      new Tuple2<>("new york", "yankees")),
+        Arrays.asList(new Tuple2<>("california", "sharks"),
+                      new Tuple2<>("new york", "rangers")));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "giants"),
+                      new Tuple2<>("new york", "mets")),
+        Arrays.asList(new Tuple2<>("california", "ducks"),
+                      new Tuple2<>("new york", "islanders")));
+
+
+    List<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("california",
+                         new Tuple2<>(Arrays.asList("dodgers"), Arrays.asList("giants"))),
+            new Tuple2<>("new york",
+                         new Tuple2<>(Arrays.asList("yankees"), Arrays.asList("mets")))),
+        Arrays.asList(
+            new Tuple2<>("california",
+                         new Tuple2<>(Arrays.asList("sharks"), Arrays.asList("ducks"))),
+            new Tuple2<>("new york",
+                         new Tuple2<>(Arrays.asList("rangers"), Arrays.asList("islanders")))));
+
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<Iterable<String>, Iterable<String>>> grouped =
+        pairStream1.cogroup(pairStream2);
+    JavaTestUtils.attachTestOutputStream(grouped);
+    List<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> result =
+        JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected.size(), result.size());
+    Iterator<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> resultItr =
+        result.iterator();
+    Iterator<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expectedItr =
+        expected.iterator();
+    while (resultItr.hasNext() && expectedItr.hasNext()) {
+      Iterator<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>> resultElements =
+          resultItr.next().iterator();
+      Iterator<Tuple2<String, Tuple2<List<String>, List<String>>>> expectedElements =
+          expectedItr.next().iterator();
+      while (resultElements.hasNext() && expectedElements.hasNext()) {
+        Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>> resultElement =
+            resultElements.next();
+        Tuple2<String, Tuple2<List<String>, List<String>>> expectedElement =
+            expectedElements.next();
+        Assert.assertEquals(expectedElement._1(), resultElement._1());
+        equalIterable(expectedElement._2()._1(), resultElement._2()._1());
+        equalIterable(expectedElement._2()._2(), resultElement._2()._2());
+      }
+      Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testJoin() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "dodgers"),
+                      new Tuple2<>("new york", "yankees")),
+        Arrays.asList(new Tuple2<>("california", "sharks"),
+                      new Tuple2<>("new york", "rangers")));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "giants"),
+                      new Tuple2<>("new york", "mets")),
+        Arrays.asList(new Tuple2<>("california", "ducks"),
+                      new Tuple2<>("new york", "islanders")));
+
+
+    List<List<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<>("california",
+                         new Tuple2<>("dodgers", "giants")),
+            new Tuple2<>("new york",
+                         new Tuple2<>("yankees", "mets"))),
+        Arrays.asList(
+            new Tuple2<>("california",
+                         new Tuple2<>("sharks", "ducks")),
+            new Tuple2<>("new york",
+                         new Tuple2<>("rangers", "islanders"))));
+
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.join(pairStream2);
+    JavaTestUtils.attachTestOutputStream(joined);
+    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testLeftOuterJoin() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "dodgers"),
+                      new Tuple2<>("new york", "yankees")),
+        Arrays.asList(new Tuple2<>("california", "sharks") ));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(new Tuple2<>("california", "giants") ),
+        Arrays.asList(new Tuple2<>("new york", "islanders") )
+
+    );
+
+    List<List<Long>> expected = Arrays.asList(Arrays.asList(2L), Arrays.asList(1L));
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<String, Optional<String>>> joined =
+        pairStream1.leftOuterJoin(pairStream2);
+    JavaDStream<Long> counted = joined.count();
+    JavaTestUtils.attachTestOutputStream(counted);
+    List<List<Long>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCheckpointMasterRecovery() throws InterruptedException {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("this", "is"),
+        Arrays.asList("a", "test"),
+        Arrays.asList("counting", "letters"));
+
+    List<List<Integer>> expectedInitial = Arrays.asList(
+        Arrays.asList(4,2));
+    List<List<Integer>> expectedFinal = Arrays.asList(
+        Arrays.asList(1,4),
+        Arrays.asList(8,7));
+
+    File tempDir = Files.createTempDir();
+    tempDir.deleteOnExit();
+    ssc.checkpoint(tempDir.getAbsolutePath());
+
+    JavaDStream<String> stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
+      @Override
+      public Integer call(String s) {
+        return s.length();
+      }
+    });
+    JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
+    List<List<Integer>> initialResult = JavaTestUtils.runStreams(ssc, 1, 1);
+
+    assertOrderInvariantEquals(expectedInitial, initialResult);
+    Thread.sleep(1000);
+    ssc.stop();
+
+    ssc = new JavaStreamingContext(tempDir.getAbsolutePath());
+    // Tweak to take into consideration that the last batch before failure
+    // will be re-processed after recovery
+    List<List<Integer>> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3);
+    assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3));
+    ssc.stop();
+    Utils.deleteRecursively(tempDir);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testContextGetOrCreate() throws InterruptedException {
+    ssc.stop();
+
+    final SparkConf conf = new SparkConf()
+        .setMaster("local[2]")
+        .setAppName("test")
+        .set("newContext", "true");
+
+    File emptyDir = Files.createTempDir();
+    emptyDir.deleteOnExit();
+    StreamingContextSuite contextSuite = new StreamingContextSuite();
+    String corruptedCheckpointDir = contextSuite.createCorruptedCheckpoint();
+    String checkpointDir = contextSuite.createValidCheckpoint();
+
+    // Function to create JavaStreamingContext without any output operations
+    // (used to detect the new context)
+    final AtomicBoolean newContextCreated = new AtomicBoolean(false);
+    Function0<JavaStreamingContext> creatingFunc = new Function0<JavaStreamingContext>() {
+      @Override
+      public JavaStreamingContext call() {
+        newContextCreated.set(true);
+        return new JavaStreamingContext(conf, Seconds.apply(1));
+      }
+    };
+
+    newContextCreated.set(false);
+    ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc);
+    Assert.assertTrue("new context not created", newContextCreated.get());
+    ssc.stop();
+
+    newContextCreated.set(false);
+    ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc,
+        new Configuration(), true);
+    Assert.assertTrue("new context not created", newContextCreated.get());
+    ssc.stop();
+
+    newContextCreated.set(false);
+    ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc,
+        new Configuration());
+    Assert.assertTrue("old context not recovered", !newContextCreated.get());
+    ssc.stop();
+
+    newContextCreated.set(false);
+    JavaSparkContext sc = new JavaSparkContext(conf);
+    ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc,
+        new Configuration());
+    Assert.assertTrue("old context not recovered", !newContextCreated.get());
+    ssc.stop();
+  }
+
+  /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCheckpointofIndividualStream() throws InterruptedException {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("this", "is"),
+        Arrays.asList("a", "test"),
+        Arrays.asList("counting", "letters"));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(4,2),
+        Arrays.asList(1,4),
+        Arrays.asList(8,7));
+
+    JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+      @Override
+      public Integer call(String s) {
+        return s.length();
+      }
+    });
+    JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
+
+    letterCount.checkpoint(new Duration(1000));
+
+    List<List<Integer>> result1 = JavaCheckpointTestUtils.runStreams(ssc, 3, 3);
+    assertOrderInvariantEquals(expected, result1);
+  }
+  */
+
+  // Input stream tests. These mostly just test that we can instantiate a given InputStream with
+  // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the
+  // InputStream functionality is deferred to the existing Scala tests.
+  @Test
+  public void testSocketTextStream() {
+    ssc.socketTextStream("localhost", 12345);
+  }
+
+  @Test
+  public void testSocketString() {
+    ssc.socketStream(
+      "localhost",
+      12345,
+      new Function<InputStream, Iterable<String>>() {
+        @Override
+        public Iterable<String> call(InputStream in) throws IOException {
+          List<String> out = new ArrayList<>();
+          try (BufferedReader reader = new BufferedReader(
+              new InputStreamReader(in, StandardCharsets.UTF_8))) {
+            for (String line; (line = reader.readLine()) != null;) {
+              out.add(line);
+            }
+          }
+          return out;
+        }
+      },
+      StorageLevel.MEMORY_ONLY());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testTextFileStream() throws IOException {
+    File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark");
+    List<List<String>> expected = fileTestPrepare(testDir);
+
+    JavaDStream<String> input = ssc.textFileStream(testDir.toString());
+    JavaTestUtils.attachTestOutputStream(input);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 1, 1);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testFileStream() throws IOException {
+    File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark");
+    List<List<String>> expected = fileTestPrepare(testDir);
+
+    JavaPairInputDStream<LongWritable, Text> inputStream = ssc.fileStream(
+      testDir.toString(),
+      LongWritable.class,
+      Text.class,
+      TextInputFormat.class,
+      new Function<Path, Boolean>() {
+        @Override
+        public Boolean call(Path v1) {
+          return Boolean.TRUE;
+        }
+      },
+      true);
+
+    JavaDStream<String> test = inputStream.map(
+      new Function<Tuple2<LongWritable, Text>, String>() {
+        @Override
+        public String call(Tuple2<LongWritable, Text> v1) {
+          return v1._2().toString();
+        }
+    });
+
+    JavaTestUtils.attachTestOutputStream(test);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 1, 1);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testRawSocketStream() {
+    ssc.rawSocketStream("localhost", 12345);
+  }
+
+  private static List<List<String>> fileTestPrepare(File testDir) throws IOException {
+    File existingFile = new File(testDir, "0");
+    Files.write("0\n", existingFile, StandardCharsets.UTF_8);
+    Assert.assertTrue(existingFile.setLastModified(1000));
+    Assert.assertEquals(1000, existingFile.lastModified());
+    return Arrays.asList(Arrays.asList("0"));
+  }
+
+  @SuppressWarnings("unchecked")
+  // SPARK-5795: no logic assertions, just testing that intended API invocations compile
+  private void compileSaveAsJavaAPI(JavaPairDStream<LongWritable,Text> pds) {
+    pds.saveAsNewAPIHadoopFiles(
+        "", "", LongWritable.class, Text.class,
+        org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class);
+    pds.saveAsHadoopFiles(
+        "", "", LongWritable.class, Text.class,
+        org.apache.hadoop.mapred.SequenceFileOutputFormat.class);
+    // Checks that a previous common workaround for this API still compiles
+    pds.saveAsNewAPIHadoopFiles(
+        "", "", LongWritable.class, Text.class,
+        (Class) org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class);
+    pds.saveAsHadoopFiles(
+        "", "", LongWritable.class, Text.class,
+        (Class) org.apache.hadoop.mapred.SequenceFileOutputFormat.class);
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[4/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java
index c7959ae..ff80453 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java
@@ -44,12 +44,7 @@ class OutputRedirector {
   OutputRedirector(InputStream in, String loggerName, ThreadFactory tf) {
     this.active = true;
     this.reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8));
-    this.thread = tf.newThread(new Runnable() {
-      @Override
-      public void run() {
-        redirect();
-      }
-    });
+    this.thread = tf.newThread(this::redirect);
     this.sink = Logger.getLogger(loggerName);
     thread.start();
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
index 0aa7bd1..cefb4d1 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java
@@ -91,9 +91,6 @@ public interface SparkAppHandle {
    * Tries to kill the underlying application. Implies {@link #disconnect()}. This will not send
    * a {@link #stop()} message to the application, so it's recommended that users first try to
    * stop the application cleanly and only resort to this method if that fails.
-   * <p>
-   * Note that if the application is running as a child process, this method fail to kill the
-   * process when using Java 7. This may happen if, for example, the application is deadlocked.
    */
   void kill();
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
index 82b593a..8178684 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
@@ -49,35 +49,44 @@ class SparkClassCommandBuilder extends AbstractCommandBuilder {
 
     // Master, Worker, HistoryServer, ExternalShuffleService, MesosClusterDispatcher use
     // SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY.
-    if (className.equals("org.apache.spark.deploy.master.Master")) {
-      javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
-      javaOptsKeys.add("SPARK_MASTER_OPTS");
-      memKey = "SPARK_DAEMON_MEMORY";
-    } else if (className.equals("org.apache.spark.deploy.worker.Worker")) {
-      javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
-      javaOptsKeys.add("SPARK_WORKER_OPTS");
-      memKey = "SPARK_DAEMON_MEMORY";
-    } else if (className.equals("org.apache.spark.deploy.history.HistoryServer")) {
-      javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
-      javaOptsKeys.add("SPARK_HISTORY_OPTS");
-      memKey = "SPARK_DAEMON_MEMORY";
-    } else if (className.equals("org.apache.spark.executor.CoarseGrainedExecutorBackend")) {
-      javaOptsKeys.add("SPARK_JAVA_OPTS");
-      javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
-      memKey = "SPARK_EXECUTOR_MEMORY";
-    } else if (className.equals("org.apache.spark.executor.MesosExecutorBackend")) {
-      javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
-      memKey = "SPARK_EXECUTOR_MEMORY";
-    } else if (className.equals("org.apache.spark.deploy.mesos.MesosClusterDispatcher")) {
-      javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
-    } else if (className.equals("org.apache.spark.deploy.ExternalShuffleService") ||
-        className.equals("org.apache.spark.deploy.mesos.MesosExternalShuffleService")) {
-      javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
-      javaOptsKeys.add("SPARK_SHUFFLE_OPTS");
-      memKey = "SPARK_DAEMON_MEMORY";
-    } else {
-      javaOptsKeys.add("SPARK_JAVA_OPTS");
-      memKey = "SPARK_DRIVER_MEMORY";
+    switch (className) {
+      case "org.apache.spark.deploy.master.Master":
+        javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+        javaOptsKeys.add("SPARK_MASTER_OPTS");
+        memKey = "SPARK_DAEMON_MEMORY";
+        break;
+      case "org.apache.spark.deploy.worker.Worker":
+        javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+        javaOptsKeys.add("SPARK_WORKER_OPTS");
+        memKey = "SPARK_DAEMON_MEMORY";
+        break;
+      case "org.apache.spark.deploy.history.HistoryServer":
+        javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+        javaOptsKeys.add("SPARK_HISTORY_OPTS");
+        memKey = "SPARK_DAEMON_MEMORY";
+        break;
+      case "org.apache.spark.executor.CoarseGrainedExecutorBackend":
+        javaOptsKeys.add("SPARK_JAVA_OPTS");
+        javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
+        memKey = "SPARK_EXECUTOR_MEMORY";
+        break;
+      case "org.apache.spark.executor.MesosExecutorBackend":
+        javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
+        memKey = "SPARK_EXECUTOR_MEMORY";
+        break;
+      case "org.apache.spark.deploy.mesos.MesosClusterDispatcher":
+        javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+        break;
+      case "org.apache.spark.deploy.ExternalShuffleService":
+      case "org.apache.spark.deploy.mesos.MesosExternalShuffleService":
+        javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+        javaOptsKeys.add("SPARK_SHUFFLE_OPTS");
+        memKey = "SPARK_DAEMON_MEMORY";
+        break;
+      default:
+        javaOptsKeys.add("SPARK_JAVA_OPTS");
+        memKey = "SPARK_DRIVER_MEMORY";
+        break;
     }
 
     List<String> cmd = buildJavaCommand(extraClassPath);
@@ -94,7 +103,6 @@ class SparkClassCommandBuilder extends AbstractCommandBuilder {
 
     String mem = firstNonEmpty(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM);
     cmd.add("-Xmx" + mem);
-    addPermGenSizeOpt(cmd);
     cmd.add(className);
     cmd.addAll(classArgs);
     return cmd;

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
index 29c6d82..5e64fa7 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
@@ -271,7 +271,6 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
         config.get(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH));
     }
 
-    addPermGenSizeOpt(cmd);
     cmd.add("org.apache.spark.deploy.SparkSubmit");
     cmd.addAll(buildSparkSubmitArgs());
     return cmd;
@@ -405,49 +404,65 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
 
     @Override
     protected boolean handle(String opt, String value) {
-      if (opt.equals(MASTER)) {
-        master = value;
-      } else if (opt.equals(DEPLOY_MODE)) {
-        deployMode = value;
-      } else if (opt.equals(PROPERTIES_FILE)) {
-        propertiesFile = value;
-      } else if (opt.equals(DRIVER_MEMORY)) {
-        conf.put(SparkLauncher.DRIVER_MEMORY, value);
-      } else if (opt.equals(DRIVER_JAVA_OPTIONS)) {
-        conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value);
-      } else if (opt.equals(DRIVER_LIBRARY_PATH)) {
-        conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value);
-      } else if (opt.equals(DRIVER_CLASS_PATH)) {
-        conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value);
-      } else if (opt.equals(CONF)) {
-        String[] setConf = value.split("=", 2);
-        checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value);
-        conf.put(setConf[0], setConf[1]);
-      } else if (opt.equals(CLASS)) {
-        // The special classes require some special command line handling, since they allow
-        // mixing spark-submit arguments with arguments that should be propagated to the shell
-        // itself. Note that for this to work, the "--class" argument must come before any
-        // non-spark-submit arguments.
-        mainClass = value;
-        if (specialClasses.containsKey(value)) {
-          allowsMixedArguments = true;
-          appResource = specialClasses.get(value);
-        }
-      } else if (opt.equals(KILL_SUBMISSION) || opt.equals(STATUS)) {
-        isAppResourceReq = false;
-        sparkArgs.add(opt);
-        sparkArgs.add(value);
-      } else if (opt.equals(HELP) || opt.equals(USAGE_ERROR)) {
-        isAppResourceReq = false;
-        sparkArgs.add(opt);
-      } else if (opt.equals(VERSION)) {
-        isAppResourceReq = false;
-        sparkArgs.add(opt);
-      } else {
-        sparkArgs.add(opt);
-        if (value != null) {
+      switch (opt) {
+        case MASTER:
+          master = value;
+          break;
+        case DEPLOY_MODE:
+          deployMode = value;
+          break;
+        case PROPERTIES_FILE:
+          propertiesFile = value;
+          break;
+        case DRIVER_MEMORY:
+          conf.put(SparkLauncher.DRIVER_MEMORY, value);
+          break;
+        case DRIVER_JAVA_OPTIONS:
+          conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value);
+          break;
+        case DRIVER_LIBRARY_PATH:
+          conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value);
+          break;
+        case DRIVER_CLASS_PATH:
+          conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value);
+          break;
+        case CONF:
+          String[] setConf = value.split("=", 2);
+          checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value);
+          conf.put(setConf[0], setConf[1]);
+          break;
+        case CLASS:
+          // The special classes require some special command line handling, since they allow
+          // mixing spark-submit arguments with arguments that should be propagated to the shell
+          // itself. Note that for this to work, the "--class" argument must come before any
+          // non-spark-submit arguments.
+          mainClass = value;
+          if (specialClasses.containsKey(value)) {
+            allowsMixedArguments = true;
+            appResource = specialClasses.get(value);
+          }
+          break;
+        case KILL_SUBMISSION:
+        case STATUS:
+          isAppResourceReq = false;
+          sparkArgs.add(opt);
           sparkArgs.add(value);
-        }
+          break;
+        case HELP:
+        case USAGE_ERROR:
+          isAppResourceReq = false;
+          sparkArgs.add(opt);
+          break;
+        case VERSION:
+          isAppResourceReq = false;
+          sparkArgs.add(opt);
+          break;
+        default:
+          sparkArgs.add(opt);
+          if (value != null) {
+            sparkArgs.add(value);
+          }
+          break;
       }
       return true;
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java
----------------------------------------------------------------------
diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java
index caeeea5..9795041 100644
--- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java
+++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java
@@ -99,42 +99,6 @@ public class CommandBuilderUtilsSuite {
     assertEquals(10, javaMajorVersion("10"));
   }
 
-  @Test
-  public void testAddPermGenSizeOpt() {
-    List<String> cmd = new ArrayList<>();
-
-    if (javaMajorVersion(System.getProperty("java.version")) > 7) {
-      // Does nothing in Java 8
-      addPermGenSizeOpt(cmd);
-      assertEquals(0, cmd.size());
-      cmd.clear();
-
-    } else {
-      addPermGenSizeOpt(cmd);
-      assertEquals(1, cmd.size());
-      assertTrue(cmd.get(0).startsWith("-XX:MaxPermSize="));
-      cmd.clear();
-
-      cmd.add("foo");
-      addPermGenSizeOpt(cmd);
-      assertEquals(2, cmd.size());
-      assertTrue(cmd.get(1).startsWith("-XX:MaxPermSize="));
-      cmd.clear();
-
-      cmd.add("-XX:MaxPermSize=512m");
-      addPermGenSizeOpt(cmd);
-      assertEquals(1, cmd.size());
-      assertEquals("-XX:MaxPermSize=512m", cmd.get(0));
-      cmd.clear();
-
-      cmd.add("'-XX:MaxPermSize=512m'");
-      addPermGenSizeOpt(cmd);
-      assertEquals(1, cmd.size());
-      assertEquals("'-XX:MaxPermSize=512m'", cmd.get(0));
-      cmd.clear();
-    }
-  }
-
   private static void testOpt(String opts, List<String> expected) {
     assertEquals(String.format("test string failed to parse: [[ %s ]]", opts),
         expected, parseOptionString(opts));

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
----------------------------------------------------------------------
diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
index ad2e7a7..d569b66 100644
--- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
+++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
@@ -233,7 +233,7 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite {
       launcher.setPropertiesFile(dummyPropsFile.getAbsolutePath());
       launcher.conf.put(SparkLauncher.DRIVER_MEMORY, "1g");
       launcher.conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver");
-      launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m");
+      launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver");
       launcher.conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native");
     } else {
       launcher.childEnv.put("SPARK_CONF_DIR", System.getProperty("spark.test.home")
@@ -258,12 +258,6 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite {
       assertFalse("Memory arguments should not be set.", found);
     }
 
-    for (String arg : cmd) {
-      if (arg.startsWith("-XX:MaxPermSize=")) {
-        assertEquals("-XX:MaxPermSize=256m", arg);
-      }
-    }
-
     String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator));
     if (isDriver) {
       assertTrue("Driver classpath should contain provided entry.", contains("/driver", cp));

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/test/resources/spark-defaults.conf
----------------------------------------------------------------------
diff --git a/launcher/src/test/resources/spark-defaults.conf b/launcher/src/test/resources/spark-defaults.conf
index 239fc57..3a51208 100644
--- a/launcher/src/test/resources/spark-defaults.conf
+++ b/launcher/src/test/resources/spark-defaults.conf
@@ -17,5 +17,5 @@
 
 spark.driver.memory=1g
 spark.driver.extraClassPath=/driver
-spark.driver.extraJavaOptions=-Ddriver -XX:MaxPermSize=256m
+spark.driver.extraJavaOptions=-Ddriver
 spark.driver.extraLibraryPath=/native
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ac61a57..60e4c72 100644
--- a/pom.xml
+++ b/pom.xml
@@ -117,7 +117,7 @@
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-    <java.version>1.7</java.version>
+    <java.version>1.8</java.version>
     <maven.version>3.3.9</maven.version>
     <sbt.project.name>spark</sbt.project.name>
     <slf4j.version>1.7.16</slf4j.version>
@@ -186,9 +186,6 @@
     <test.java.home>${java.home}</test.java.home>
     <test.exclude.tags></test.exclude.tags>
 
-    <!-- When using different JDKs for the build, we can't use Zinc for the jdk8 part. -->
-    <useZincForJdk8>true</useZincForJdk8>
-
     <!-- Package to use when relocating shaded classes. -->
     <spark.shade.packageName>org.spark_project</spark.shade.packageName>
 
@@ -219,8 +216,6 @@
     -->
     <spark.test.home>${session.executionRootDirectory}</spark.test.home>
 
-    <PermGen>64m</PermGen>
-    <MaxPermGen>512m</MaxPermGen>
     <CodeCacheSize>512m</CodeCacheSize>
   </properties>
   <repositories>
@@ -1920,7 +1915,7 @@
         <plugin>
           <groupId>org.codehaus.mojo</groupId>
           <artifactId>build-helper-maven-plugin</artifactId>
-          <version>1.12</version>
+          <version>3.0.0</version>
         </plugin>
         <plugin>
           <groupId>net.alchim31.maven</groupId>
@@ -1967,8 +1962,6 @@
             <jvmArgs>
               <jvmArg>-Xms1024m</jvmArg>
               <jvmArg>-Xmx1024m</jvmArg>
-              <jvmArg>-XX:PermSize=${PermGen}</jvmArg>
-              <jvmArg>-XX:MaxPermSize=${MaxPermGen}</jvmArg>
               <jvmArg>-XX:ReservedCodeCacheSize=${CodeCacheSize}</jvmArg>
             </jvmArgs>
             <javacArgs>
@@ -1983,7 +1976,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
-          <version>3.6.0</version>
+          <version>3.6.1</version>
           <configuration>
             <source>${java.version}</source>
             <target>${java.version}</target>
@@ -2014,7 +2007,7 @@
               <include>**/*Suite.java</include>
             </includes>
             <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
-            <argLine>-Xmx3g -Xss4096k -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m</argLine>
+            <argLine>-Xmx3g -Xss4096k -XX:ReservedCodeCacheSize=${CodeCacheSize}</argLine>
             <environmentVariables>
               <!--
                 Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes
@@ -2063,7 +2056,7 @@
             <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
             <junitxml>.</junitxml>
             <filereports>SparkTestSuite.txt</filereports>
-            <argLine>-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize}</argLine>
+            <argLine>-ea -Xmx3g -XX:ReservedCodeCacheSize=${CodeCacheSize}</argLine>
             <stderr/>
             <environmentVariables>
               <!--
@@ -2149,6 +2142,41 @@
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-javadoc-plugin</artifactId>
           <version>2.10.4</version>
+          <configuration>
+            <additionalparam>-Xdoclint:all -Xdoclint:-missing</additionalparam>
+            <tags>
+              <tag>
+                <name>example</name>
+                <placement>a</placement>
+                <head>Example:</head>
+              </tag>
+              <tag>
+                <name>note</name>
+                <placement>a</placement>
+                <head>Note:</head>
+              </tag>
+              <tag>
+                <name>group</name>
+                <placement>X</placement>
+              </tag>
+              <tag>
+                <name>tparam</name>
+                <placement>X</placement>
+              </tag>
+              <tag>
+                <name>constructor</name>
+                <placement>X</placement>
+              </tag>
+              <tag>
+                <name>todo</name>
+                <placement>X</placement>
+              </tag>
+              <tag>
+                <name>groupname</name>
+                <placement>X</placement>
+              </tag>
+            </tags>
+          </configuration>
         </plugin>
         <plugin>
           <groupId>org.codehaus.mojo</groupId>
@@ -2163,7 +2191,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-shade-plugin</artifactId>
-          <version>2.4.3</version>
+          <version>3.0.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -2178,6 +2206,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-dependency-plugin</artifactId>
+          <version>3.0.0</version>
           <executions>
             <execution>
               <id>default-cli</id>
@@ -2252,7 +2281,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
-        <version>2.10</version>
         <executions>
           <execution>
             <id>generate-test-classpath</id>
@@ -2474,67 +2502,6 @@
     </profile>
 
     <profile>
-      <id>java8-tests</id>
-      <activation>
-        <jdk>[1.8,)</jdk>
-      </activation>
-      <modules>
-        <module>external/java8-tests</module>
-      </modules>
-    </profile>
-
-    <profile>
-      <id>doclint-java8-disable</id>
-      <activation>
-        <jdk>[1.8,)</jdk>
-      </activation>
-
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-javadoc-plugin</artifactId>
-            <configuration>
-              <additionalparam>-Xdoclint:all -Xdoclint:-missing</additionalparam>
-              <tags>
-                <tag>
-                  <name>example</name>
-                  <placement>a</placement>
-                  <head>Example:</head>
-                </tag>
-                <tag>
-                  <name>note</name>
-                  <placement>a</placement>
-                  <head>Note:</head>
-                </tag>
-                <tag>
-                  <name>group</name>
-                  <placement>X</placement>
-                </tag>
-                <tag>
-                  <name>tparam</name>
-                  <placement>X</placement>
-                </tag>
-                <tag>
-                  <name>constructor</name>
-                  <placement>X</placement>
-                </tag>
-                <tag>
-                  <name>todo</name>
-                  <placement>X</placement>
-                </tag>
-                <tag>
-                  <name>groupname</name>
-                  <placement>X</placement>
-                </tag>
-              </tags>
-            </configuration>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-
-    <profile>
       <id>docker-integration-tests</id>
       <modules>
         <module>external/docker-integration-tests</module>
@@ -2630,60 +2597,6 @@
     </profile>
 
     <profile>
-      <id>java7</id>
-      <activation>
-        <property><name>env.JAVA_7_HOME</name></property>
-      </activation>
-      <properties>
-        <useZincForJdk8>false</useZincForJdk8>
-      </properties>
-      <build>
-        <pluginManagement>
-          <plugins>
-            <plugin>
-              <groupId>org.apache.maven.plugins</groupId>
-              <artifactId>maven-compiler-plugin</artifactId>
-              <configuration>
-                <compilerArgs combine.children="append">
-                  <arg>-bootclasspath</arg>
-                  <arg>${env.JAVA_7_HOME}/jre/lib/rt.jar${path.separator}${env.JAVA_7_HOME}/jre/lib/jce.jar</arg>
-                </compilerArgs>
-                <verbose>true</verbose>
-              </configuration>
-            </plugin>
-            <plugin>
-              <groupId>net.alchim31.maven</groupId>
-              <artifactId>scala-maven-plugin</artifactId>
-              <!-- Note: -javabootclasspath is set on a per-execution basis rather than as a
-                   plugin-wide configuration because doc-jar generation will break if it's
-                   set; see SPARK-15839 for more details -->
-              <executions>
-                <execution>
-                  <id>scala-compile-first</id>
-                  <configuration>
-                    <args combine.children="append">
-                      <arg>-javabootclasspath</arg>
-                      <arg>${env.JAVA_7_HOME}/jre/lib/rt.jar${path.separator}${env.JAVA_7_HOME}/jre/lib/jce.jar</arg>
-                    </args>
-                  </configuration>
-                </execution>
-                <execution>
-                  <id>scala-test-compile-first</id>
-                  <configuration>
-                    <args combine.children="append">
-                      <arg>-javabootclasspath</arg>
-                      <arg>${env.JAVA_7_HOME}/jre/lib/rt.jar${path.separator}${env.JAVA_7_HOME}/jre/lib/jce.jar</arg>
-                    </args>
-                  </configuration>
-                </execution>
-              </executions>
-            </plugin>
-          </plugins>
-        </pluginManagement>
-      </build>
-    </profile>
-
-    <profile>
       <id>scala-2.11</id>
       <activation>
         <property><name>!scala-2.10</name></property>

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index bcc00fa..b48879f 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -56,9 +56,9 @@ object BuildCommons {
     "tags", "sketch"
   ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects
 
-  val optionallyEnabledProjects@Seq(mesos, yarn, java8Tests, sparkGangliaLgpl,
+  val optionallyEnabledProjects@Seq(mesos, yarn, sparkGangliaLgpl,
     streamingKinesisAsl, dockerIntegrationTests) =
-    Seq("mesos", "yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl",
+    Seq("mesos", "yarn", "ganglia-lgpl", "streaming-kinesis-asl",
       "docker-integration-tests").map(ProjectRef(buildLocation, _))
 
   val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) =
@@ -233,8 +233,8 @@ object SparkBuild extends PomBuild {
       if (major >= 8) Seq("-Xdoclint:all", "-Xdoclint:-missing") else Seq.empty
     },
 
-    javacJVMVersion := "1.7",
-    scalacJVMVersion := "1.7",
+    javacJVMVersion := "1.8",
+    scalacJVMVersion := "1.8",
 
     javacOptions in Compile ++= Seq(
       "-encoding", "UTF-8",
@@ -245,24 +245,12 @@ object SparkBuild extends PomBuild {
     // additional discussion and explanation.
     javacOptions in (Compile, compile) ++= Seq(
       "-target", javacJVMVersion.value
-    ) ++ sys.env.get("JAVA_7_HOME").toSeq.flatMap { jdk7 =>
-      if (javacJVMVersion.value == "1.7") {
-        Seq("-bootclasspath", s"$jdk7/jre/lib/rt.jar${File.pathSeparator}$jdk7/jre/lib/jce.jar")
-      } else {
-        Nil
-      }
-    },
+    ),
 
     scalacOptions in Compile ++= Seq(
       s"-target:jvm-${scalacJVMVersion.value}",
       "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath  // Required for relative source links in scaladoc
-    ) ++ sys.env.get("JAVA_7_HOME").toSeq.flatMap { jdk7 =>
-      if (javacJVMVersion.value == "1.7") {
-        Seq("-javabootclasspath", s"$jdk7/jre/lib/rt.jar${File.pathSeparator}$jdk7/jre/lib/jce.jar")
-      } else {
-        Nil
-      }
-    },
+    ),
 
     // Implements -Xfatal-warnings, ignoring deprecation warnings.
     // Code snippet taken from https://issues.scala-lang.org/browse/SI-8410.
@@ -363,8 +351,6 @@ object SparkBuild extends PomBuild {
 
   enable(Flume.settings)(streamingFlumeSink)
 
-  enable(Java8TestSettings.settings)(java8Tests)
-
   // SPARK-14738 - Remove docker tests from main Spark build
   // enable(DockerIntegrationTests.settings)(dockerIntegrationTests)
 
@@ -387,7 +373,7 @@ object SparkBuild extends PomBuild {
     fork := true,
     outputStrategy in run := Some (StdoutOutput),
 
-    javaOptions ++= Seq("-Xmx2G", "-XX:MaxPermSize=256m"),
+    javaOptions += "-Xmx2g",
 
     sparkShell := {
       (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value
@@ -531,7 +517,6 @@ object SQL {
 object Hive {
 
   lazy val settings = Seq(
-    javaOptions += "-XX:MaxPermSize=256m",
     // Specially disable assertions since some Hive tests fail them
     javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"),
     // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings
@@ -765,16 +750,6 @@ object CopyDependencies {
 
 }
 
-object Java8TestSettings {
-  import BuildCommons._
-
-  lazy val settings = Seq(
-    javacJVMVersion := "1.8",
-    // Targeting Java 8 bytecode is only supported in Scala 2.11.4 and higher:
-    scalacJVMVersion := (if (System.getProperty("scala-2.10") == "true") "1.7" else "1.8")
-  )
-}
-
 object TestSettings {
   import BuildCommons._
 
@@ -812,7 +787,7 @@ object TestSettings {
     javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark"))
       .map { case (k,v) => s"-D$k=$v" }.toSeq,
     javaOptions in Test += "-ea",
-    javaOptions in Test ++= "-Xmx3g -Xss4096k -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g"
+    javaOptions in Test ++= "-Xmx3g -Xss4096k"
       .split(" ").toSeq,
     javaOptions += "-Xmx3g",
     // Exclude tags defined in a system property

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index f4f4518..a00234c 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -911,7 +911,6 @@ private[spark] class Client(
 
     // For log4j configuration to reference
     javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
-    YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
 
     val userClass =
       if (isClusterMode) {

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
index b55b4b1..ee85c04 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
@@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
 import org.apache.spark.{SecurityManager, SparkConf, SparkException}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
-import org.apache.spark.launcher.YarnCommandBuilderUtils
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.util.Utils
 
@@ -190,7 +189,6 @@ private[yarn] class ExecutorRunnable(
 
     // For log4j configuration to reference
     javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
-    YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
 
     val userClassPath = Client.getUserClasspath(sparkConf).flatMap { uri =>
       val absPath =

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
index 6c3556a..0c3d080 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
@@ -38,16 +38,4 @@ private[spark] object YarnCommandBuilderUtils {
     CommandBuilderUtils.findJarsDir(sparkHome, scalaVer, true)
   }
 
-  /**
-   * Adds the perm gen configuration to the list of java options if needed and not yet added.
-   *
-   * Note that this method adds the option based on the local JVM version; if the node where
-   * the container is running has a different Java version, there's a risk that the option will
-   * not be added (e.g. if the AM is running Java 8 but the container's node is set up to use
-   * Java 7).
-   */
-  def addPermGenSizeOpt(args: ListBuffer[String]): Unit = {
-    CommandBuilderUtils.addPermGenSizeOpt(args.asJava)
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
index 465fb83..089c84d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
@@ -134,12 +134,8 @@ final class Decimal extends Ordered[Decimal] with Serializable {
    * Set this Decimal to the given BigInteger value. Will have precision 38 and scale 0.
    */
   def set(bigintval: BigInteger): Decimal = {
-    // TODO: Remove this once we migrate to java8 and use longValueExact() instead.
-    require(
-      bigintval.compareTo(LONG_MAX_BIG_INT) <= 0 && bigintval.compareTo(LONG_MIN_BIG_INT) >= 0,
-      s"BigInteger $bigintval too large for decimal")
     this.decimalVal = null
-    this.longVal = bigintval.longValue()
+    this.longVal = bigintval.longValueExact()
     this._precision = DecimalType.MAX_PRECISION
     this._scale = 0
     this
@@ -178,7 +174,7 @@ final class Decimal extends Ordered[Decimal] with Serializable {
 
   def toUnscaledLong: Long = {
     if (decimalVal.ne(null)) {
-      decimalVal.underlying().unscaledValue().longValue()
+      decimalVal.underlying().unscaledValue().longValueExact()
     } else {
       longVal
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java b/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java
index 2570c8d..d44af7e 100644
--- a/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java
+++ b/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java
@@ -22,13 +22,13 @@ import java.util.Iterator;
 
 import org.apache.spark.annotation.Experimental;
 import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.Encoder;
 import org.apache.spark.sql.KeyedState;
 
 /**
  * ::Experimental::
  * Base interface for a map function used in
- * {@link org.apache.spark.sql.KeyValueGroupedDataset#flatMapGroupsWithState(FlatMapGroupsWithStateFunction, Encoder, Encoder)}.
+ * {@link org.apache.spark.sql.KeyValueGroupedDataset#flatMapGroupsWithState(
+ * FlatMapGroupsWithStateFunction, org.apache.spark.sql.Encoder, org.apache.spark.sql.Encoder)}.
  * @since 2.1.1
  */
 @Experimental

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java b/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java
index 614d392..75986d1 100644
--- a/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java
+++ b/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java
@@ -22,13 +22,13 @@ import java.util.Iterator;
 
 import org.apache.spark.annotation.Experimental;
 import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.Encoder;
 import org.apache.spark.sql.KeyedState;
 
 /**
  * ::Experimental::
  * Base interface for a map function used in
- * {@link org.apache.spark.sql.KeyValueGroupedDataset#mapGroupsWithState(MapGroupsWithStateFunction, Encoder, Encoder)}
+ * {@link org.apache.spark.sql.KeyValueGroupedDataset#mapGroupsWithState(
+ * MapGroupsWithStateFunction, org.apache.spark.sql.Encoder, org.apache.spark.sql.Encoder)}
  * @since 2.1.1
  */
 @Experimental

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index e62cd9f..38a24cc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -103,7 +103,7 @@ private[sql] object Dataset {
  * the following creates a new Dataset by applying a filter on the existing one:
  * {{{
  *   val names = people.map(_.name)  // in Scala; names is a Dataset[String]
- *   Dataset<String> names = people.map((Person p) -> p.name, Encoders.STRING)); // in Java 8
+ *   Dataset<String> names = people.map((Person p) -> p.name, Encoders.STRING));
  * }}}
  *
  * Dataset operations can also be untyped, through various domain-specific-language (DSL)

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
index 94e689a..3a548c2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
@@ -98,7 +98,7 @@ class KeyValueGroupedDataset[K, V] private[sql](
    *   // Create Integer values grouped by String key from a Dataset<Tuple2<String, Integer>>
    *   Dataset<Tuple2<String, Integer>> ds = ...;
    *   KeyValueGroupedDataset<String, Integer> grouped =
-   *     ds.groupByKey(t -> t._1, Encoders.STRING()).mapValues(t -> t._2, Encoders.INT()); // Java 8
+   *     ds.groupByKey(t -> t._1, Encoders.STRING()).mapValues(t -> t._2, Encoders.INT());
    * }}}
    *
    * @since 2.1.0

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index ea465e2..dbe5509 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -190,17 +190,6 @@ class SQLContext private[sql](val sparkSession: SparkSession)
    * The following example registers a UDF in Java:
    * {{{
    *   sqlContext.udf().register("myUDF",
-   *       new UDF2<Integer, String, String>() {
-   *           @Override
-   *           public String call(Integer arg1, String arg2) {
-   *               return arg2 + arg1;
-   *           }
-   *      }, DataTypes.StringType);
-   * }}}
-   *
-   * Or, to use Java 8 lambda syntax:
-   * {{{
-   *   sqlContext.udf().register("myUDF",
    *       (Integer arg1, String arg2) -> arg2 + arg1,
    *       DataTypes.StringType);
    * }}}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index e1fdb2f..1975a56 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -164,17 +164,6 @@ class SparkSession private(
    * The following example registers a UDF in Java:
    * {{{
    *   sparkSession.udf().register("myUDF",
-   *       new UDF2<Integer, String, String>() {
-   *           @Override
-   *           public String call(Integer arg1, String arg2) {
-   *               return arg2 + arg1;
-   *           }
-   *      }, DataTypes.StringType);
-   * }}}
-   *
-   * Or, to use Java 8 lambda syntax:
-   * {{{
-   *   sparkSession.udf().register("myUDF",
    *       (Integer arg1, String arg2) -> arg2 + arg1,
    *       DataTypes.StringType);
    * }}}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java
new file mode 100644
index 0000000..8b8a403
--- /dev/null
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark.sql;
+
+import java.util.Arrays;
+
+import org.junit.Assert;
+import org.junit.Test;
+import scala.Tuple2;
+
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.KeyValueGroupedDataset;
+import org.apache.spark.sql.expressions.javalang.typed;
+
+/**
+ * Suite that replicates tests in JavaDatasetAggregatorSuite using lambda syntax.
+ */
+public class Java8DatasetAggregatorSuite extends JavaDatasetAggregatorSuiteBase {
+  @Test
+  public void testTypedAggregationAverage() {
+    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
+    Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.avg(v -> (double)(v._2() * 2)));
+    Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 6.0)), agged.collectAsList());
+  }
+
+  @Test
+  public void testTypedAggregationCount() {
+    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
+    Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.count(v -> v));
+    Assert.assertEquals(Arrays.asList(tuple2("a", 2L), tuple2("b", 1L)), agged.collectAsList());
+  }
+
+  @Test
+  public void testTypedAggregationSumDouble() {
+    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
+    Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.sum(v -> (double)v._2()));
+    Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 3.0)), agged.collectAsList());
+  }
+
+  @Test
+  public void testTypedAggregationSumLong() {
+    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
+    Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.sumLong(v -> (long)v._2()));
+    Assert.assertEquals(Arrays.asList(tuple2("a", 3L), tuple2("b", 3L)), agged.collectAsList());
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
index 5ef4e88..a94a37c 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
@@ -228,7 +228,7 @@ public class JavaDatasetSuite implements Serializable {
     Dataset<String> mapped2 = grouped.mapGroupsWithState(
       new MapGroupsWithStateFunction<Integer, String, Long, String>() {
         @Override
-        public String call(Integer key, Iterator<String> values, KeyedState<Long> s) throws Exception {
+        public String call(Integer key, Iterator<String> values, KeyedState<Long> s) {
           StringBuilder sb = new StringBuilder(key.toString());
           while (values.hasNext()) {
             sb.append(values.next());

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/hive/pom.xml
----------------------------------------------------------------------
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index 9aedaf2..0f249d7 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -190,6 +190,7 @@
           <plugin>
             <groupId>org.codehaus.mojo</groupId>
             <artifactId>build-helper-maven-plugin</artifactId>
+            <version>3.0.0</version>
             <executions>
               <execution>
                 <id>add-scala-test-sources</id>
@@ -219,7 +220,7 @@
         <artifactId>scalatest-maven-plugin</artifactId>
         <configuration>
           <!-- Specially disable assertions since some Hive tests fail them -->
-          <argLine>-da -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m</argLine>
+          <argLine>-da -Xmx3g -XX:ReservedCodeCacheSize=${CodeCacheSize}</argLine>
         </configuration>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala
index e7c165c..d786a61 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala
@@ -137,21 +137,13 @@ case class ScriptTransformationExec(
             throw writerThread.exception.get
           }
 
-          // Checks if the proc is still alive (incase the command ran was bad)
-          // The ideal way to do this is to use Java 8's Process#isAlive()
-          // but it cannot be used because Spark still supports Java 7.
-          // Following is a workaround used to check if a process is alive in Java 7
-          // TODO: Once builds are switched to Java 8, this can be changed
-          try {
+          if (!proc.isAlive) {
             val exitCode = proc.exitValue()
             if (exitCode != 0) {
               logError(stderrBuffer.toString) // log the stderr circular buffer
               throw new SparkException(s"Subprocess exited with status $exitCode. " +
                 s"Error: ${stderrBuffer.toString}", cause)
             }
-          } catch {
-            case _: IllegalThreadStateException =>
-            // This means that the process is still alive. Move ahead
           }
         }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[2/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java
new file mode 100644
index 0000000..646cb97
--- /dev/null
+++ b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java
@@ -0,0 +1,887 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark.streaming;
+
+import java.io.Serializable;
+import java.util.*;
+
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.Durations;
+import org.apache.spark.streaming.JavaTestUtils;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.StateSpec;
+import org.apache.spark.streaming.Time;
+import scala.Tuple2;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.spark.HashPartitioner;
+import org.apache.spark.api.java.Optional;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaMapWithStateDStream;
+
+/**
+ * Most of these tests replicate org.apache.spark.streaming.JavaAPISuite using java 8
+ * lambda syntax.
+ */
+@SuppressWarnings("unchecked")
+public class Java8APISuite extends LocalJavaStreamingContext implements Serializable {
+
+  @Test
+  public void testMap() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("hello", "world"),
+      Arrays.asList("goodnight", "moon"));
+
+    List<List<Integer>> expected = Arrays.asList(
+      Arrays.asList(5, 5),
+      Arrays.asList(9, 4));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> letterCount = stream.map(String::length);
+    JavaTestUtils.attachTestOutputStream(letterCount);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testFilter() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("giants", "dodgers"),
+      Arrays.asList("yankees", "red sox"));
+
+    List<List<String>> expected = Arrays.asList(
+      Arrays.asList("giants"),
+      Arrays.asList("yankees"));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<String> filtered = stream.filter(s -> s.contains("a"));
+    JavaTestUtils.attachTestOutputStream(filtered);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testMapPartitions() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("giants", "dodgers"),
+      Arrays.asList("yankees", "red sox"));
+
+    List<List<String>> expected = Arrays.asList(
+      Arrays.asList("GIANTSDODGERS"),
+      Arrays.asList("YANKEESRED SOX"));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<String> mapped = stream.mapPartitions(in -> {
+      String out = "";
+      while (in.hasNext()) {
+        out = out + in.next().toUpperCase();
+      }
+      return Lists.newArrayList(out).iterator();
+    });
+    JavaTestUtils.attachTestOutputStream(mapped);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testReduce() {
+    List<List<Integer>> inputData = Arrays.asList(
+      Arrays.asList(1, 2, 3),
+      Arrays.asList(4, 5, 6),
+      Arrays.asList(7, 8, 9));
+
+    List<List<Integer>> expected = Arrays.asList(
+      Arrays.asList(6),
+      Arrays.asList(15),
+      Arrays.asList(24));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> reduced = stream.reduce((x, y) -> x + y);
+    JavaTestUtils.attachTestOutputStream(reduced);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testReduceByWindow() {
+    List<List<Integer>> inputData = Arrays.asList(
+      Arrays.asList(1, 2, 3),
+      Arrays.asList(4, 5, 6),
+      Arrays.asList(7, 8, 9));
+
+    List<List<Integer>> expected = Arrays.asList(
+      Arrays.asList(6),
+      Arrays.asList(21),
+      Arrays.asList(39),
+      Arrays.asList(24));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> reducedWindowed = stream.reduceByWindow((x, y) -> x + y,
+                                                                 (x, y) -> x - y, new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reducedWindowed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testTransform() {
+    List<List<Integer>> inputData = Arrays.asList(
+      Arrays.asList(1, 2, 3),
+      Arrays.asList(4, 5, 6),
+      Arrays.asList(7, 8, 9));
+
+    List<List<Integer>> expected = Arrays.asList(
+      Arrays.asList(3, 4, 5),
+      Arrays.asList(6, 7, 8),
+      Arrays.asList(9, 10, 11));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> transformed = stream.transform(in -> in.map(i -> i + 2));
+
+    JavaTestUtils.attachTestOutputStream(transformed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testVariousTransform() {
+    // tests whether all variations of transform can be called from Java
+
+    List<List<Integer>> inputData = Arrays.asList(Arrays.asList(1));
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+
+    List<List<Tuple2<String, Integer>>> pairInputData =
+      Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(
+      JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1));
+
+    JavaDStream<Integer> transformed1 = stream.transform(in -> null);
+    JavaDStream<Integer> transformed2 = stream.transform((x, time) -> null);
+    JavaPairDStream<String, Integer> transformed3 = stream.transformToPair(x -> null);
+    JavaPairDStream<String, Integer> transformed4 = stream.transformToPair((x, time) -> null);
+    JavaDStream<Integer> pairTransformed1 = pairStream.transform(x -> null);
+    JavaDStream<Integer> pairTransformed2 = pairStream.transform((x, time) -> null);
+    JavaPairDStream<String, String> pairTransformed3 = pairStream.transformToPair(x -> null);
+    JavaPairDStream<String, String> pairTransformed4 =
+      pairStream.transformToPair((x, time) -> null);
+
+  }
+
+  @Test
+  public void testTransformWith() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>("california", "dodgers"),
+        new Tuple2<>("new york", "yankees")),
+      Arrays.asList(
+        new Tuple2<>("california", "sharks"),
+        new Tuple2<>("new york", "rangers")));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>("california", "giants"),
+        new Tuple2<>("new york", "mets")),
+      Arrays.asList(
+        new Tuple2<>("california", "ducks"),
+        new Tuple2<>("new york", "islanders")));
+
+
+    List<Set<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
+      Sets.newHashSet(
+        new Tuple2<>("california",
+          new Tuple2<>("dodgers", "giants")),
+        new Tuple2<>("new york",
+          new Tuple2<>("yankees", "mets"))),
+      Sets.newHashSet(
+        new Tuple2<>("california",
+          new Tuple2<>("sharks", "ducks")),
+        new Tuple2<>("new york",
+          new Tuple2<>("rangers", "islanders"))));
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+      ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+      ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<String, String>> joined =
+      pairStream1.transformWithToPair(pairStream2,(x, y, z) -> x.join(y));
+
+    JavaTestUtils.attachTestOutputStream(joined);
+    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+    List<Set<Tuple2<String, Tuple2<String, String>>>> unorderedResult = Lists.newArrayList();
+    for (List<Tuple2<String, Tuple2<String, String>>> res : result) {
+      unorderedResult.add(Sets.newHashSet(res));
+    }
+
+    Assert.assertEquals(expected, unorderedResult);
+  }
+
+
+  @Test
+  public void testVariousTransformWith() {
+    // tests whether all variations of transformWith can be called from Java
+
+    List<List<Integer>> inputData1 = Arrays.asList(Arrays.asList(1));
+    List<List<String>> inputData2 = Arrays.asList(Arrays.asList("x"));
+    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1);
+    JavaDStream<String> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1);
+
+    List<List<Tuple2<String, Integer>>> pairInputData1 =
+      Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
+    List<List<Tuple2<Double, Character>>> pairInputData2 =
+      Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x')));
+    JavaPairDStream<String, Integer> pairStream1 = JavaPairDStream.fromJavaDStream(
+      JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1));
+    JavaPairDStream<Double, Character> pairStream2 = JavaPairDStream.fromJavaDStream(
+      JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1));
+
+    JavaDStream<Double> transformed1 = stream1.transformWith(stream2, (x, y, z) -> null);
+    JavaDStream<Double> transformed2 = stream1.transformWith(pairStream1,(x, y, z) -> null);
+
+    JavaPairDStream<Double, Double> transformed3 =
+      stream1.transformWithToPair(stream2,(x, y, z) -> null);
+
+    JavaPairDStream<Double, Double> transformed4 =
+      stream1.transformWithToPair(pairStream1,(x, y, z) -> null);
+
+    JavaDStream<Double> pairTransformed1 = pairStream1.transformWith(stream2,(x, y, z) -> null);
+
+    JavaDStream<Double> pairTransformed2_ =
+      pairStream1.transformWith(pairStream1,(x, y, z) -> null);
+
+    JavaPairDStream<Double, Double> pairTransformed3 =
+      pairStream1.transformWithToPair(stream2,(x, y, z) -> null);
+
+    JavaPairDStream<Double, Double> pairTransformed4 =
+      pairStream1.transformWithToPair(pairStream2,(x, y, z) -> null);
+  }
+
+  @Test
+  public void testStreamingContextTransform() {
+    List<List<Integer>> stream1input = Arrays.asList(
+      Arrays.asList(1),
+      Arrays.asList(2)
+    );
+
+    List<List<Integer>> stream2input = Arrays.asList(
+      Arrays.asList(3),
+      Arrays.asList(4)
+    );
+
+    List<List<Tuple2<Integer, String>>> pairStream1input = Arrays.asList(
+      Arrays.asList(new Tuple2<>(1, "x")),
+      Arrays.asList(new Tuple2<>(2, "y"))
+    );
+
+    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> expected = Arrays.asList(
+      Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))),
+      Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y")))
+    );
+
+    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1);
+    JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1);
+    JavaPairDStream<Integer, String> pairStream1 = JavaPairDStream.fromJavaDStream(
+      JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1));
+
+    List<JavaDStream<?>> listOfDStreams1 = Arrays.<JavaDStream<?>>asList(stream1, stream2);
+
+    // This is just to test whether this transform to JavaStream compiles
+    JavaDStream<Long> transformed1 = ssc.transform(
+      listOfDStreams1, (List<JavaRDD<?>> listOfRDDs, Time time) -> {
+      Assert.assertEquals(2, listOfRDDs.size());
+      return null;
+    });
+
+    List<JavaDStream<?>> listOfDStreams2 =
+      Arrays.<JavaDStream<?>>asList(stream1, stream2, pairStream1.toJavaDStream());
+
+    JavaPairDStream<Integer, Tuple2<Integer, String>> transformed2 = ssc.transformToPair(
+      listOfDStreams2, (List<JavaRDD<?>> listOfRDDs, Time time) -> {
+      Assert.assertEquals(3, listOfRDDs.size());
+      JavaRDD<Integer> rdd1 = (JavaRDD<Integer>) listOfRDDs.get(0);
+      JavaRDD<Integer> rdd2 = (JavaRDD<Integer>) listOfRDDs.get(1);
+      JavaRDD<Tuple2<Integer, String>> rdd3 = (JavaRDD<Tuple2<Integer, String>>) listOfRDDs.get(2);
+      JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3);
+      PairFunction<Integer, Integer, Integer> mapToTuple =
+        (Integer i) -> new Tuple2<>(i, i);
+      return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3);
+    });
+    JavaTestUtils.attachTestOutputStream(transformed2);
+    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> result =
+      JavaTestUtils.runStreams(ssc, 2, 2);
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testFlatMap() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("go", "giants"),
+      Arrays.asList("boo", "dodgers"),
+      Arrays.asList("athletics"));
+
+    List<List<String>> expected = Arrays.asList(
+      Arrays.asList("g", "o", "g", "i", "a", "n", "t", "s"),
+      Arrays.asList("b", "o", "o", "d", "o", "d", "g", "e", "r", "s"),
+      Arrays.asList("a", "t", "h", "l", "e", "t", "i", "c", "s"));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<String> flatMapped = stream.flatMap(
+        s -> Lists.newArrayList(s.split("(?!^)")).iterator());
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testPairFlatMap() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("giants"),
+      Arrays.asList("dodgers"),
+      Arrays.asList("athletics"));
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>(6, "g"),
+        new Tuple2<>(6, "i"),
+        new Tuple2<>(6, "a"),
+        new Tuple2<>(6, "n"),
+        new Tuple2<>(6, "t"),
+        new Tuple2<>(6, "s")),
+      Arrays.asList(
+        new Tuple2<>(7, "d"),
+        new Tuple2<>(7, "o"),
+        new Tuple2<>(7, "d"),
+        new Tuple2<>(7, "g"),
+        new Tuple2<>(7, "e"),
+        new Tuple2<>(7, "r"),
+        new Tuple2<>(7, "s")),
+      Arrays.asList(
+        new Tuple2<>(9, "a"),
+        new Tuple2<>(9, "t"),
+        new Tuple2<>(9, "h"),
+        new Tuple2<>(9, "l"),
+        new Tuple2<>(9, "e"),
+        new Tuple2<>(9, "t"),
+        new Tuple2<>(9, "i"),
+        new Tuple2<>(9, "c"),
+        new Tuple2<>(9, "s")));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<Integer, String> flatMapped = stream.flatMapToPair(s -> {
+      List<Tuple2<Integer, String>> out = Lists.newArrayList();
+      for (String letter : s.split("(?!^)")) {
+        out.add(new Tuple2<>(s.length(), letter));
+      }
+      return out.iterator();
+    });
+
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  /*
+   * Performs an order-invariant comparison of lists representing two RDD streams. This allows
+   * us to account for ordering variation within individual RDD's which occurs during windowing.
+   */
+  public static <T extends Comparable<T>> void assertOrderInvariantEquals(
+    List<List<T>> expected, List<List<T>> actual) {
+    expected.forEach(list -> Collections.sort(list));
+    List<List<T>> sortedActual = new ArrayList<>();
+    actual.forEach(list -> {
+        List<T> sortedList = new ArrayList<>(list);
+        Collections.sort(sortedList);
+        sortedActual.add(sortedList);
+    });
+    Assert.assertEquals(expected, sortedActual);
+  }
+
+  @Test
+  public void testPairFilter() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("giants", "dodgers"),
+      Arrays.asList("yankees", "red sox"));
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+      Arrays.asList(new Tuple2<>("giants", 6)),
+      Arrays.asList(new Tuple2<>("yankees", 7)));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream =
+      stream.mapToPair(x -> new Tuple2<>(x, x.length()));
+    JavaPairDStream<String, Integer> filtered = pairStream.filter(x -> x._1().contains("a"));
+    JavaTestUtils.attachTestOutputStream(filtered);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  List<List<Tuple2<String, String>>> stringStringKVStream = Arrays.asList(
+    Arrays.asList(new Tuple2<>("california", "dodgers"),
+      new Tuple2<>("california", "giants"),
+      new Tuple2<>("new york", "yankees"),
+      new Tuple2<>("new york", "mets")),
+    Arrays.asList(new Tuple2<>("california", "sharks"),
+      new Tuple2<>("california", "ducks"),
+      new Tuple2<>("new york", "rangers"),
+      new Tuple2<>("new york", "islanders")));
+
+  List<List<Tuple2<String, Integer>>> stringIntKVStream = Arrays.asList(
+    Arrays.asList(
+      new Tuple2<>("california", 1),
+      new Tuple2<>("california", 3),
+      new Tuple2<>("new york", 4),
+      new Tuple2<>("new york", 1)),
+    Arrays.asList(
+      new Tuple2<>("california", 5),
+      new Tuple2<>("california", 5),
+      new Tuple2<>("new york", 3),
+      new Tuple2<>("new york", 1)));
+
+  @Test
+  public void testPairMap() { // Maps pair -> pair of different type
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>(1, "california"),
+        new Tuple2<>(3, "california"),
+        new Tuple2<>(4, "new york"),
+        new Tuple2<>(1, "new york")),
+      Arrays.asList(
+        new Tuple2<>(5, "california"),
+        new Tuple2<>(5, "california"),
+        new Tuple2<>(3, "new york"),
+        new Tuple2<>(1, "new york")));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> reversed = pairStream.mapToPair(x -> x.swap());
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairMapPartitions() { // Maps pair -> pair of different type
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>(1, "california"),
+        new Tuple2<>(3, "california"),
+        new Tuple2<>(4, "new york"),
+        new Tuple2<>(1, "new york")),
+      Arrays.asList(
+        new Tuple2<>(5, "california"),
+        new Tuple2<>(5, "california"),
+        new Tuple2<>(3, "new york"),
+        new Tuple2<>(1, "new york")));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> reversed = pairStream.mapPartitionsToPair(in -> {
+      LinkedList<Tuple2<Integer, String>> out = new LinkedList<>();
+      while (in.hasNext()) {
+        Tuple2<String, Integer> next = in.next();
+        out.add(next.swap());
+      }
+      return out.iterator();
+    });
+
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairMap2() { // Maps pair -> single
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Integer>> expected = Arrays.asList(
+      Arrays.asList(1, 3, 4, 1),
+      Arrays.asList(5, 5, 3, 1));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaDStream<Integer> reversed = pairStream.map(in -> in._2());
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair
+    List<List<Tuple2<String, Integer>>> inputData = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>("hi", 1),
+        new Tuple2<>("ho", 2)),
+      Arrays.asList(
+        new Tuple2<>("hi", 1),
+        new Tuple2<>("ho", 2)));
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>(1, "h"),
+        new Tuple2<>(1, "i"),
+        new Tuple2<>(2, "h"),
+        new Tuple2<>(2, "o")),
+      Arrays.asList(
+        new Tuple2<>(1, "h"),
+        new Tuple2<>(1, "i"),
+        new Tuple2<>(2, "h"),
+        new Tuple2<>(2, "o")));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> flatMapped = pairStream.flatMapToPair(in -> {
+      List<Tuple2<Integer, String>> out = new LinkedList<>();
+      for (Character s : in._1().toCharArray()) {
+        out.add(new Tuple2<>(in._2(), s.toString()));
+      }
+      return out.iterator();
+    });
+
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairReduceByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>("california", 4),
+        new Tuple2<>("new york", 5)),
+      Arrays.asList(
+        new Tuple2<>("california", 10),
+        new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+      ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduced = pairStream.reduceByKey((x, y) -> x + y);
+
+    JavaTestUtils.attachTestOutputStream(reduced);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testCombineByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>("california", 4),
+        new Tuple2<>("new york", 5)),
+      Arrays.asList(
+        new Tuple2<>("california", 10),
+        new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+      ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> combined = pairStream.<Integer>combineByKey(i -> i,
+      (x, y) -> x + y, (x, y) -> x + y, new HashPartitioner(2));
+
+    JavaTestUtils.attachTestOutputStream(combined);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testReduceByKeyAndWindow() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+      Arrays.asList(new Tuple2<>("california", 4),
+        new Tuple2<>("new york", 5)),
+      Arrays.asList(new Tuple2<>("california", 14),
+        new Tuple2<>("new york", 9)),
+      Arrays.asList(new Tuple2<>("california", 10),
+        new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduceWindowed =
+      pairStream.reduceByKeyAndWindow((x, y) -> x + y, new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reduceWindowed);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testUpdateStateByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+      Arrays.asList(new Tuple2<>("california", 4),
+        new Tuple2<>("new york", 5)),
+      Arrays.asList(new Tuple2<>("california", 14),
+        new Tuple2<>("new york", 9)),
+      Arrays.asList(new Tuple2<>("california", 14),
+        new Tuple2<>("new york", 9)));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey((values, state) -> {
+      int out = 0;
+      if (state.isPresent()) {
+        out = out + state.get();
+      }
+      for (Integer v : values) {
+        out = out + v;
+      }
+      return Optional.of(out);
+    });
+
+    JavaTestUtils.attachTestOutputStream(updated);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testReduceByKeyAndWindowWithInverse() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+      Arrays.asList(new Tuple2<>("california", 4),
+        new Tuple2<>("new york", 5)),
+      Arrays.asList(new Tuple2<>("california", 14),
+        new Tuple2<>("new york", 9)),
+      Arrays.asList(new Tuple2<>("california", 10),
+        new Tuple2<>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduceWindowed =
+      pairStream.reduceByKeyAndWindow((x, y) -> x + y, (x, y) -> x - y, new Duration(2000),
+        new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reduceWindowed);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairTransform() {
+    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>(3, 5),
+        new Tuple2<>(1, 5),
+        new Tuple2<>(4, 5),
+        new Tuple2<>(2, 5)),
+      Arrays.asList(
+        new Tuple2<>(2, 5),
+        new Tuple2<>(3, 5),
+        new Tuple2<>(4, 5),
+        new Tuple2<>(1, 5)));
+
+    List<List<Tuple2<Integer, Integer>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>(1, 5),
+        new Tuple2<>(2, 5),
+        new Tuple2<>(3, 5),
+        new Tuple2<>(4, 5)),
+      Arrays.asList(
+        new Tuple2<>(1, 5),
+        new Tuple2<>(2, 5),
+        new Tuple2<>(3, 5),
+        new Tuple2<>(4, 5)));
+
+    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
+      ssc, inputData, 1);
+    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<Integer, Integer> sorted = pairStream.transformToPair(in -> in.sortByKey());
+
+    JavaTestUtils.attachTestOutputStream(sorted);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairToNormalRDDTransform() {
+    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<>(3, 5),
+        new Tuple2<>(1, 5),
+        new Tuple2<>(4, 5),
+        new Tuple2<>(2, 5)),
+      Arrays.asList(
+        new Tuple2<>(2, 5),
+        new Tuple2<>(3, 5),
+        new Tuple2<>(4, 5),
+        new Tuple2<>(1, 5)));
+
+    List<List<Integer>> expected = Arrays.asList(
+      Arrays.asList(3, 1, 4, 2),
+      Arrays.asList(2, 3, 4, 1));
+
+    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
+      ssc, inputData, 1);
+    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaDStream<Integer> firstParts = pairStream.transform(in -> in.map(x -> x._1()));
+    JavaTestUtils.attachTestOutputStream(firstParts);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testMapValues() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, String>>> expected = Arrays.asList(
+      Arrays.asList(new Tuple2<>("california", "DODGERS"),
+        new Tuple2<>("california", "GIANTS"),
+        new Tuple2<>("new york", "YANKEES"),
+        new Tuple2<>("new york", "METS")),
+      Arrays.asList(new Tuple2<>("california", "SHARKS"),
+        new Tuple2<>("california", "DUCKS"),
+        new Tuple2<>("new york", "RANGERS"),
+        new Tuple2<>("new york", "ISLANDERS")));
+
+    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+      ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, String> mapped = pairStream.mapValues(String::toUpperCase);
+    JavaTestUtils.attachTestOutputStream(mapped);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testFlatMapValues() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, String>>> expected = Arrays.asList(
+      Arrays.asList(new Tuple2<>("california", "dodgers1"),
+        new Tuple2<>("california", "dodgers2"),
+        new Tuple2<>("california", "giants1"),
+        new Tuple2<>("california", "giants2"),
+        new Tuple2<>("new york", "yankees1"),
+        new Tuple2<>("new york", "yankees2"),
+        new Tuple2<>("new york", "mets1"),
+        new Tuple2<>("new york", "mets2")),
+      Arrays.asList(new Tuple2<>("california", "sharks1"),
+        new Tuple2<>("california", "sharks2"),
+        new Tuple2<>("california", "ducks1"),
+        new Tuple2<>("california", "ducks2"),
+        new Tuple2<>("new york", "rangers1"),
+        new Tuple2<>("new york", "rangers2"),
+        new Tuple2<>("new york", "islanders1"),
+        new Tuple2<>("new york", "islanders2")));
+
+    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+      ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, String> flatMapped =
+      pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2"));
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+    Assert.assertEquals(expected, result);
+  }
+
+  /**
+   * This test is only for testing the APIs. It's not necessary to run it.
+   */
+  public void testMapWithStateAPI() {
+    JavaPairRDD<String, Boolean> initialRDD = null;
+    JavaPairDStream<String, Integer> wordsDstream = null;
+
+    JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream =
+        wordsDstream.mapWithState(
+            StateSpec.<String, Integer, Boolean, Double>function((time, key, value, state) -> {
+              // Use all State's methods here
+              state.exists();
+              state.get();
+              state.isTimingOut();
+              state.remove();
+              state.update(true);
+              return Optional.of(2.0);
+            }).initialState(initialRDD)
+                .numPartitions(10)
+                .partitioner(new HashPartitioner(10))
+                .timeout(Durations.seconds(10)));
+
+    JavaPairDStream<String, Boolean> emittedRecords = stateDstream.stateSnapshots();
+
+    JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream2 =
+        wordsDstream.mapWithState(
+            StateSpec.<String, Integer, Boolean, Double>function((key, value, state) -> {
+              state.exists();
+              state.get();
+              state.isTimingOut();
+              state.remove();
+              state.update(true);
+              return 2.0;
+            }).initialState(initialRDD)
+                .numPartitions(10)
+                .partitioner(new HashPartitioner(10))
+                .timeout(Durations.seconds(10)));
+
+    JavaPairDStream<String, Boolean> mappedDStream = stateDstream2.stateSnapshots();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[8/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Posted by sr...@apache.org.
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings

For the future:

- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas

## How was this patch tested?

Existing tests

Author: Sean Owen <so...@cloudera.com>

Closes #16871 from srowen/SPARK-19493.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0e240549
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0e240549
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0e240549

Branch: refs/heads/master
Commit: 0e2405490f2056728d1353abbac6f3ea177ae533
Parents: 3871d94
Author: Sean Owen <so...@cloudera.com>
Authored: Thu Feb 16 12:32:45 2017 +0000
Committer: Sean Owen <so...@cloudera.com>
Committed: Thu Feb 16 12:32:45 2017 +0000

----------------------------------------------------------------------
 assembly/pom.xml                                |    1 +
 build/mvn                                       |    8 +-
 build/sbt-launch-lib.bash                       |    2 +-
 .../spark/network/client/TransportClient.java   |  111 +-
 .../network/crypto/AuthClientBootstrap.java     |   16 +-
 .../spark/network/crypto/AuthRpcHandler.java    |    3 -
 .../network/server/TransportRequestHandler.java |   27 +-
 .../spark/network/crypto/AuthEngineSuite.java   |    2 -
 .../shuffle/ExternalShuffleBlockHandler.java    |    8 +-
 .../shuffle/ExternalShuffleBlockResolver.java   |    7 +-
 .../network/shuffle/ExternalShuffleClient.java  |   21 +-
 .../network/shuffle/RetryingBlockFetcher.java   |    9 +-
 common/sketch/pom.xml                           |    2 +
 common/unsafe/pom.xml                           |    2 +
 .../java/org/apache/spark/unsafe/Platform.java  |    9 +-
 .../spark/unsafe/types/CalendarInterval.java    |   88 +-
 .../org/apache/spark/api/java/Optional.java     |    7 +-
 .../api/java/function/CoGroupFunction.java      |    1 +
 .../java/function/DoubleFlatMapFunction.java    |    1 +
 .../spark/api/java/function/DoubleFunction.java |    1 +
 .../spark/api/java/function/FilterFunction.java |    1 +
 .../api/java/function/FlatMapFunction.java      |    1 +
 .../api/java/function/FlatMapFunction2.java     |    1 +
 .../java/function/FlatMapGroupsFunction.java    |    1 +
 .../api/java/function/ForeachFunction.java      |    1 +
 .../java/function/ForeachPartitionFunction.java |    1 +
 .../spark/api/java/function/Function.java       |    1 +
 .../spark/api/java/function/Function0.java      |    1 +
 .../spark/api/java/function/Function2.java      |    1 +
 .../spark/api/java/function/Function3.java      |    1 +
 .../spark/api/java/function/Function4.java      |    1 +
 .../spark/api/java/function/MapFunction.java    |    1 +
 .../api/java/function/MapGroupsFunction.java    |    1 +
 .../java/function/MapPartitionsFunction.java    |    1 +
 .../api/java/function/PairFlatMapFunction.java  |    1 +
 .../spark/api/java/function/PairFunction.java   |    1 +
 .../spark/api/java/function/ReduceFunction.java |    1 +
 .../spark/api/java/function/VoidFunction.java   |    1 +
 .../spark/api/java/function/VoidFunction2.java  |    1 +
 .../unsafe/sort/UnsafeExternalSorter.java       |    9 +-
 .../unsafe/sort/UnsafeSorterSpillMerger.java    |   28 +-
 .../scala/org/apache/spark/SparkContext.scala   |    3 -
 .../spark/launcher/WorkerCommandBuilder.scala   |    1 -
 .../scala/org/apache/spark/util/Utils.scala     |   44 +-
 .../java/org/apache/spark/JavaAPISuite.java     | 1836 ----------------
 .../test/org/apache/spark/Java8RDDAPISuite.java |  356 ++++
 .../test/org/apache/spark/JavaAPISuite.java     | 1842 ++++++++++++++++
 .../org/apache/spark/util/UtilsSuite.scala      |    6 +-
 dev/appveyor-install-dependencies.ps1           |    2 +-
 dev/create-release/release-build.sh             |    1 -
 dev/make-distribution.sh                        |    2 +-
 dev/mima                                        |    1 -
 dev/run-tests.py                                |    3 -
 dev/test-dependencies.sh                        |    2 +-
 docs/building-spark.md                          |   32 +-
 docs/index.md                                   |    6 +-
 docs/mllib-linear-methods.md                    |    2 +-
 docs/mllib-statistics.md                        |    7 +-
 docs/programming-guide.md                       |   11 +-
 docs/quick-start.md                             |    9 +-
 docs/streaming-custom-receivers.md              |   10 +-
 docs/streaming-kafka-0-10-integration.md        |   62 +-
 docs/streaming-kafka-0-8-integration.md         |   41 +-
 docs/streaming-programming-guide.md             |  219 +-
 docs/structured-streaming-programming-guide.md  |   38 +-
 .../spark/examples/ml/JavaTokenizerExample.java |    4 +-
 .../examples/sql/JavaSQLDataSourceExample.java  |    2 +-
 external/java8-tests/README.md                  |   22 -
 external/java8-tests/pom.xml                    |  132 --
 .../apache/spark/java8/Java8RDDAPISuite.java    |  356 ----
 .../spark/java8/dstream/Java8APISuite.java      |  882 --------
 .../java8/sql/Java8DatasetAggregatorSuite.java  |   62 -
 .../src/test/resources/log4j.properties         |   27 -
 .../org/apache/spark/java8/JDK8ScalaSuite.scala |   30 -
 .../apache/spark/sql/kafka010/KafkaSource.scala |    3 +-
 .../spark/streaming/kafka010/KafkaRDD.scala     |    7 +-
 .../spark/launcher/AbstractCommandBuilder.java  |    7 +-
 .../spark/launcher/ChildProcAppHandle.java      |   10 +-
 .../spark/launcher/CommandBuilderUtils.java     |   21 -
 .../apache/spark/launcher/LauncherServer.java   |    7 +-
 .../apache/spark/launcher/OutputRedirector.java |    7 +-
 .../apache/spark/launcher/SparkAppHandle.java   |    3 -
 .../launcher/SparkClassCommandBuilder.java      |   68 +-
 .../launcher/SparkSubmitCommandBuilder.java     |  101 +-
 .../launcher/CommandBuilderUtilsSuite.java      |   36 -
 .../SparkSubmitCommandBuilderSuite.java         |    8 +-
 launcher/src/test/resources/spark-defaults.conf |    2 +-
 pom.xml                                         |  171 +-
 project/SparkBuild.scala                        |   41 +-
 .../org/apache/spark/deploy/yarn/Client.scala   |    1 -
 .../spark/deploy/yarn/ExecutorRunnable.scala    |    2 -
 .../launcher/YarnCommandBuilderUtils.scala      |   12 -
 .../org/apache/spark/sql/types/Decimal.scala    |    8 +-
 .../FlatMapGroupsWithStateFunction.java         |    4 +-
 .../function/MapGroupsWithStateFunction.java    |    4 +-
 .../scala/org/apache/spark/sql/Dataset.scala    |    2 +-
 .../spark/sql/KeyValueGroupedDataset.scala      |    2 +-
 .../scala/org/apache/spark/sql/SQLContext.scala |   11 -
 .../org/apache/spark/sql/SparkSession.scala     |   11 -
 .../spark/sql/Java8DatasetAggregatorSuite.java  |   61 +
 .../org/apache/spark/sql/JavaDatasetSuite.java  |    2 +-
 sql/hive/pom.xml                                |    3 +-
 .../execution/ScriptTransformationExec.scala    |   10 +-
 .../apache/spark/streaming/JavaAPISuite.java    | 2000 -----------------
 .../apache/spark/streaming/Java8APISuite.java   |  887 ++++++++
 .../apache/spark/streaming/JavaAPISuite.java    | 2008 ++++++++++++++++++
 106 files changed, 5641 insertions(+), 6314 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 53f1879..9d8607d 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -187,6 +187,7 @@
           <plugin>
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-assembly-plugin</artifactId>
+            <version>3.0.0</version>
             <executions>
               <execution>
                 <id>dist</id>

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/build/mvn
----------------------------------------------------------------------
diff --git a/build/mvn b/build/mvn
index 866bad8..1e393c3 100755
--- a/build/mvn
+++ b/build/mvn
@@ -22,7 +22,7 @@ _DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
 # Preserve the calling directory
 _CALLING_DIR="$(pwd)"
 # Options used during compilation
-_COMPILE_JVM_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"
+_COMPILE_JVM_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m"
 
 # Installs any application tarball given a URL, the expected tarball name,
 # and, optionally, a checkable binary path to determine if the binary has
@@ -141,13 +141,9 @@ cd "${_CALLING_DIR}"
 # Now that zinc is ensured to be installed, check its status and, if its
 # not running or just installed, start it
 if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}`" ]; then
-  ZINC_JAVA_HOME=
-  if [ -n "$JAVA_7_HOME" ]; then
-    ZINC_JAVA_HOME="env JAVA_HOME=$JAVA_7_HOME"
-  fi
   export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"}
   "${ZINC_BIN}" -shutdown -port ${ZINC_PORT}
-  $ZINC_JAVA_HOME "${ZINC_BIN}" -start -port ${ZINC_PORT} \
+  "${ZINC_BIN}" -start -port ${ZINC_PORT} \
     -scala-compiler "${SCALA_COMPILER}" \
     -scala-library "${SCALA_LIBRARY}" &>/dev/null
 fi

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/build/sbt-launch-lib.bash
----------------------------------------------------------------------
diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash
index 615f848..4732669 100755
--- a/build/sbt-launch-lib.bash
+++ b/build/sbt-launch-lib.bash
@@ -117,7 +117,7 @@ get_mem_opts () {
   (( $perm < 4096 )) || perm=4096
   local codecache=$(( $perm / 2 ))
 
-  echo "-Xms${mem}m -Xmx${mem}m -XX:MaxPermSize=${perm}m -XX:ReservedCodeCacheSize=${codecache}m"
+  echo "-Xms${mem}m -Xmx${mem}m -XX:ReservedCodeCacheSize=${codecache}m"
 }
 
 require_arg () {

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java
----------------------------------------------------------------------
diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java
index 7e7d78d..a6f527c 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java
@@ -32,8 +32,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.SettableFuture;
 import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -133,40 +131,36 @@ public class TransportClient implements Closeable {
    */
   public void fetchChunk(
       long streamId,
-      final int chunkIndex,
-      final ChunkReceivedCallback callback) {
-    final long startTime = System.currentTimeMillis();
+      int chunkIndex,
+      ChunkReceivedCallback callback) {
+    long startTime = System.currentTimeMillis();
     if (logger.isDebugEnabled()) {
       logger.debug("Sending fetch chunk request {} to {}", chunkIndex, getRemoteAddress(channel));
     }
 
-    final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
+    StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex);
     handler.addFetchRequest(streamChunkId, callback);
 
-    channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(
-      new ChannelFutureListener() {
-        @Override
-        public void operationComplete(ChannelFuture future) throws Exception {
-          if (future.isSuccess()) {
-            long timeTaken = System.currentTimeMillis() - startTime;
-            if (logger.isTraceEnabled()) {
-              logger.trace("Sending request {} to {} took {} ms", streamChunkId,
-                getRemoteAddress(channel), timeTaken);
-            }
-          } else {
-            String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
-              getRemoteAddress(channel), future.cause());
-            logger.error(errorMsg, future.cause());
-            handler.removeFetchRequest(streamChunkId);
-            channel.close();
-            try {
-              callback.onFailure(chunkIndex, new IOException(errorMsg, future.cause()));
-            } catch (Exception e) {
-              logger.error("Uncaught exception in RPC response callback handler!", e);
-            }
-          }
+    channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(future -> {
+      if (future.isSuccess()) {
+        long timeTaken = System.currentTimeMillis() - startTime;
+        if (logger.isTraceEnabled()) {
+          logger.trace("Sending request {} to {} took {} ms", streamChunkId,
+            getRemoteAddress(channel), timeTaken);
         }
-      });
+      } else {
+        String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId,
+          getRemoteAddress(channel), future.cause());
+        logger.error(errorMsg, future.cause());
+        handler.removeFetchRequest(streamChunkId);
+        channel.close();
+        try {
+          callback.onFailure(chunkIndex, new IOException(errorMsg, future.cause()));
+        } catch (Exception e) {
+          logger.error("Uncaught exception in RPC response callback handler!", e);
+        }
+      }
+    });
   }
 
   /**
@@ -175,8 +169,8 @@ public class TransportClient implements Closeable {
    * @param streamId The stream to fetch.
    * @param callback Object to call with the stream data.
    */
-  public void stream(final String streamId, final StreamCallback callback) {
-    final long startTime = System.currentTimeMillis();
+  public void stream(String streamId, StreamCallback callback) {
+    long startTime = System.currentTimeMillis();
     if (logger.isDebugEnabled()) {
       logger.debug("Sending stream request for {} to {}", streamId, getRemoteAddress(channel));
     }
@@ -186,29 +180,25 @@ public class TransportClient implements Closeable {
     // when responses arrive.
     synchronized (this) {
       handler.addStreamCallback(callback);
-      channel.writeAndFlush(new StreamRequest(streamId)).addListener(
-        new ChannelFutureListener() {
-          @Override
-          public void operationComplete(ChannelFuture future) throws Exception {
-            if (future.isSuccess()) {
-              long timeTaken = System.currentTimeMillis() - startTime;
-              if (logger.isTraceEnabled()) {
-                logger.trace("Sending request for {} to {} took {} ms", streamId,
-                  getRemoteAddress(channel), timeTaken);
-              }
-            } else {
-              String errorMsg = String.format("Failed to send request for %s to %s: %s", streamId,
-                getRemoteAddress(channel), future.cause());
-              logger.error(errorMsg, future.cause());
-              channel.close();
-              try {
-                callback.onFailure(streamId, new IOException(errorMsg, future.cause()));
-              } catch (Exception e) {
-                logger.error("Uncaught exception in RPC response callback handler!", e);
-              }
-            }
+      channel.writeAndFlush(new StreamRequest(streamId)).addListener(future -> {
+        if (future.isSuccess()) {
+          long timeTaken = System.currentTimeMillis() - startTime;
+          if (logger.isTraceEnabled()) {
+            logger.trace("Sending request for {} to {} took {} ms", streamId,
+              getRemoteAddress(channel), timeTaken);
           }
-        });
+        } else {
+          String errorMsg = String.format("Failed to send request for %s to %s: %s", streamId,
+            getRemoteAddress(channel), future.cause());
+          logger.error(errorMsg, future.cause());
+          channel.close();
+          try {
+            callback.onFailure(streamId, new IOException(errorMsg, future.cause()));
+          } catch (Exception e) {
+            logger.error("Uncaught exception in RPC response callback handler!", e);
+          }
+        }
+      });
     }
   }
 
@@ -220,19 +210,17 @@ public class TransportClient implements Closeable {
    * @param callback Callback to handle the RPC's reply.
    * @return The RPC's id.
    */
-  public long sendRpc(ByteBuffer message, final RpcResponseCallback callback) {
-    final long startTime = System.currentTimeMillis();
+  public long sendRpc(ByteBuffer message, RpcResponseCallback callback) {
+    long startTime = System.currentTimeMillis();
     if (logger.isTraceEnabled()) {
       logger.trace("Sending RPC to {}", getRemoteAddress(channel));
     }
 
-    final long requestId = Math.abs(UUID.randomUUID().getLeastSignificantBits());
+    long requestId = Math.abs(UUID.randomUUID().getLeastSignificantBits());
     handler.addRpcRequest(requestId, callback);
 
-    channel.writeAndFlush(new RpcRequest(requestId, new NioManagedBuffer(message))).addListener(
-      new ChannelFutureListener() {
-        @Override
-        public void operationComplete(ChannelFuture future) throws Exception {
+    channel.writeAndFlush(new RpcRequest(requestId, new NioManagedBuffer(message)))
+        .addListener(future -> {
           if (future.isSuccess()) {
             long timeTaken = System.currentTimeMillis() - startTime;
             if (logger.isTraceEnabled()) {
@@ -251,8 +239,7 @@ public class TransportClient implements Closeable {
               logger.error("Uncaught exception in RPC response callback handler!", e);
             }
           }
-        }
-      });
+        });
 
     return requestId;
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java
----------------------------------------------------------------------
diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java
index 980525d..799f454 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java
@@ -20,12 +20,7 @@ package org.apache.spark.network.crypto;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.GeneralSecurityException;
-import java.security.Key;
-import javax.crypto.KeyGenerator;
-import javax.crypto.Mac;
-import static java.nio.charset.StandardCharsets.UTF_8;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
@@ -37,7 +32,6 @@ import org.apache.spark.network.client.TransportClient;
 import org.apache.spark.network.client.TransportClientBootstrap;
 import org.apache.spark.network.sasl.SaslClientBootstrap;
 import org.apache.spark.network.sasl.SecretKeyHolder;
-import org.apache.spark.network.util.JavaUtils;
 import org.apache.spark.network.util.TransportConf;
 
 /**
@@ -103,20 +97,18 @@ public class AuthClientBootstrap implements TransportClientBootstrap {
   private void doSparkAuth(TransportClient client, Channel channel)
     throws GeneralSecurityException, IOException {
 
-    AuthEngine engine = new AuthEngine(authUser, secretKeyHolder.getSecretKey(authUser), conf);
-    try {
+    String secretKey = secretKeyHolder.getSecretKey(authUser);
+    try (AuthEngine engine = new AuthEngine(authUser, secretKey, conf)) {
       ClientChallenge challenge = engine.challenge();
       ByteBuf challengeData = Unpooled.buffer(challenge.encodedLength());
       challenge.encode(challengeData);
 
-      ByteBuffer responseData = client.sendRpcSync(challengeData.nioBuffer(),
-        conf.authRTTimeoutMs());
+      ByteBuffer responseData =
+          client.sendRpcSync(challengeData.nioBuffer(), conf.authRTTimeoutMs());
       ServerResponse response = ServerResponse.decodeMessage(responseData);
 
       engine.validate(response);
       engine.sessionCipher().addToChannel(channel);
-    } finally {
-      engine.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java
----------------------------------------------------------------------
diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java
index 991d8ba..0a5c029 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java
@@ -17,9 +17,7 @@
 
 package org.apache.spark.network.crypto;
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
-import javax.security.sasl.Sasl;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
@@ -35,7 +33,6 @@ import org.apache.spark.network.sasl.SecretKeyHolder;
 import org.apache.spark.network.sasl.SaslRpcHandler;
 import org.apache.spark.network.server.RpcHandler;
 import org.apache.spark.network.server.StreamManager;
-import org.apache.spark.network.util.JavaUtils;
 import org.apache.spark.network.util.TransportConf;
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
----------------------------------------------------------------------
diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
index 900e8eb..8193bc1 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
@@ -22,8 +22,6 @@ import java.nio.ByteBuffer;
 
 import com.google.common.base.Throwables;
 import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -189,21 +187,16 @@ public class TransportRequestHandler extends MessageHandler<RequestMessage> {
    * Responds to a single message with some Encodable object. If a failure occurs while sending,
    * it will be logged and the channel closed.
    */
-  private void respond(final Encodable result) {
-    final SocketAddress remoteAddress = channel.remoteAddress();
-    channel.writeAndFlush(result).addListener(
-      new ChannelFutureListener() {
-        @Override
-        public void operationComplete(ChannelFuture future) throws Exception {
-          if (future.isSuccess()) {
-            logger.trace("Sent result {} to client {}", result, remoteAddress);
-          } else {
-            logger.error(String.format("Error sending result %s to %s; closing connection",
-              result, remoteAddress), future.cause());
-            channel.close();
-          }
-        }
+  private void respond(Encodable result) {
+    SocketAddress remoteAddress = channel.remoteAddress();
+    channel.writeAndFlush(result).addListener(future -> {
+      if (future.isSuccess()) {
+        logger.trace("Sent result {} to client {}", result, remoteAddress);
+      } else {
+        logger.error(String.format("Error sending result %s to %s; closing connection",
+          result, remoteAddress), future.cause());
+        channel.close();
       }
-    );
+    });
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java
----------------------------------------------------------------------
diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java
index 9a186f2..a3519fe 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java
@@ -18,10 +18,8 @@
 package org.apache.spark.network.crypto;
 
 import java.util.Arrays;
-import java.util.Map;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import com.google.common.collect.ImmutableMap;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
----------------------------------------------------------------------
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
index 6e02430..6daf960 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
@@ -190,12 +190,8 @@ public class ExternalShuffleBlockHandler extends RpcHandler {
       allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis);
       allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis);
       allMetrics.put("blockTransferRateBytes", blockTransferRateBytes);
-      allMetrics.put("registeredExecutorsSize", new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return blockManager.getRegisteredExecutorsSize();
-        }
-      });
+      allMetrics.put("registeredExecutorsSize",
+                     (Gauge<Integer>) () -> blockManager.getRegisteredExecutorsSize());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
----------------------------------------------------------------------
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
index 25e9abd..62d58ab 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
@@ -205,12 +205,7 @@ public class ExternalShuffleBlockResolver {
           logger.info("Cleaning up executor {}'s {} local dirs", fullId, executor.localDirs.length);
 
           // Execute the actual deletion in a different thread, as it may take some time.
-          directoryCleaner.execute(new Runnable() {
-            @Override
-            public void run() {
-              deleteExecutorDirs(executor.localDirs);
-            }
-          });
+          directoryCleaner.execute(() -> deleteExecutorDirs(executor.localDirs));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java
----------------------------------------------------------------------
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java
index 8c0c400..2c5827b 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java
@@ -82,23 +82,19 @@ public class ExternalShuffleClient extends ShuffleClient {
 
   @Override
   public void fetchBlocks(
-      final String host,
-      final int port,
-      final String execId,
+      String host,
+      int port,
+      String execId,
       String[] blockIds,
       BlockFetchingListener listener) {
     checkInit();
     logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId);
     try {
       RetryingBlockFetcher.BlockFetchStarter blockFetchStarter =
-        new RetryingBlockFetcher.BlockFetchStarter() {
-          @Override
-          public void createAndStart(String[] blockIds, BlockFetchingListener listener)
-              throws IOException, InterruptedException {
+          (blockIds1, listener1) -> {
             TransportClient client = clientFactory.createClient(host, port);
-            new OneForOneBlockFetcher(client, appId, execId, blockIds, listener).start();
-          }
-        };
+            new OneForOneBlockFetcher(client, appId, execId, blockIds1, listener1).start();
+          };
 
       int maxRetries = conf.maxIORetries();
       if (maxRetries > 0) {
@@ -131,12 +127,9 @@ public class ExternalShuffleClient extends ShuffleClient {
       String execId,
       ExecutorShuffleInfo executorInfo) throws IOException, InterruptedException {
     checkInit();
-    TransportClient client = clientFactory.createUnmanagedClient(host, port);
-    try {
+    try (TransportClient client = clientFactory.createUnmanagedClient(host, port)) {
       ByteBuffer registerMessage = new RegisterExecutor(appId, execId, executorInfo).toByteBuffer();
       client.sendRpcSync(registerMessage, 5000 /* timeoutMs */);
-    } finally {
-      client.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java
----------------------------------------------------------------------
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java
index 5be8550..f309dda 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java
@@ -164,12 +164,9 @@ public class RetryingBlockFetcher {
     logger.info("Retrying fetch ({}/{}) for {} outstanding blocks after {} ms",
       retryCount, maxRetries, outstandingBlocksIds.size(), retryWaitTime);
 
-    executorService.submit(new Runnable() {
-      @Override
-      public void run() {
-        Uninterruptibles.sleepUninterruptibly(retryWaitTime, TimeUnit.MILLISECONDS);
-        fetchAllOutstanding();
-      }
+    executorService.submit(() -> {
+      Uninterruptibles.sleepUninterruptibly(retryWaitTime, TimeUnit.MILLISECONDS);
+      fetchAllOutstanding();
     });
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/sketch/pom.xml
----------------------------------------------------------------------
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index bcd26d4..1356c47 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -61,6 +61,7 @@
         <plugin>
           <groupId>net.alchim31.maven</groupId>
           <artifactId>scala-maven-plugin</artifactId>
+          <version>3.2.2</version>
           <configuration>
             <javacArgs combine.children="append">
               <!-- This option is needed to suppress warnings from sun.misc.Unsafe usage -->
@@ -71,6 +72,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
+          <version>3.6.1</version>
           <configuration>
             <compilerArgs combine.children="append">
               <!-- This option is needed to suppress warnings from sun.misc.Unsafe usage -->

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/unsafe/pom.xml
----------------------------------------------------------------------
diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml
index dc19f4a..f03a4da 100644
--- a/common/unsafe/pom.xml
+++ b/common/unsafe/pom.xml
@@ -98,6 +98,7 @@
         <plugin>
           <groupId>net.alchim31.maven</groupId>
           <artifactId>scala-maven-plugin</artifactId>
+          <version>3.2.2</version>
           <configuration>
             <javacArgs combine.children="append">
               <!-- This option is needed to suppress warnings from sun.misc.Unsafe usage -->
@@ -108,6 +109,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
+          <version>3.6.1</version>
           <configuration>
             <compilerArgs combine.children="append">
               <!-- This option is needed to suppress warnings from sun.misc.Unsafe usage -->

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java
----------------------------------------------------------------------
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java
index 671b8c7..f13c24a 100644
--- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java
@@ -162,14 +162,9 @@ public final class Platform {
       constructor.setAccessible(true);
       Field cleanerField = cls.getDeclaredField("cleaner");
       cleanerField.setAccessible(true);
-      final long memory = allocateMemory(size);
+      long memory = allocateMemory(size);
       ByteBuffer buffer = (ByteBuffer) constructor.newInstance(memory, size);
-      Cleaner cleaner = Cleaner.create(buffer, new Runnable() {
-        @Override
-        public void run() {
-          freeMemory(memory);
-        }
-      });
+      Cleaner cleaner = Cleaner.create(buffer, () -> freeMemory(memory));
       cleanerField.set(buffer, cleaner);
       return buffer;
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
----------------------------------------------------------------------
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
index fd6e95c..621f2c6 100644
--- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
@@ -178,48 +178,52 @@ public final class CalendarInterval implements Serializable {
         "Interval string does not match day-time format of 'd h:m:s.n': " + s);
     } else {
       try {
-        if (unit.equals("year")) {
-          int year = (int) toLongWithRange("year", m.group(1),
-            Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12);
-          result = new CalendarInterval(year * 12, 0L);
-
-        } else if (unit.equals("month")) {
-          int month = (int) toLongWithRange("month", m.group(1),
-            Integer.MIN_VALUE, Integer.MAX_VALUE);
-          result = new CalendarInterval(month, 0L);
-
-        } else if (unit.equals("week")) {
-          long week = toLongWithRange("week", m.group(1),
-                  Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK);
-          result = new CalendarInterval(0, week * MICROS_PER_WEEK);
-
-        } else if (unit.equals("day")) {
-          long day = toLongWithRange("day", m.group(1),
-            Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY);
-          result = new CalendarInterval(0, day * MICROS_PER_DAY);
-
-        } else if (unit.equals("hour")) {
-          long hour = toLongWithRange("hour", m.group(1),
-            Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR);
-          result = new CalendarInterval(0, hour * MICROS_PER_HOUR);
-
-        } else if (unit.equals("minute")) {
-          long minute = toLongWithRange("minute", m.group(1),
-            Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE);
-          result = new CalendarInterval(0, minute * MICROS_PER_MINUTE);
-
-        } else if (unit.equals("second")) {
-          long micros = parseSecondNano(m.group(1));
-          result = new CalendarInterval(0, micros);
-
-        } else if (unit.equals("millisecond")) {
-          long millisecond = toLongWithRange("millisecond", m.group(1),
-                  Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI);
-          result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI);
-
-        } else if (unit.equals("microsecond")) {
-          long micros = Long.parseLong(m.group(1));
-          result = new CalendarInterval(0, micros);
+        switch (unit) {
+          case "year":
+            int year = (int) toLongWithRange("year", m.group(1),
+              Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12);
+            result = new CalendarInterval(year * 12, 0L);
+            break;
+          case "month":
+            int month = (int) toLongWithRange("month", m.group(1),
+              Integer.MIN_VALUE, Integer.MAX_VALUE);
+            result = new CalendarInterval(month, 0L);
+            break;
+          case "week":
+            long week = toLongWithRange("week", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK);
+            result = new CalendarInterval(0, week * MICROS_PER_WEEK);
+            break;
+          case "day":
+            long day = toLongWithRange("day", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY);
+            result = new CalendarInterval(0, day * MICROS_PER_DAY);
+            break;
+          case "hour":
+            long hour = toLongWithRange("hour", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR);
+            result = new CalendarInterval(0, hour * MICROS_PER_HOUR);
+            break;
+          case "minute":
+            long minute = toLongWithRange("minute", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE);
+            result = new CalendarInterval(0, minute * MICROS_PER_MINUTE);
+            break;
+          case "second": {
+            long micros = parseSecondNano(m.group(1));
+            result = new CalendarInterval(0, micros);
+            break;
+          }
+          case "millisecond":
+            long millisecond = toLongWithRange("millisecond", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI);
+            result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI);
+            break;
+          case "microsecond": {
+            long micros = Long.parseLong(m.group(1));
+            result = new CalendarInterval(0, micros);
+            break;
+          }
         }
       } catch (Exception e) {
         throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/Optional.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/Optional.java b/core/src/main/java/org/apache/spark/api/java/Optional.java
index ca7babc..fd0f495 100644
--- a/core/src/main/java/org/apache/spark/api/java/Optional.java
+++ b/core/src/main/java/org/apache/spark/api/java/Optional.java
@@ -18,6 +18,7 @@
 package org.apache.spark.api.java;
 
 import java.io.Serializable;
+import java.util.Objects;
 
 import com.google.common.base.Preconditions;
 
@@ -52,8 +53,8 @@ import com.google.common.base.Preconditions;
  *   <li>{@link #isPresent()}</li>
  * </ul>
  *
- * <p>{@code java.util.Optional} itself is not used at this time because the
- * project does not require Java 8. Using {@code com.google.common.base.Optional}
+ * <p>{@code java.util.Optional} itself was not used because at the time, the
+ * project did not require Java 8. Using {@code com.google.common.base.Optional}
  * has in the past caused serious library version conflicts with Guava that can't
  * be resolved by shading. Hence this work-alike clone.</p>
  *
@@ -171,7 +172,7 @@ public final class Optional<T> implements Serializable {
       return false;
     }
     Optional<?> other = (Optional<?>) obj;
-    return value == null ? other.value == null : value.equals(other.value);
+    return Objects.equals(value, other.value);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java b/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java
index 07aebb7..33bedf7 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
  * A function that returns zero or more output records from each grouping key and its values from 2
  * Datasets.
  */
+@FunctionalInterface
 public interface CoGroupFunction<K, V1, V2, R> extends Serializable {
   Iterator<R> call(K key, Iterator<V1> left, Iterator<V2> right) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
index 576087b..2f23da5 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 /**
  * A function that returns zero or more records of type Double from each input record.
  */
+@FunctionalInterface
 public interface DoubleFlatMapFunction<T> extends Serializable {
   Iterator<Double> call(T t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java
index bf16f79..3c0291c 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  *  A function that returns Doubles, and can be used to construct DoubleRDDs.
  */
+@FunctionalInterface
 public interface DoubleFunction<T> extends Serializable {
   double call(T t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java
index 462ca3f..a6f69f7 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java
@@ -24,6 +24,7 @@ import java.io.Serializable;
  *
  * If the function returns true, the element is included in the returned Dataset.
  */
+@FunctionalInterface
 public interface FilterFunction<T> extends Serializable {
   boolean call(T value) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java
index 2d8ea6d..91d6129 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 /**
  * A function that returns zero or more output records from each input record.
  */
+@FunctionalInterface
 public interface FlatMapFunction<T, R> extends Serializable {
   Iterator<R> call(T t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java
index fc97b63..f9f2580 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 /**
  * A function that takes two inputs and returns zero or more output records.
  */
+@FunctionalInterface
 public interface FlatMapFunction2<T1, T2, R> extends Serializable {
   Iterator<R> call(T1 t1, T2 t2) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java
index bae574a..6423c5d 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 /**
  * A function that returns zero or more output records from each grouping key and its values.
  */
+@FunctionalInterface
 public interface FlatMapGroupsFunction<K, V, R> extends Serializable {
   Iterator<R> call(K key, Iterator<V> values) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java b/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java
index 07e54b2..2e6e908 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java
@@ -24,6 +24,7 @@ import java.io.Serializable;
  *
  * Spark will invoke the call function on each element in the input Dataset.
  */
+@FunctionalInterface
 public interface ForeachFunction<T> extends Serializable {
   void call(T t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java b/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java
index 4938a51..d8f55d0 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 /**
  * Base interface for a function used in Dataset's foreachPartition function.
  */
+@FunctionalInterface
 public interface ForeachPartitionFunction<T> extends Serializable {
   void call(Iterator<T> t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/Function.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function.java b/core/src/main/java/org/apache/spark/api/java/function/Function.java
index b9d9777..8b2bbd5 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/Function.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/Function.java
@@ -24,6 +24,7 @@ import java.io.Serializable;
  * DoubleFunction are handled separately, to allow PairRDDs and DoubleRDDs to be constructed
  * when mapping RDDs of other types.
  */
+@FunctionalInterface
 public interface Function<T1, R> extends Serializable {
   R call(T1 v1) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/Function0.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function0.java b/core/src/main/java/org/apache/spark/api/java/function/Function0.java
index c86928d..5c649d9 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/Function0.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/Function0.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * A zero-argument function that returns an R.
  */
+@FunctionalInterface
 public interface Function0<R> extends Serializable {
   R call() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/Function2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function2.java b/core/src/main/java/org/apache/spark/api/java/function/Function2.java
index a975ce3..a7d9647 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/Function2.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/Function2.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * A two-argument function that takes arguments of type T1 and T2 and returns an R.
  */
+@FunctionalInterface
 public interface Function2<T1, T2, R> extends Serializable {
   R call(T1 v1, T2 v2) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/Function3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function3.java b/core/src/main/java/org/apache/spark/api/java/function/Function3.java
index 6eecfb6..77acd21 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/Function3.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/Function3.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R.
  */
+@FunctionalInterface
 public interface Function3<T1, T2, T3, R> extends Serializable {
   R call(T1 v1, T2 v2, T3 v3) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/Function4.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function4.java b/core/src/main/java/org/apache/spark/api/java/function/Function4.java
index 9c35a22..d530ba4 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/Function4.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/Function4.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * A four-argument function that takes arguments of type T1, T2, T3 and T4 and returns an R.
  */
+@FunctionalInterface
 public interface Function4<T1, T2, T3, T4, R> extends Serializable {
   R call(T1 v1, T2 v2, T3 v3, T4 v4) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java
index 3ae6ef4..5efff94 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * Base interface for a map function used in Dataset's map function.
  */
+@FunctionalInterface
 public interface MapFunction<T, U> extends Serializable {
   U call(T value) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java b/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java
index faa59ea..2c3d43a 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 /**
  * Base interface for a map function used in GroupedDataset's mapGroup function.
  */
+@FunctionalInterface
 public interface MapGroupsFunction<K, V, R> extends Serializable {
   R call(K key, Iterator<V> values) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java b/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java
index cf9945a..68e8557 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 /**
  * Base interface for function used in Dataset's mapPartitions.
  */
+@FunctionalInterface
 public interface MapPartitionsFunction<T, U> extends Serializable {
   Iterator<U> call(Iterator<T> input) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java
index 51eed2e..97bd2b3 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java
@@ -26,6 +26,7 @@ import scala.Tuple2;
  * A function that returns zero or more key-value pair records from each input record. The
  * key-value pairs are represented as scala.Tuple2 objects.
  */
+@FunctionalInterface
 public interface PairFlatMapFunction<T, K, V> extends Serializable {
   Iterator<Tuple2<K, V>> call(T t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java
index 2fdfa71..34a7e44 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java
@@ -25,6 +25,7 @@ import scala.Tuple2;
  * A function that returns key-value pairs (Tuple2&lt;K, V&gt;), and can be used to
  * construct PairRDDs.
  */
+@FunctionalInterface
 public interface PairFunction<T, K, V> extends Serializable {
   Tuple2<K, V> call(T t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java b/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java
index ee092d0..d9029d8 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * Base interface for function used in Dataset's reduce.
  */
+@FunctionalInterface
 public interface ReduceFunction<T> extends Serializable {
   T call(T v1, T v2) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java
index f30d42e..aff2bc6 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * A function with no return value.
  */
+@FunctionalInterface
 public interface VoidFunction<T> extends Serializable {
   void call(T t) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java
index da9ae1c..ddb6162 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java
+++ b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java
@@ -22,6 +22,7 @@ import java.io.Serializable;
 /**
  * A two-argument function that takes arguments of type T1 and T2 with no return value.
  */
+@FunctionalInterface
 public interface VoidFunction2<T1, T2> extends Serializable {
   void call(T1 v1, T2 v2) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index dcae4a3..189d607 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -162,14 +162,7 @@ public final class UnsafeExternalSorter extends MemoryConsumer {
     // Register a cleanup task with TaskContext to ensure that memory is guaranteed to be freed at
     // the end of the task. This is necessary to avoid memory leaks in when the downstream operator
     // does not fully consume the sorter's output (e.g. sort followed by limit).
-    taskContext.addTaskCompletionListener(
-      new TaskCompletionListener() {
-        @Override
-        public void onTaskCompletion(TaskContext context) {
-          cleanupResources();
-        }
-      }
-    );
+    taskContext.addTaskCompletionListener(context -> { cleanupResources(); });
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
index 01aed95..cf4dfde 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
@@ -27,22 +27,18 @@ final class UnsafeSorterSpillMerger {
   private final PriorityQueue<UnsafeSorterIterator> priorityQueue;
 
   UnsafeSorterSpillMerger(
-      final RecordComparator recordComparator,
-      final PrefixComparator prefixComparator,
-      final int numSpills) {
-    final Comparator<UnsafeSorterIterator> comparator = new Comparator<UnsafeSorterIterator>() {
-
-      @Override
-      public int compare(UnsafeSorterIterator left, UnsafeSorterIterator right) {
-        final int prefixComparisonResult =
-          prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix());
-        if (prefixComparisonResult == 0) {
-          return recordComparator.compare(
-            left.getBaseObject(), left.getBaseOffset(),
-            right.getBaseObject(), right.getBaseOffset());
-        } else {
-          return prefixComparisonResult;
-        }
+      RecordComparator recordComparator,
+      PrefixComparator prefixComparator,
+      int numSpills) {
+    Comparator<UnsafeSorterIterator> comparator = (left, right) -> {
+      int prefixComparisonResult =
+        prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix());
+      if (prefixComparisonResult == 0) {
+        return recordComparator.compare(
+          left.getBaseObject(), left.getBaseOffset(),
+          right.getBaseObject(), right.getBaseOffset());
+      } else {
+        return prefixComparisonResult;
       }
     };
     priorityQueue = new PriorityQueue<>(numSpills, comparator);

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index cbab7b8..7e56406 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -350,9 +350,6 @@ class SparkContext(config: SparkConf) extends Logging {
 
   private def warnDeprecatedVersions(): Unit = {
     val javaVersion = System.getProperty("java.version").split("[+.\\-]+", 3)
-    if (javaVersion.length >= 2 && javaVersion(1).toInt == 7) {
-      logWarning("Support for Java 7 is deprecated as of Spark 2.0.0")
-    }
     if (scala.util.Properties.releaseVersion.exists(_.startsWith("2.10"))) {
       logWarning("Support for Scala 2.10 is deprecated as of Spark 2.1.0")
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
index 31b9c5e..3fd812e 100644
--- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
@@ -39,7 +39,6 @@ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, comm
     val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator))
     cmd.add(s"-Xmx${memoryMb}M")
     command.javaOpts.foreach(cmd.add)
-    CommandBuilderUtils.addPermGenSizeOpt(cmd)
     addOptionString(cmd, getenv("SPARK_JAVA_OPTS"))
     cmd
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index fe6fe6a..1e6e9a2 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1882,20 +1882,17 @@ private[spark] object Utils extends Logging {
   def terminateProcess(process: Process, timeoutMs: Long): Option[Int] = {
     // Politely destroy first
     process.destroy()
-
-    if (waitForProcess(process, timeoutMs)) {
+    if (process.waitFor(timeoutMs, TimeUnit.MILLISECONDS)) {
       // Successful exit
       Option(process.exitValue())
     } else {
-      // Java 8 added a new API which will more forcibly kill the process. Use that if available.
       try {
-        classOf[Process].getMethod("destroyForcibly").invoke(process)
+        process.destroyForcibly()
       } catch {
-        case _: NoSuchMethodException => return None // Not available; give up
         case NonFatal(e) => logWarning("Exception when attempting to kill process", e)
       }
       // Wait, again, although this really should return almost immediately
-      if (waitForProcess(process, timeoutMs)) {
+      if (process.waitFor(timeoutMs, TimeUnit.MILLISECONDS)) {
         Option(process.exitValue())
       } else {
         logWarning("Timed out waiting to forcibly kill process")
@@ -1905,44 +1902,11 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Wait for a process to terminate for at most the specified duration.
-   *
-   * @return whether the process actually terminated before the given timeout.
-   */
-  def waitForProcess(process: Process, timeoutMs: Long): Boolean = {
-    try {
-      // Use Java 8 method if available
-      classOf[Process].getMethod("waitFor", java.lang.Long.TYPE, classOf[TimeUnit])
-        .invoke(process, timeoutMs.asInstanceOf[java.lang.Long], TimeUnit.MILLISECONDS)
-        .asInstanceOf[Boolean]
-    } catch {
-      case _: NoSuchMethodException =>
-        // Otherwise implement it manually
-        var terminated = false
-        val startTime = System.currentTimeMillis
-        while (!terminated) {
-          try {
-            process.exitValue()
-            terminated = true
-          } catch {
-            case e: IllegalThreadStateException =>
-              // Process not terminated yet
-              if (System.currentTimeMillis - startTime > timeoutMs) {
-                return false
-              }
-              Thread.sleep(100)
-          }
-        }
-        true
-    }
-  }
-
-  /**
    * Return the stderr of a process after waiting for the process to terminate.
    * If the process does not terminate within the specified timeout, return None.
    */
   def getStderr(process: Process, timeoutMs: Long): Option[String] = {
-    val terminated = Utils.waitForProcess(process, timeoutMs)
+    val terminated = process.waitFor(timeoutMs, TimeUnit.MILLISECONDS)
     if (terminated) {
       Some(Source.fromInputStream(process.getErrorStream).getLines().mkString("\n"))
     } else {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[6/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/test/java/test/org/apache/spark/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java
new file mode 100644
index 0000000..80aab10
--- /dev/null
+++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java
@@ -0,0 +1,1842 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark;
+
+import java.io.*;
+import java.nio.channels.FileChannel;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import org.apache.spark.Accumulator;
+import org.apache.spark.AccumulatorParam;
+import org.apache.spark.Partitioner;
+import org.apache.spark.SparkConf;
+import org.apache.spark.TaskContext;
+import org.apache.spark.TaskContext$;
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.Tuple4;
+import scala.collection.JavaConverters;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.base.Throwables;
+import com.google.common.io.Files;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.After;
+import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.spark.api.java.JavaDoubleRDD;
+import org.apache.spark.api.java.JavaFutureAction;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.Optional;
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.input.PortableDataStream;
+import org.apache.spark.partial.BoundedDouble;
+import org.apache.spark.partial.PartialResult;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.serializer.KryoSerializer;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.util.LongAccumulator;
+import org.apache.spark.util.StatCounter;
+
+// The test suite itself is Serializable so that anonymous Function implementations can be
+// serialized, as an alternative to converting these anonymous classes to static inner classes;
+// see http://stackoverflow.com/questions/758570/.
+public class JavaAPISuite implements Serializable {
+  private transient JavaSparkContext sc;
+  private transient File tempDir;
+
+  @Before
+  public void setUp() {
+    sc = new JavaSparkContext("local", "JavaAPISuite");
+    tempDir = Files.createTempDir();
+    tempDir.deleteOnExit();
+  }
+
+  @After
+  public void tearDown() {
+    sc.stop();
+    sc = null;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void sparkContextUnion() {
+    // Union of non-specialized JavaRDDs
+    List<String> strings = Arrays.asList("Hello", "World");
+    JavaRDD<String> s1 = sc.parallelize(strings);
+    JavaRDD<String> s2 = sc.parallelize(strings);
+    // Varargs
+    JavaRDD<String> sUnion = sc.union(s1, s2);
+    assertEquals(4, sUnion.count());
+    // List
+    List<JavaRDD<String>> list = new ArrayList<>();
+    list.add(s2);
+    sUnion = sc.union(s1, list);
+    assertEquals(4, sUnion.count());
+
+    // Union of JavaDoubleRDDs
+    List<Double> doubles = Arrays.asList(1.0, 2.0);
+    JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles);
+    JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles);
+    JavaDoubleRDD dUnion = sc.union(d1, d2);
+    assertEquals(4, dUnion.count());
+
+    // Union of JavaPairRDDs
+    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
+    pairs.add(new Tuple2<>(1, 2));
+    pairs.add(new Tuple2<>(3, 4));
+    JavaPairRDD<Integer, Integer> p1 = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> p2 = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> pUnion = sc.union(p1, p2);
+    assertEquals(4, pUnion.count());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void intersection() {
+    List<Integer> ints1 = Arrays.asList(1, 10, 2, 3, 4, 5);
+    List<Integer> ints2 = Arrays.asList(1, 6, 2, 3, 7, 8);
+    JavaRDD<Integer> s1 = sc.parallelize(ints1);
+    JavaRDD<Integer> s2 = sc.parallelize(ints2);
+
+    JavaRDD<Integer> intersections = s1.intersection(s2);
+    assertEquals(3, intersections.count());
+
+    JavaRDD<Integer> empty = sc.emptyRDD();
+    JavaRDD<Integer> emptyIntersection = empty.intersection(s2);
+    assertEquals(0, emptyIntersection.count());
+
+    List<Double> doubles = Arrays.asList(1.0, 2.0);
+    JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles);
+    JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles);
+    JavaDoubleRDD dIntersection = d1.intersection(d2);
+    assertEquals(2, dIntersection.count());
+
+    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
+    pairs.add(new Tuple2<>(1, 2));
+    pairs.add(new Tuple2<>(3, 4));
+    JavaPairRDD<Integer, Integer> p1 = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> p2 = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> pIntersection = p1.intersection(p2);
+    assertEquals(2, pIntersection.count());
+  }
+
+  @Test
+  public void sample() {
+    List<Integer> ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+    JavaRDD<Integer> rdd = sc.parallelize(ints);
+    // the seeds here are "magic" to make this work out nicely
+    JavaRDD<Integer> sample20 = rdd.sample(true, 0.2, 8);
+    assertEquals(2, sample20.count());
+    JavaRDD<Integer> sample20WithoutReplacement = rdd.sample(false, 0.2, 2);
+    assertEquals(2, sample20WithoutReplacement.count());
+  }
+
+  @Test
+  public void randomSplit() {
+    List<Integer> ints = new ArrayList<>(1000);
+    for (int i = 0; i < 1000; i++) {
+      ints.add(i);
+    }
+    JavaRDD<Integer> rdd = sc.parallelize(ints);
+    JavaRDD<Integer>[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 31);
+    // the splits aren't perfect -- not enough data for them to be -- just check they're about right
+    assertEquals(3, splits.length);
+    long s0 = splits[0].count();
+    long s1 = splits[1].count();
+    long s2 = splits[2].count();
+    assertTrue(s0 + " not within expected range", s0 > 150 && s0 < 250);
+    assertTrue(s1 + " not within expected range", s1 > 250 && s0 < 350);
+    assertTrue(s2 + " not within expected range", s2 > 430 && s2 < 570);
+  }
+
+  @Test
+  public void sortByKey() {
+    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
+    pairs.add(new Tuple2<>(0, 4));
+    pairs.add(new Tuple2<>(3, 2));
+    pairs.add(new Tuple2<>(-1, 1));
+
+    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
+
+    // Default comparator
+    JavaPairRDD<Integer, Integer> sortedRDD = rdd.sortByKey();
+    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
+    List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect();
+    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1));
+    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2));
+
+    // Custom comparator
+    sortedRDD = rdd.sortByKey(Collections.<Integer>reverseOrder(), false);
+    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
+    sortedPairs = sortedRDD.collect();
+    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1));
+    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void repartitionAndSortWithinPartitions() {
+    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
+    pairs.add(new Tuple2<>(0, 5));
+    pairs.add(new Tuple2<>(3, 8));
+    pairs.add(new Tuple2<>(2, 6));
+    pairs.add(new Tuple2<>(0, 8));
+    pairs.add(new Tuple2<>(3, 8));
+    pairs.add(new Tuple2<>(1, 3));
+
+    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
+
+    Partitioner partitioner = new Partitioner() {
+      @Override
+      public int numPartitions() {
+        return 2;
+      }
+      @Override
+      public int getPartition(Object key) {
+        return (Integer) key % 2;
+      }
+    };
+
+    JavaPairRDD<Integer, Integer> repartitioned =
+        rdd.repartitionAndSortWithinPartitions(partitioner);
+    assertTrue(repartitioned.partitioner().isPresent());
+    assertEquals(repartitioned.partitioner().get(), partitioner);
+    List<List<Tuple2<Integer, Integer>>> partitions = repartitioned.glom().collect();
+    assertEquals(partitions.get(0),
+        Arrays.asList(new Tuple2<>(0, 5), new Tuple2<>(0, 8), new Tuple2<>(2, 6)));
+    assertEquals(partitions.get(1),
+        Arrays.asList(new Tuple2<>(1, 3), new Tuple2<>(3, 8), new Tuple2<>(3, 8)));
+  }
+
+  @Test
+  public void emptyRDD() {
+    JavaRDD<String> rdd = sc.emptyRDD();
+    assertEquals("Empty RDD shouldn't have any values", 0, rdd.count());
+  }
+
+  @Test
+  public void sortBy() {
+    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
+    pairs.add(new Tuple2<>(0, 4));
+    pairs.add(new Tuple2<>(3, 2));
+    pairs.add(new Tuple2<>(-1, 1));
+
+    JavaRDD<Tuple2<Integer, Integer>> rdd = sc.parallelize(pairs);
+
+    // compare on first value
+    JavaRDD<Tuple2<Integer, Integer>> sortedRDD =
+        rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() {
+      @Override
+      public Integer call(Tuple2<Integer, Integer> t) {
+        return t._1();
+      }
+    }, true, 2);
+
+    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
+    List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect();
+    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1));
+    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2));
+
+    // compare on second value
+    sortedRDD = rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() {
+      @Override
+      public Integer call(Tuple2<Integer, Integer> t) {
+        return t._2();
+      }
+    }, true, 2);
+    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
+    sortedPairs = sortedRDD.collect();
+    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1));
+    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(2));
+  }
+
+  @Test
+  public void foreach() {
+    final LongAccumulator accum = sc.sc().longAccumulator();
+    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
+    rdd.foreach(new VoidFunction<String>() {
+      @Override
+      public void call(String s) {
+        accum.add(1);
+      }
+    });
+    assertEquals(2, accum.value().intValue());
+  }
+
+  @Test
+  public void foreachPartition() {
+    final LongAccumulator accum = sc.sc().longAccumulator();
+    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
+    rdd.foreachPartition(new VoidFunction<Iterator<String>>() {
+      @Override
+      public void call(Iterator<String> iter) {
+        while (iter.hasNext()) {
+          iter.next();
+          accum.add(1);
+        }
+      }
+    });
+    assertEquals(2, accum.value().intValue());
+  }
+
+  @Test
+  public void toLocalIterator() {
+    List<Integer> correct = Arrays.asList(1, 2, 3, 4);
+    JavaRDD<Integer> rdd = sc.parallelize(correct);
+    List<Integer> result = Lists.newArrayList(rdd.toLocalIterator());
+    assertEquals(correct, result);
+  }
+
+  @Test
+  public void zipWithUniqueId() {
+    List<Integer> dataArray = Arrays.asList(1, 2, 3, 4);
+    JavaPairRDD<Integer, Long> zip = sc.parallelize(dataArray).zipWithUniqueId();
+    JavaRDD<Long> indexes = zip.values();
+    assertEquals(4, new HashSet<>(indexes.collect()).size());
+  }
+
+  @Test
+  public void zipWithIndex() {
+    List<Integer> dataArray = Arrays.asList(1, 2, 3, 4);
+    JavaPairRDD<Integer, Long> zip = sc.parallelize(dataArray).zipWithIndex();
+    JavaRDD<Long> indexes = zip.values();
+    List<Long> correctIndexes = Arrays.asList(0L, 1L, 2L, 3L);
+    assertEquals(correctIndexes, indexes.collect());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void lookup() {
+    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Apples", "Fruit"),
+      new Tuple2<>("Oranges", "Fruit"),
+      new Tuple2<>("Oranges", "Citrus")
+    ));
+    assertEquals(2, categories.lookup("Oranges").size());
+    assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0)));
+  }
+
+  @Test
+  public void groupBy() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    Function<Integer, Boolean> isOdd = new Function<Integer, Boolean>() {
+      @Override
+      public Boolean call(Integer x) {
+        return x % 2 == 0;
+      }
+    };
+    JavaPairRDD<Boolean, Iterable<Integer>> oddsAndEvens = rdd.groupBy(isOdd);
+    assertEquals(2, oddsAndEvens.count());
+    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
+    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
+
+    oddsAndEvens = rdd.groupBy(isOdd, 1);
+    assertEquals(2, oddsAndEvens.count());
+    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
+    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
+  }
+
+  @Test
+  public void groupByOnPairRDD() {
+    // Regression test for SPARK-4459
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    Function<Tuple2<Integer, Integer>, Boolean> areOdd =
+      new Function<Tuple2<Integer, Integer>, Boolean>() {
+        @Override
+        public Boolean call(Tuple2<Integer, Integer> x) {
+          return (x._1() % 2 == 0) && (x._2() % 2 == 0);
+        }
+      };
+    JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd);
+    JavaPairRDD<Boolean, Iterable<Tuple2<Integer, Integer>>> oddsAndEvens = pairRDD.groupBy(areOdd);
+    assertEquals(2, oddsAndEvens.count());
+    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
+    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
+
+    oddsAndEvens = pairRDD.groupBy(areOdd, 1);
+    assertEquals(2, oddsAndEvens.count());
+    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
+    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void keyByOnPairRDD() {
+    // Regression test for SPARK-4459
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    Function<Tuple2<Integer, Integer>, String> sumToString =
+      new Function<Tuple2<Integer, Integer>, String>() {
+        @Override
+        public String call(Tuple2<Integer, Integer> x) {
+          return String.valueOf(x._1() + x._2());
+        }
+      };
+    JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd);
+    JavaPairRDD<String, Tuple2<Integer, Integer>> keyed = pairRDD.keyBy(sumToString);
+    assertEquals(7, keyed.count());
+    assertEquals(1, (long) keyed.lookup("2").get(0)._1());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void cogroup() {
+    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Apples", "Fruit"),
+      new Tuple2<>("Oranges", "Fruit"),
+      new Tuple2<>("Oranges", "Citrus")
+      ));
+    JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Oranges", 2),
+      new Tuple2<>("Apples", 3)
+    ));
+    JavaPairRDD<String, Tuple2<Iterable<String>, Iterable<Integer>>> cogrouped =
+        categories.cogroup(prices);
+    assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1()));
+    assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2()));
+
+    cogrouped.collect();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void cogroup3() {
+    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Apples", "Fruit"),
+      new Tuple2<>("Oranges", "Fruit"),
+      new Tuple2<>("Oranges", "Citrus")
+      ));
+    JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Oranges", 2),
+      new Tuple2<>("Apples", 3)
+    ));
+    JavaPairRDD<String, Integer> quantities = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Oranges", 21),
+      new Tuple2<>("Apples", 42)
+    ));
+
+    JavaPairRDD<String, Tuple3<Iterable<String>, Iterable<Integer>, Iterable<Integer>>> cogrouped =
+        categories.cogroup(prices, quantities);
+    assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1()));
+    assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2()));
+    assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3()));
+
+
+    cogrouped.collect();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void cogroup4() {
+    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Apples", "Fruit"),
+      new Tuple2<>("Oranges", "Fruit"),
+      new Tuple2<>("Oranges", "Citrus")
+      ));
+    JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Oranges", 2),
+      new Tuple2<>("Apples", 3)
+    ));
+    JavaPairRDD<String, Integer> quantities = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Oranges", 21),
+      new Tuple2<>("Apples", 42)
+    ));
+    JavaPairRDD<String, String> countries = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>("Oranges", "BR"),
+      new Tuple2<>("Apples", "US")
+    ));
+
+    JavaPairRDD<String, Tuple4<Iterable<String>, Iterable<Integer>, Iterable<Integer>,
+        Iterable<String>>> cogrouped = categories.cogroup(prices, quantities, countries);
+    assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1()));
+    assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2()));
+    assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3()));
+    assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4()));
+
+    cogrouped.collect();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void leftOuterJoin() {
+    JavaPairRDD<Integer, Integer> rdd1 = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>(1, 1),
+      new Tuple2<>(1, 2),
+      new Tuple2<>(2, 1),
+      new Tuple2<>(3, 1)
+      ));
+    JavaPairRDD<Integer, Character> rdd2 = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>(1, 'x'),
+      new Tuple2<>(2, 'y'),
+      new Tuple2<>(2, 'z'),
+      new Tuple2<>(4, 'w')
+    ));
+    List<Tuple2<Integer,Tuple2<Integer,Optional<Character>>>> joined =
+      rdd1.leftOuterJoin(rdd2).collect();
+    assertEquals(5, joined.size());
+    Tuple2<Integer,Tuple2<Integer,Optional<Character>>> firstUnmatched =
+      rdd1.leftOuterJoin(rdd2).filter(
+        new Function<Tuple2<Integer, Tuple2<Integer, Optional<Character>>>, Boolean>() {
+          @Override
+          public Boolean call(Tuple2<Integer, Tuple2<Integer, Optional<Character>>> tup) {
+            return !tup._2()._2().isPresent();
+          }
+      }).first();
+    assertEquals(3, firstUnmatched._1().intValue());
+  }
+
+  @Test
+  public void foldReduce() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() {
+      @Override
+      public Integer call(Integer a, Integer b) {
+        return a + b;
+      }
+    };
+
+    int sum = rdd.fold(0, add);
+    assertEquals(33, sum);
+
+    sum = rdd.reduce(add);
+    assertEquals(33, sum);
+  }
+
+  @Test
+  public void treeReduce() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10);
+    Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() {
+      @Override
+      public Integer call(Integer a, Integer b) {
+        return a + b;
+      }
+    };
+    for (int depth = 1; depth <= 10; depth++) {
+      int sum = rdd.treeReduce(add, depth);
+      assertEquals(-5, sum);
+    }
+  }
+
+  @Test
+  public void treeAggregate() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10);
+    Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() {
+      @Override
+      public Integer call(Integer a, Integer b) {
+        return a + b;
+      }
+    };
+    for (int depth = 1; depth <= 10; depth++) {
+      int sum = rdd.treeAggregate(0, add, add, depth);
+      assertEquals(-5, sum);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void aggregateByKey() {
+    JavaPairRDD<Integer, Integer> pairs = sc.parallelizePairs(
+      Arrays.asList(
+        new Tuple2<>(1, 1),
+        new Tuple2<>(1, 1),
+        new Tuple2<>(3, 2),
+        new Tuple2<>(5, 1),
+        new Tuple2<>(5, 3)), 2);
+
+    Map<Integer, Set<Integer>> sets = pairs.aggregateByKey(new HashSet<Integer>(),
+      new Function2<Set<Integer>, Integer, Set<Integer>>() {
+        @Override
+        public Set<Integer> call(Set<Integer> a, Integer b) {
+          a.add(b);
+          return a;
+        }
+      },
+      new Function2<Set<Integer>, Set<Integer>, Set<Integer>>() {
+        @Override
+        public Set<Integer> call(Set<Integer> a, Set<Integer> b) {
+          a.addAll(b);
+          return a;
+        }
+      }).collectAsMap();
+    assertEquals(3, sets.size());
+    assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1));
+    assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3));
+    assertEquals(new HashSet<>(Arrays.asList(1, 3)), sets.get(5));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void foldByKey() {
+    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
+      new Tuple2<>(2, 1),
+      new Tuple2<>(2, 1),
+      new Tuple2<>(1, 1),
+      new Tuple2<>(3, 2),
+      new Tuple2<>(3, 1)
+    );
+    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0,
+      new Function2<Integer, Integer, Integer>() {
+        @Override
+        public Integer call(Integer a, Integer b) {
+          return a + b;
+        }
+    });
+    assertEquals(1, sums.lookup(1).get(0).intValue());
+    assertEquals(2, sums.lookup(2).get(0).intValue());
+    assertEquals(3, sums.lookup(3).get(0).intValue());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void reduceByKey() {
+    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
+      new Tuple2<>(2, 1),
+      new Tuple2<>(2, 1),
+      new Tuple2<>(1, 1),
+      new Tuple2<>(3, 2),
+      new Tuple2<>(3, 1)
+    );
+    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> counts = rdd.reduceByKey(
+      new Function2<Integer, Integer, Integer>() {
+        @Override
+        public Integer call(Integer a, Integer b) {
+         return a + b;
+        }
+    });
+    assertEquals(1, counts.lookup(1).get(0).intValue());
+    assertEquals(2, counts.lookup(2).get(0).intValue());
+    assertEquals(3, counts.lookup(3).get(0).intValue());
+
+    Map<Integer, Integer> localCounts = counts.collectAsMap();
+    assertEquals(1, localCounts.get(1).intValue());
+    assertEquals(2, localCounts.get(2).intValue());
+    assertEquals(3, localCounts.get(3).intValue());
+
+    localCounts = rdd.reduceByKeyLocally(new Function2<Integer, Integer, Integer>() {
+      @Override
+      public Integer call(Integer a, Integer b) {
+        return a + b;
+      }
+    });
+    assertEquals(1, localCounts.get(1).intValue());
+    assertEquals(2, localCounts.get(2).intValue());
+    assertEquals(3, localCounts.get(3).intValue());
+  }
+
+  @Test
+  public void approximateResults() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    Map<Integer, Long> countsByValue = rdd.countByValue();
+    assertEquals(2, countsByValue.get(1).longValue());
+    assertEquals(1, countsByValue.get(13).longValue());
+
+    PartialResult<Map<Integer, BoundedDouble>> approx = rdd.countByValueApprox(1);
+    Map<Integer, BoundedDouble> finalValue = approx.getFinalValue();
+    assertEquals(2.0, finalValue.get(1).mean(), 0.01);
+    assertEquals(1.0, finalValue.get(13).mean(), 0.01);
+  }
+
+  @Test
+  public void take() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    assertEquals(1, rdd.first().intValue());
+    rdd.take(2);
+    rdd.takeSample(false, 2, 42);
+  }
+
+  @Test
+  public void isEmpty() {
+    assertTrue(sc.emptyRDD().isEmpty());
+    assertTrue(sc.parallelize(new ArrayList<Integer>()).isEmpty());
+    assertFalse(sc.parallelize(Arrays.asList(1)).isEmpty());
+    assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter(
+        new Function<Integer,Boolean>() {
+          @Override
+          public Boolean call(Integer i) {
+            return i < 0;
+          }
+        }).isEmpty());
+    assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter(
+        new Function<Integer, Boolean>() {
+          @Override
+          public Boolean call(Integer i) {
+            return i > 1;
+          }
+        }).isEmpty());
+  }
+
+  @Test
+  public void cartesian() {
+    JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
+    JavaRDD<String> stringRDD = sc.parallelize(Arrays.asList("Hello", "World"));
+    JavaPairRDD<String, Double> cartesian = stringRDD.cartesian(doubleRDD);
+    assertEquals(new Tuple2<>("Hello", 1.0), cartesian.first());
+  }
+
+  @Test
+  public void javaDoubleRDD() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
+    JavaDoubleRDD distinct = rdd.distinct();
+    assertEquals(5, distinct.count());
+    JavaDoubleRDD filter = rdd.filter(new Function<Double, Boolean>() {
+      @Override
+      public Boolean call(Double x) {
+        return x > 2.0;
+      }
+    });
+    assertEquals(3, filter.count());
+    JavaDoubleRDD union = rdd.union(rdd);
+    assertEquals(12, union.count());
+    union = union.cache();
+    assertEquals(12, union.count());
+
+    assertEquals(20, rdd.sum(), 0.01);
+    StatCounter stats = rdd.stats();
+    assertEquals(20, stats.sum(), 0.01);
+    assertEquals(20/6.0, rdd.mean(), 0.01);
+    assertEquals(20/6.0, rdd.mean(), 0.01);
+    assertEquals(6.22222, rdd.variance(), 0.01);
+    assertEquals(rdd.variance(), rdd.popVariance(), 1e-14);
+    assertEquals(7.46667, rdd.sampleVariance(), 0.01);
+    assertEquals(2.49444, rdd.stdev(), 0.01);
+    assertEquals(rdd.stdev(), rdd.popStdev(), 1e-14);
+    assertEquals(2.73252, rdd.sampleStdev(), 0.01);
+
+    rdd.first();
+    rdd.take(5);
+  }
+
+  @Test
+  public void javaDoubleRDDHistoGram() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+    // Test using generated buckets
+    Tuple2<double[], long[]> results = rdd.histogram(2);
+    double[] expected_buckets = {1.0, 2.5, 4.0};
+    long[] expected_counts = {2, 2};
+    assertArrayEquals(expected_buckets, results._1(), 0.1);
+    assertArrayEquals(expected_counts, results._2());
+    // Test with provided buckets
+    long[] histogram = rdd.histogram(expected_buckets);
+    assertArrayEquals(expected_counts, histogram);
+    // SPARK-5744
+    assertArrayEquals(
+        new long[] {0},
+        sc.parallelizeDoubles(new ArrayList<Double>(0), 1).histogram(new double[]{0.0, 1.0}));
+  }
+
+  private static class DoubleComparator implements Comparator<Double>, Serializable {
+    @Override
+    public int compare(Double o1, Double o2) {
+      return o1.compareTo(o2);
+    }
+  }
+
+  @Test
+  public void max() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+    double max = rdd.max(new DoubleComparator());
+    assertEquals(4.0, max, 0.001);
+  }
+
+  @Test
+  public void min() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+    double max = rdd.min(new DoubleComparator());
+    assertEquals(1.0, max, 0.001);
+  }
+
+  @Test
+  public void naturalMax() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+    double max = rdd.max();
+    assertEquals(4.0, max, 0.0);
+  }
+
+  @Test
+  public void naturalMin() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+    double max = rdd.min();
+    assertEquals(1.0, max, 0.0);
+  }
+
+  @Test
+  public void takeOrdered() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+    assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator()));
+    assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2));
+  }
+
+  @Test
+  public void top() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    List<Integer> top2 = rdd.top(2);
+    assertEquals(Arrays.asList(4, 3), top2);
+  }
+
+  private static class AddInts implements Function2<Integer, Integer, Integer> {
+    @Override
+    public Integer call(Integer a, Integer b) {
+      return a + b;
+    }
+  }
+
+  @Test
+  public void reduce() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    int sum = rdd.reduce(new AddInts());
+    assertEquals(10, sum);
+  }
+
+  @Test
+  public void reduceOnJavaDoubleRDD() {
+    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+    double sum = rdd.reduce(new Function2<Double, Double, Double>() {
+      @Override
+      public Double call(Double v1, Double v2) {
+        return v1 + v2;
+      }
+    });
+    assertEquals(10.0, sum, 0.001);
+  }
+
+  @Test
+  public void fold() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    int sum = rdd.fold(0, new AddInts());
+    assertEquals(10, sum);
+  }
+
+  @Test
+  public void aggregate() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    int sum = rdd.aggregate(0, new AddInts(), new AddInts());
+    assertEquals(10, sum);
+  }
+
+  @Test
+  public void map() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction<Integer>() {
+      @Override
+      public double call(Integer x) {
+        return x.doubleValue();
+      }
+    }).cache();
+    doubles.collect();
+    JavaPairRDD<Integer, Integer> pairs = rdd.mapToPair(
+        new PairFunction<Integer, Integer, Integer>() {
+          @Override
+          public Tuple2<Integer, Integer> call(Integer x) {
+            return new Tuple2<>(x, x);
+          }
+        }).cache();
+    pairs.collect();
+    JavaRDD<String> strings = rdd.map(new Function<Integer, String>() {
+      @Override
+      public String call(Integer x) {
+        return x.toString();
+      }
+    }).cache();
+    strings.collect();
+  }
+
+  @Test
+  public void flatMap() {
+    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello World!",
+      "The quick brown fox jumps over the lazy dog."));
+    JavaRDD<String> words = rdd.flatMap(new FlatMapFunction<String, String>() {
+      @Override
+      public Iterator<String> call(String x) {
+        return Arrays.asList(x.split(" ")).iterator();
+      }
+    });
+    assertEquals("Hello", words.first());
+    assertEquals(11, words.count());
+
+    JavaPairRDD<String, String> pairsRDD = rdd.flatMapToPair(
+      new PairFlatMapFunction<String, String, String>() {
+        @Override
+        public Iterator<Tuple2<String, String>> call(String s) {
+          List<Tuple2<String, String>> pairs = new LinkedList<>();
+          for (String word : s.split(" ")) {
+            pairs.add(new Tuple2<>(word, word));
+          }
+          return pairs.iterator();
+        }
+      }
+    );
+    assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first());
+    assertEquals(11, pairsRDD.count());
+
+    JavaDoubleRDD doubles = rdd.flatMapToDouble(new DoubleFlatMapFunction<String>() {
+      @Override
+      public Iterator<Double> call(String s) {
+        List<Double> lengths = new LinkedList<>();
+        for (String word : s.split(" ")) {
+          lengths.add((double) word.length());
+        }
+        return lengths.iterator();
+      }
+    });
+    assertEquals(5.0, doubles.first(), 0.01);
+    assertEquals(11, pairsRDD.count());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void mapsFromPairsToPairs() {
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
+
+    // Regression test for SPARK-668:
+    JavaPairRDD<String, Integer> swapped = pairRDD.flatMapToPair(
+      new PairFlatMapFunction<Tuple2<Integer, String>, String, Integer>() {
+        @Override
+        public Iterator<Tuple2<String, Integer>> call(Tuple2<Integer, String> item) {
+          return Collections.singletonList(item.swap()).iterator();
+        }
+      });
+    swapped.collect();
+
+    // There was never a bug here, but it's worth testing:
+    pairRDD.mapToPair(new PairFunction<Tuple2<Integer, String>, String, Integer>() {
+      @Override
+      public Tuple2<String, Integer> call(Tuple2<Integer, String> item) {
+        return item.swap();
+      }
+    }).collect();
+  }
+
+  @Test
+  public void mapPartitions() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
+    JavaRDD<Integer> partitionSums = rdd.mapPartitions(
+      new FlatMapFunction<Iterator<Integer>, Integer>() {
+        @Override
+        public Iterator<Integer> call(Iterator<Integer> iter) {
+          int sum = 0;
+          while (iter.hasNext()) {
+            sum += iter.next();
+          }
+          return Collections.singletonList(sum).iterator();
+        }
+    });
+    assertEquals("[3, 7]", partitionSums.collect().toString());
+  }
+
+
+  @Test
+  public void mapPartitionsWithIndex() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
+    JavaRDD<Integer> partitionSums = rdd.mapPartitionsWithIndex(
+      new Function2<Integer, Iterator<Integer>, Iterator<Integer>>() {
+        @Override
+        public Iterator<Integer> call(Integer index, Iterator<Integer> iter) {
+          int sum = 0;
+          while (iter.hasNext()) {
+            sum += iter.next();
+          }
+          return Collections.singletonList(sum).iterator();
+        }
+    }, false);
+    assertEquals("[3, 7]", partitionSums.collect().toString());
+  }
+
+  @Test
+  public void getNumPartitions(){
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3);
+    JavaDoubleRDD rdd2 = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0), 2);
+    JavaPairRDD<String, Integer> rdd3 = sc.parallelizePairs(Arrays.asList(
+            new Tuple2<>("a", 1),
+            new Tuple2<>("aa", 2),
+            new Tuple2<>("aaa", 3)
+    ), 2);
+    assertEquals(3, rdd1.getNumPartitions());
+    assertEquals(2, rdd2.getNumPartitions());
+    assertEquals(2, rdd3.getNumPartitions());
+  }
+
+  @Test
+  public void repartition() {
+    // Shrinking number of partitions
+    JavaRDD<Integer> in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 2);
+    JavaRDD<Integer> repartitioned1 = in1.repartition(4);
+    List<List<Integer>> result1 = repartitioned1.glom().collect();
+    assertEquals(4, result1.size());
+    for (List<Integer> l : result1) {
+      assertFalse(l.isEmpty());
+    }
+
+    // Growing number of partitions
+    JavaRDD<Integer> in2 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 4);
+    JavaRDD<Integer> repartitioned2 = in2.repartition(2);
+    List<List<Integer>> result2 = repartitioned2.glom().collect();
+    assertEquals(2, result2.size());
+    for (List<Integer> l: result2) {
+      assertFalse(l.isEmpty());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void persist() {
+    JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
+    doubleRDD = doubleRDD.persist(StorageLevel.DISK_ONLY());
+    assertEquals(20, doubleRDD.sum(), 0.1);
+
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
+    pairRDD = pairRDD.persist(StorageLevel.DISK_ONLY());
+    assertEquals("a", pairRDD.first()._2());
+
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    rdd = rdd.persist(StorageLevel.DISK_ONLY());
+    assertEquals(1, rdd.first().intValue());
+  }
+
+  @Test
+  public void iterator() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
+    TaskContext context = TaskContext$.MODULE$.empty();
+    assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue());
+  }
+
+  @Test
+  public void glom() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
+    assertEquals("[1, 2]", rdd.glom().first().toString());
+  }
+
+  // File input / output tests are largely adapted from FileSuite:
+
+  @Test
+  public void textFiles() throws IOException {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    rdd.saveAsTextFile(outputDir);
+    // Read the plain text file and check it's OK
+    File outputFile = new File(outputDir, "part-00000");
+    String content = Files.toString(outputFile, StandardCharsets.UTF_8);
+    assertEquals("1\n2\n3\n4\n", content);
+    // Also try reading it in as a text file RDD
+    List<String> expected = Arrays.asList("1", "2", "3", "4");
+    JavaRDD<String> readRDD = sc.textFile(outputDir);
+    assertEquals(expected, readRDD.collect());
+  }
+
+  @Test
+  public void wholeTextFiles() throws Exception {
+    byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8);
+    byte[] content2 = "spark is also easy to use.\n".getBytes(StandardCharsets.UTF_8);
+
+    String tempDirName = tempDir.getAbsolutePath();
+    String path1 = new Path(tempDirName, "part-00000").toUri().getPath();
+    String path2 = new Path(tempDirName, "part-00001").toUri().getPath();
+
+    Files.write(content1, new File(path1));
+    Files.write(content2, new File(path2));
+
+    Map<String, String> container = new HashMap<>();
+    container.put(path1, new Text(content1).toString());
+    container.put(path2, new Text(content2).toString());
+
+    JavaPairRDD<String, String> readRDD = sc.wholeTextFiles(tempDirName, 3);
+    List<Tuple2<String, String>> result = readRDD.collect();
+
+    for (Tuple2<String, String> res : result) {
+      // Note that the paths from `wholeTextFiles` are in URI format on Windows,
+      // for example, file:/C:/a/b/c.
+      assertEquals(res._2(), container.get(new Path(res._1()).toUri().getPath()));
+    }
+  }
+
+  @Test
+  public void textFilesCompressed() throws IOException {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    rdd.saveAsTextFile(outputDir, DefaultCodec.class);
+
+    // Try reading it in as a text file RDD
+    List<String> expected = Arrays.asList("1", "2", "3", "4");
+    JavaRDD<String> readRDD = sc.textFile(outputDir);
+    assertEquals(expected, readRDD.collect());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void sequenceFile() {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
+
+    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
+      @Override
+      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
+        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
+      }
+    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
+
+    // Try reading the output back as an object file
+    JavaPairRDD<Integer, String> readRDD = sc.sequenceFile(outputDir, IntWritable.class,
+      Text.class).mapToPair(new PairFunction<Tuple2<IntWritable, Text>, Integer, String>() {
+      @Override
+      public Tuple2<Integer, String> call(Tuple2<IntWritable, Text> pair) {
+        return new Tuple2<>(pair._1().get(), pair._2().toString());
+      }
+    });
+    assertEquals(pairs, readRDD.collect());
+  }
+
+  @Test
+  public void binaryFiles() throws Exception {
+    // Reusing the wholeText files example
+    byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8);
+
+    String tempDirName = tempDir.getAbsolutePath();
+    File file1 = new File(tempDirName + "/part-00000");
+
+    FileOutputStream fos1 = new FileOutputStream(file1);
+
+    FileChannel channel1 = fos1.getChannel();
+    ByteBuffer bbuf = ByteBuffer.wrap(content1);
+    channel1.write(bbuf);
+    channel1.close();
+    JavaPairRDD<String, PortableDataStream> readRDD = sc.binaryFiles(tempDirName, 3);
+    List<Tuple2<String, PortableDataStream>> result = readRDD.collect();
+    for (Tuple2<String, PortableDataStream> res : result) {
+      assertArrayEquals(content1, res._2().toArray());
+    }
+  }
+
+  @Test
+  public void binaryFilesCaching() throws Exception {
+    // Reusing the wholeText files example
+    byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8);
+
+    String tempDirName = tempDir.getAbsolutePath();
+    File file1 = new File(tempDirName + "/part-00000");
+
+    FileOutputStream fos1 = new FileOutputStream(file1);
+
+    FileChannel channel1 = fos1.getChannel();
+    ByteBuffer bbuf = ByteBuffer.wrap(content1);
+    channel1.write(bbuf);
+    channel1.close();
+
+    JavaPairRDD<String, PortableDataStream> readRDD = sc.binaryFiles(tempDirName).cache();
+    readRDD.foreach(new VoidFunction<Tuple2<String,PortableDataStream>>() {
+      @Override
+      public void call(Tuple2<String, PortableDataStream> pair) {
+        pair._2().toArray(); // force the file to read
+      }
+    });
+
+    List<Tuple2<String, PortableDataStream>> result = readRDD.collect();
+    for (Tuple2<String, PortableDataStream> res : result) {
+      assertArrayEquals(content1, res._2().toArray());
+    }
+  }
+
+  @Test
+  public void binaryRecords() throws Exception {
+    // Reusing the wholeText files example
+    byte[] content1 = "spark isn't always easy to use.\n".getBytes(StandardCharsets.UTF_8);
+    int numOfCopies = 10;
+    String tempDirName = tempDir.getAbsolutePath();
+    File file1 = new File(tempDirName + "/part-00000");
+
+    FileOutputStream fos1 = new FileOutputStream(file1);
+
+    FileChannel channel1 = fos1.getChannel();
+
+    for (int i = 0; i < numOfCopies; i++) {
+      ByteBuffer bbuf = ByteBuffer.wrap(content1);
+      channel1.write(bbuf);
+    }
+    channel1.close();
+
+    JavaRDD<byte[]> readRDD = sc.binaryRecords(tempDirName, content1.length);
+    assertEquals(numOfCopies,readRDD.count());
+    List<byte[]> result = readRDD.collect();
+    for (byte[] res : result) {
+      assertArrayEquals(content1, res);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void writeWithNewAPIHadoopFile() {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
+
+    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
+      @Override
+      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
+        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
+      }
+    }).saveAsNewAPIHadoopFile(
+        outputDir, IntWritable.class, Text.class,
+        org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class);
+
+    JavaPairRDD<IntWritable, Text> output =
+        sc.sequenceFile(outputDir, IntWritable.class, Text.class);
+    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
+      @Override
+      public String call(Tuple2<IntWritable, Text> x) {
+        return x.toString();
+      }
+    }).collect().toString());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void readWithNewAPIHadoopFile() throws IOException {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
+
+    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
+      @Override
+      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
+        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
+      }
+    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
+
+    JavaPairRDD<IntWritable, Text> output = sc.newAPIHadoopFile(outputDir,
+        org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class,
+        IntWritable.class, Text.class, Job.getInstance().getConfiguration());
+    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
+      @Override
+      public String call(Tuple2<IntWritable, Text> x) {
+        return x.toString();
+      }
+    }).collect().toString());
+  }
+
+  @Test
+  public void objectFilesOfInts() {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    rdd.saveAsObjectFile(outputDir);
+    // Try reading the output back as an object file
+    List<Integer> expected = Arrays.asList(1, 2, 3, 4);
+    JavaRDD<Integer> readRDD = sc.objectFile(outputDir);
+    assertEquals(expected, readRDD.collect());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void objectFilesOfComplexTypes() {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
+    rdd.saveAsObjectFile(outputDir);
+    // Try reading the output back as an object file
+    JavaRDD<Tuple2<Integer, String>> readRDD = sc.objectFile(outputDir);
+    assertEquals(pairs, readRDD.collect());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void hadoopFile() {
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
+
+    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
+      @Override
+      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
+        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
+      }
+    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
+
+    JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir,
+        SequenceFileInputFormat.class, IntWritable.class, Text.class);
+    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
+      @Override
+      public String call(Tuple2<IntWritable, Text> x) {
+        return x.toString();
+      }
+    }).collect().toString());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void hadoopFileCompressed() {
+    String outputDir = new File(tempDir, "output_compressed").getAbsolutePath();
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
+
+    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
+      @Override
+      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
+        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
+      }
+    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class,
+        DefaultCodec.class);
+
+    JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir,
+        SequenceFileInputFormat.class, IntWritable.class, Text.class);
+
+    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
+      @Override
+      public String call(Tuple2<IntWritable, Text> x) {
+        return x.toString();
+      }
+    }).collect().toString());
+  }
+
+  @Test
+  public void zip() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction<Integer>() {
+      @Override
+      public double call(Integer x) {
+        return x.doubleValue();
+      }
+    });
+    JavaPairRDD<Integer, Double> zipped = rdd.zip(doubles);
+    zipped.count();
+  }
+
+  @Test
+  public void zipPartitions() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2);
+    JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2);
+    FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer> sizesFn =
+      new FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer>() {
+        @Override
+        public Iterator<Integer> call(Iterator<Integer> i, Iterator<String> s) {
+          return Arrays.asList(Iterators.size(i), Iterators.size(s)).iterator();
+        }
+      };
+
+    JavaRDD<Integer> sizes = rdd1.zipPartitions(rdd2, sizesFn);
+    assertEquals("[3, 2, 3, 2]", sizes.collect().toString());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void accumulators() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+
+    final Accumulator<Integer> intAccum = sc.intAccumulator(10);
+    rdd.foreach(new VoidFunction<Integer>() {
+      @Override
+      public void call(Integer x) {
+        intAccum.add(x);
+      }
+    });
+    assertEquals((Integer) 25, intAccum.value());
+
+    final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0);
+    rdd.foreach(new VoidFunction<Integer>() {
+      @Override
+      public void call(Integer x) {
+        doubleAccum.add((double) x);
+      }
+    });
+    assertEquals((Double) 25.0, doubleAccum.value());
+
+    // Try a custom accumulator type
+    AccumulatorParam<Float> floatAccumulatorParam = new AccumulatorParam<Float>() {
+      @Override
+      public Float addInPlace(Float r, Float t) {
+        return r + t;
+      }
+
+      @Override
+      public Float addAccumulator(Float r, Float t) {
+        return r + t;
+      }
+
+      @Override
+      public Float zero(Float initialValue) {
+        return 0.0f;
+      }
+    };
+
+    final Accumulator<Float> floatAccum = sc.accumulator(10.0f, floatAccumulatorParam);
+    rdd.foreach(new VoidFunction<Integer>() {
+      @Override
+      public void call(Integer x) {
+        floatAccum.add((float) x);
+      }
+    });
+    assertEquals((Float) 25.0f, floatAccum.value());
+
+    // Test the setValue method
+    floatAccum.setValue(5.0f);
+    assertEquals((Float) 5.0f, floatAccum.value());
+  }
+
+  @Test
+  public void keyBy() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2));
+    List<Tuple2<String, Integer>> s = rdd.keyBy(new Function<Integer, String>() {
+      @Override
+      public String call(Integer t) {
+        return t.toString();
+      }
+    }).collect();
+    assertEquals(new Tuple2<>("1", 1), s.get(0));
+    assertEquals(new Tuple2<>("2", 2), s.get(1));
+  }
+
+  @Test
+  public void checkpointAndComputation() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    sc.setCheckpointDir(tempDir.getAbsolutePath());
+    assertFalse(rdd.isCheckpointed());
+    rdd.checkpoint();
+    rdd.count(); // Forces the DAG to cause a checkpoint
+    assertTrue(rdd.isCheckpointed());
+    assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect());
+  }
+
+  @Test
+  public void checkpointAndRestore() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    sc.setCheckpointDir(tempDir.getAbsolutePath());
+    assertFalse(rdd.isCheckpointed());
+    rdd.checkpoint();
+    rdd.count(); // Forces the DAG to cause a checkpoint
+    assertTrue(rdd.isCheckpointed());
+
+    assertTrue(rdd.getCheckpointFile().isPresent());
+    JavaRDD<Integer> recovered = sc.checkpointFile(rdd.getCheckpointFile().get());
+    assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect());
+  }
+
+  @Test
+  public void combineByKey() {
+    JavaRDD<Integer> originalRDD = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6));
+    Function<Integer, Integer> keyFunction = new Function<Integer, Integer>() {
+      @Override
+      public Integer call(Integer v1) {
+        return v1 % 3;
+      }
+    };
+    Function<Integer, Integer> createCombinerFunction = new Function<Integer, Integer>() {
+      @Override
+      public Integer call(Integer v1) {
+        return v1;
+      }
+    };
+
+    Function2<Integer, Integer, Integer> mergeValueFunction =
+        new Function2<Integer, Integer, Integer>() {
+      @Override
+      public Integer call(Integer v1, Integer v2) {
+        return v1 + v2;
+      }
+    };
+
+    JavaPairRDD<Integer, Integer> combinedRDD = originalRDD.keyBy(keyFunction)
+        .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction);
+    Map<Integer, Integer> results = combinedRDD.collectAsMap();
+    ImmutableMap<Integer, Integer> expected = ImmutableMap.of(0, 9, 1, 5, 2, 7);
+    assertEquals(expected, results);
+
+    Partitioner defaultPartitioner = Partitioner.defaultPartitioner(
+        combinedRDD.rdd(),
+        JavaConverters.collectionAsScalaIterableConverter(
+            Collections.<RDD<?>>emptyList()).asScala().toSeq());
+    combinedRDD = originalRDD.keyBy(keyFunction)
+        .combineByKey(
+             createCombinerFunction,
+             mergeValueFunction,
+             mergeValueFunction,
+             defaultPartitioner,
+             false,
+             new KryoSerializer(new SparkConf()));
+    results = combinedRDD.collectAsMap();
+    assertEquals(expected, results);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void mapOnPairRDD() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1,2,3,4));
+    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
+        new PairFunction<Integer, Integer, Integer>() {
+          @Override
+          public Tuple2<Integer, Integer> call(Integer i) {
+            return new Tuple2<>(i, i % 2);
+          }
+        });
+    JavaPairRDD<Integer, Integer> rdd3 = rdd2.mapToPair(
+        new PairFunction<Tuple2<Integer, Integer>, Integer, Integer>() {
+          @Override
+          public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> in) {
+            return new Tuple2<>(in._2(), in._1());
+          }
+        });
+    assertEquals(Arrays.asList(
+        new Tuple2<>(1, 1),
+        new Tuple2<>(0, 2),
+        new Tuple2<>(1, 3),
+        new Tuple2<>(0, 4)), rdd3.collect());
+
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void collectPartitions() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3);
+
+    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
+        new PairFunction<Integer, Integer, Integer>() {
+          @Override
+          public Tuple2<Integer, Integer> call(Integer i) {
+            return new Tuple2<>(i, i % 2);
+          }
+        });
+
+    List<Integer>[] parts = rdd1.collectPartitions(new int[] {0});
+    assertEquals(Arrays.asList(1, 2), parts[0]);
+
+    parts = rdd1.collectPartitions(new int[] {1, 2});
+    assertEquals(Arrays.asList(3, 4), parts[0]);
+    assertEquals(Arrays.asList(5, 6, 7), parts[1]);
+
+    assertEquals(Arrays.asList(new Tuple2<>(1, 1),
+                                      new Tuple2<>(2, 0)),
+                        rdd2.collectPartitions(new int[] {0})[0]);
+
+    List<Tuple2<Integer,Integer>>[] parts2 = rdd2.collectPartitions(new int[] {1, 2});
+    assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]);
+    assertEquals(Arrays.asList(new Tuple2<>(5, 1),
+                                      new Tuple2<>(6, 0),
+                                      new Tuple2<>(7, 1)),
+                        parts2[1]);
+  }
+
+  @Test
+  public void countApproxDistinct() {
+    List<Integer> arrayData = new ArrayList<>();
+    int size = 100;
+    for (int i = 0; i < 100000; i++) {
+      arrayData.add(i % size);
+    }
+    JavaRDD<Integer> simpleRdd = sc.parallelize(arrayData, 10);
+    assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1);
+  }
+
+  @Test
+  public void countApproxDistinctByKey() {
+    List<Tuple2<Integer, Integer>> arrayData = new ArrayList<>();
+    for (int i = 10; i < 100; i++) {
+      for (int j = 0; j < i; j++) {
+        arrayData.add(new Tuple2<>(i, j));
+      }
+    }
+    double relativeSD = 0.001;
+    JavaPairRDD<Integer, Integer> pairRdd = sc.parallelizePairs(arrayData);
+    List<Tuple2<Integer, Long>> res =  pairRdd.countApproxDistinctByKey(relativeSD, 8).collect();
+    for (Tuple2<Integer, Long> resItem : res) {
+      double count = resItem._1();
+      long resCount = resItem._2();
+      double error = Math.abs((resCount - count) / count);
+      assertTrue(error < 0.1);
+    }
+
+  }
+
+  @Test
+  public void collectAsMapWithIntArrayValues() {
+    // Regression test for SPARK-1040
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1));
+    JavaPairRDD<Integer, int[]> pairRDD = rdd.mapToPair(
+        new PairFunction<Integer, Integer, int[]>() {
+          @Override
+          public Tuple2<Integer, int[]> call(Integer x) {
+            return new Tuple2<>(x, new int[]{x});
+          }
+        });
+    pairRDD.collect();  // Works fine
+    pairRDD.collectAsMap();  // Used to crash with ClassCastException
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void collectAsMapAndSerialize() throws Exception {
+    JavaPairRDD<String,Integer> rdd =
+        sc.parallelizePairs(Arrays.asList(new Tuple2<>("foo", 1)));
+    Map<String,Integer> map = rdd.collectAsMap();
+    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    new ObjectOutputStream(bytes).writeObject(map);
+    Map<String,Integer> deserializedMap = (Map<String,Integer>)
+        new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject();
+    assertEquals(1, deserializedMap.get("foo").intValue());
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void sampleByKey() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3);
+    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
+      new PairFunction<Integer, Integer, Integer>() {
+        @Override
+        public Tuple2<Integer, Integer> call(Integer i) {
+          return new Tuple2<>(i % 2, 1);
+        }
+      });
+    Map<Integer, Double> fractions = new HashMap<>();
+    fractions.put(0, 0.5);
+    fractions.put(1, 1.0);
+    JavaPairRDD<Integer, Integer> wr = rdd2.sampleByKey(true, fractions, 1L);
+    Map<Integer, Long> wrCounts = wr.countByKey();
+    assertEquals(2, wrCounts.size());
+    assertTrue(wrCounts.get(0) > 0);
+    assertTrue(wrCounts.get(1) > 0);
+    JavaPairRDD<Integer, Integer> wor = rdd2.sampleByKey(false, fractions, 1L);
+    Map<Integer, Long> worCounts = wor.countByKey();
+    assertEquals(2, worCounts.size());
+    assertTrue(worCounts.get(0) > 0);
+    assertTrue(worCounts.get(1) > 0);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void sampleByKeyExact() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3);
+    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
+      new PairFunction<Integer, Integer, Integer>() {
+          @Override
+          public Tuple2<Integer, Integer> call(Integer i) {
+              return new Tuple2<>(i % 2, 1);
+          }
+      });
+    Map<Integer, Double> fractions = new HashMap<>();
+    fractions.put(0, 0.5);
+    fractions.put(1, 1.0);
+    JavaPairRDD<Integer, Integer> wrExact = rdd2.sampleByKeyExact(true, fractions, 1L);
+    Map<Integer, Long> wrExactCounts = wrExact.countByKey();
+    assertEquals(2, wrExactCounts.size());
+    assertTrue(wrExactCounts.get(0) == 2);
+    assertTrue(wrExactCounts.get(1) == 4);
+    JavaPairRDD<Integer, Integer> worExact = rdd2.sampleByKeyExact(false, fractions, 1L);
+    Map<Integer, Long> worExactCounts = worExact.countByKey();
+    assertEquals(2, worExactCounts.size());
+    assertTrue(worExactCounts.get(0) == 2);
+    assertTrue(worExactCounts.get(1) == 4);
+  }
+
+  private static class SomeCustomClass implements Serializable {
+    SomeCustomClass() {
+      // Intentionally left blank
+    }
+  }
+
+  @Test
+  public void collectUnderlyingScalaRDD() {
+    List<SomeCustomClass> data = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      data.add(new SomeCustomClass());
+    }
+    JavaRDD<SomeCustomClass> rdd = sc.parallelize(data);
+    SomeCustomClass[] collected =
+      (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect();
+    assertEquals(data.size(), collected.length);
+  }
+
+  private static final class BuggyMapFunction<T> implements Function<T, T> {
+
+    @Override
+    public T call(T x) {
+      throw new IllegalStateException("Custom exception!");
+    }
+  }
+
+  @Test
+  public void collectAsync() throws Exception {
+    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
+    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
+    JavaFutureAction<List<Integer>> future = rdd.collectAsync();
+    List<Integer> result = future.get();
+    assertEquals(data, result);
+    assertFalse(future.isCancelled());
+    assertTrue(future.isDone());
+    assertEquals(1, future.jobIds().size());
+  }
+
+  @Test
+  public void takeAsync() throws Exception {
+    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
+    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
+    JavaFutureAction<List<Integer>> future = rdd.takeAsync(1);
+    List<Integer> result = future.get();
+    assertEquals(1, result.size());
+    assertEquals((Integer) 1, result.get(0));
+    assertFalse(future.isCancelled());
+    assertTrue(future.isDone());
+    assertEquals(1, future.jobIds().size());
+  }
+
+  @Test
+  public void foreachAsync() throws Exception {
+    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
+    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
+    JavaFutureAction<Void> future = rdd.foreachAsync(
+        new VoidFunction<Integer>() {
+          @Override
+          public void call(Integer integer) {
+            // intentionally left blank.
+          }
+        }
+    );
+    future.get();
+    assertFalse(future.isCancelled());
+    assertTrue(future.isDone());
+    assertEquals(1, future.jobIds().size());
+  }
+
+  @Test
+  public void countAsync() throws Exception {
+    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
+    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
+    JavaFutureAction<Long> future = rdd.countAsync();
+    long count = future.get();
+    assertEquals(data.size(), count);
+    assertFalse(future.isCancelled());
+    assertTrue(future.isDone());
+    assertEquals(1, future.jobIds().size());
+  }
+
+  @Test
+  public void testAsyncActionCancellation() throws Exception {
+    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
+    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
+    JavaFutureAction<Void> future = rdd.foreachAsync(new VoidFunction<Integer>() {
+      @Override
+      public void call(Integer integer) throws InterruptedException {
+        Thread.sleep(10000);  // To ensure that the job won't finish before it's cancelled.
+      }
+    });
+    future.cancel(true);
+    assertTrue(future.isCancelled());
+    assertTrue(future.isDone());
+    try {
+      future.get(2000, TimeUnit.MILLISECONDS);
+      fail("Expected future.get() for cancelled job to throw CancellationException");
+    } catch (CancellationException ignored) {
+      // pass
+    }
+  }
+
+  @Test
+  public void testAsyncActionErrorWrapping() throws Exception {
+    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
+    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
+    JavaFutureAction<Long> future = rdd.map(new BuggyMapFunction<Integer>()).countAsync();
+    try {
+      future.get(2, TimeUnit.SECONDS);
+      fail("Expected future.get() for failed job to throw ExcecutionException");
+    } catch (ExecutionException ee) {
+      assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!"));
+    }
+    assertTrue(future.isDone());
+  }
+
+  static class Class1 {}
+  static class Class2 {}
+
+  @Test
+  public void testRegisterKryoClasses() {
+    SparkConf conf = new SparkConf();
+    conf.registerKryoClasses(new Class<?>[]{ Class1.class, Class2.class });
+    assertEquals(
+        Class1.class.getName() + "," + Class2.class.getName(),
+        conf.get("spark.kryo.classesToRegister"));
+  }
+
+  @Test
+  public void testGetPersistentRDDs() {
+    java.util.Map<Integer, JavaRDD<?>> cachedRddsMap = sc.getPersistentRDDs();
+    assertTrue(cachedRddsMap.isEmpty());
+    JavaRDD<String> rdd1 = sc.parallelize(Arrays.asList("a", "b")).setName("RDD1").cache();
+    JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("c", "d")).setName("RDD2").cache();
+    cachedRddsMap = sc.getPersistentRDDs();
+    assertEquals(2, cachedRddsMap.size());
+    assertEquals("RDD1", cachedRddsMap.get(0).name());
+    assertEquals("RDD2", cachedRddsMap.get(1).name());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 6027310..43f77e6 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -919,7 +919,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
         assert(pidExists(pid))
         val terminated = Utils.terminateProcess(process, 5000)
         assert(terminated.isDefined)
-        Utils.waitForProcess(process, 5000)
+        process.waitFor(5, TimeUnit.SECONDS)
         val durationMs = System.currentTimeMillis() - startTimeMs
         assert(durationMs < 5000)
         assert(!pidExists(pid))
@@ -932,7 +932,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
       var majorVersion = versionParts(0).toInt
       if (majorVersion == 1) majorVersion = versionParts(1).toInt
       if (majorVersion >= 8) {
-        // Java8 added a way to forcibly terminate a process. We'll make sure that works by
+        // We'll make sure that forcibly terminating a process works by
         // creating a very misbehaving process. It ignores SIGTERM and has been SIGSTOPed. On
         // older versions of java, this will *not* terminate.
         val file = File.createTempFile("temp-file-name", ".tmp")
@@ -953,7 +953,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
           val start = System.currentTimeMillis()
           val terminated = Utils.terminateProcess(process, 5000)
           assert(terminated.isDefined)
-          Utils.waitForProcess(process, 5000)
+          process.waitFor(5, TimeUnit.SECONDS)
           val duration = System.currentTimeMillis() - start
           assert(duration < 6000) // add a little extra time to allow a force kill to finish
           assert(!pidExists(pid))

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/dev/appveyor-install-dependencies.ps1
----------------------------------------------------------------------
diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1
index 1350095..1c34f1b 100644
--- a/dev/appveyor-install-dependencies.ps1
+++ b/dev/appveyor-install-dependencies.ps1
@@ -90,7 +90,7 @@ Invoke-Expression "7z.exe x maven.zip"
 # add maven to environment variables
 $env:Path += ";$tools\apache-maven-$mavenVer\bin"
 $env:M2_HOME = "$tools\apache-maven-$mavenVer"
-$env:MAVEN_OPTS = "-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"
+$env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=512m"
 
 Pop-Location
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/dev/create-release/release-build.sh
----------------------------------------------------------------------
diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh
index d616f80..e1db997 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -267,7 +267,6 @@ if [[ "$1" == "docs" ]]; then
   echo "Building Spark docs"
   dest_dir="$REMOTE_PARENT_DIR/${DEST_DIR_NAME}-docs"
   cd docs
-  # Compile docs with Java 7 to use nicer format
   # TODO: Make configurable to add this: PRODUCTION=1
   PRODUCTION=1 RELEASE_VERSION="$SPARK_VERSION" jekyll build
   echo "Copying release documentation to $dest_dir"

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/dev/make-distribution.sh
----------------------------------------------------------------------
diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh
index dc8dfb9..22cdfd4 100755
--- a/dev/make-distribution.sh
+++ b/dev/make-distribution.sh
@@ -146,7 +146,7 @@ fi
 # Build uber fat JAR
 cd "$SPARK_HOME"
 
-export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m}"
+export MAVEN_OPTS="${MAVEN_OPTS:-Xmx2g -XX:ReservedCodeCacheSize=512m}"
 
 # Store the command as an array because $MVN variable might have spaces in it.
 # Normal quoting tricks don't work.

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/dev/mima
----------------------------------------------------------------------
diff --git a/dev/mima b/dev/mima
index 11c4af2..eca78ad 100755
--- a/dev/mima
+++ b/dev/mima
@@ -31,7 +31,6 @@ OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export
 rm -f .generated-mima*
 
 java \
-  -XX:MaxPermSize=1g \
   -Xmx2g \
   -cp "$TOOLS_CLASSPATH:$OLD_DEPS_CLASSPATH" \
   org.apache.spark.tools.GenerateMIMAIgnore

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/dev/run-tests.py
----------------------------------------------------------------------
diff --git a/dev/run-tests.py b/dev/run-tests.py
index 0e7f5ff..04035b3 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -492,9 +492,6 @@ def main():
 
     java_version = determine_java_version(java_exe)
 
-    if java_version.minor < 8:
-        print("[warn] Java 8 tests will not run because JDK version is < 1.8.")
-
     # install SparkR
     if which("R"):
         run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")])

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/dev/test-dependencies.sh
----------------------------------------------------------------------
diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh
index eb43f22..2906a81 100755
--- a/dev/test-dependencies.sh
+++ b/dev/test-dependencies.sh
@@ -46,7 +46,7 @@ OLD_VERSION=$($MVN -q \
     -Dexec.executable="echo" \
     -Dexec.args='${project.version}' \
     --non-recursive \
-    org.codehaus.mojo:exec-maven-plugin:1.3.1:exec)
+    org.codehaus.mojo:exec-maven-plugin:1.5.0:exec)
 if [ $? != 0 ]; then
     echo -e "Error while getting version string from Maven:\n$OLD_VERSION"
     exit 1

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/building-spark.md
----------------------------------------------------------------------
diff --git a/docs/building-spark.md b/docs/building-spark.md
index 690c656..56b8926 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -12,8 +12,8 @@ redirect_from: "building-with-maven.html"
 ## Apache Maven
 
 The Maven-based build is the build of reference for Apache Spark.
-Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+.
-Note that support for Java 7 is deprecated as of Spark 2.0.0 and may be removed in Spark 2.2.0.
+Building Spark using Maven requires Maven 3.3.9 or newer and Java 8+.
+Note that support for Java 7 was removed as of Spark 2.2.0.
 
 ### Setting up Maven's Memory Usage
 
@@ -21,28 +21,18 @@ You'll need to configure Maven to use more memory than usual by setting `MAVEN_O
 
     export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m"
 
-When compiling with Java 7, you will need to add the additional option "-XX:MaxPermSize=512M" to MAVEN_OPTS.
-
+(The `ReservedCodeCacheSize` setting is optional but recommended.)
 If you don't add these parameters to `MAVEN_OPTS`, you may see errors and warnings like the following:
 
     [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes...
-    [ERROR] PermGen space -> [Help 1]
-
-    [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes...
     [ERROR] Java heap space -> [Help 1]
 
-    [INFO] Compiling 233 Scala sources and 41 Java sources to /Users/me/Development/spark/sql/core/target/scala-{site.SCALA_BINARY_VERSION}/classes...
-    OpenJDK 64-Bit Server VM warning: CodeCache is full. Compiler has been disabled.
-    OpenJDK 64-Bit Server VM warning: Try increasing the code cache size using -XX:ReservedCodeCacheSize=
-
 You can fix these problems by setting the `MAVEN_OPTS` variable as discussed before.
 
 **Note:**
 
 * If using `build/mvn` with no `MAVEN_OPTS` set, the script will automatically add the above options to the `MAVEN_OPTS` environment variable.
-* The `test` phase of the Spark build will automatically add these options to `MAVEN_OPTS`, even when not using `build/mvn`.
-* You may see warnings like "ignoring option MaxPermSize=1g; support was removed in 8.0" when building or running tests with Java 8 and `build/mvn`. These warnings are harmless.
-
+* The `test` phase of the Spark build will automatically add these options to `MAVEN_OPTS`, even when not using `build/mvn`.    
 
 ### build/mvn
 
@@ -224,20 +214,6 @@ To run test suites of a specific sub project as follows:
 
     ./build/sbt core/test
 
-## Running Java 8 Test Suites
-
-Running only Java 8 tests and nothing else.
-
-    ./build/mvn install -DskipTests
-    ./build/mvn -pl :java8-tests_2.11 test
-
-or
-
-    ./build/sbt java8-tests/test
-
-Java 8 tests are automatically enabled when a Java 8 JDK is detected.
-If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests.
-
 ## PySpark pip installable
 
 If you are building Spark for use in a Python environment and you wish to pip install it, you will first need to build the Spark JARs as described above. Then you can construct an sdist package suitable for setup.py and pip installable package.

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index 023e06a..19a9d3b 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -26,11 +26,13 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy
 locally on one machine --- all you need is to have `java` installed on your system `PATH`,
 or the `JAVA_HOME` environment variable pointing to a Java installation.
 
-Spark runs on Java 7+, Python 2.6+/3.4+ and R 3.1+. For the Scala API, Spark {{site.SPARK_VERSION}}
+Spark runs on Java 8+, Python 2.6+/3.4+ and R 3.1+. For the Scala API, Spark {{site.SPARK_VERSION}}
 uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version
 ({{site.SCALA_BINARY_VERSION}}.x).
 
-Note that support for Java 7 and Python 2.6 are deprecated as of Spark 2.0.0, and support for 
+Note that support for Java 7 was removed as of Spark 2.2.0.
+
+Note that support for Python 2.6 is deprecated as of Spark 2.0.0, and support for 
 Scala 2.10 and versions of Hadoop before 2.6 are deprecated as of Spark 2.1.0, and may be 
 removed in Spark 2.2.0.
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/mllib-linear-methods.md
----------------------------------------------------------------------
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md
index 3085539..034e89e 100644
--- a/docs/mllib-linear-methods.md
+++ b/docs/mllib-linear-methods.md
@@ -222,7 +222,7 @@ svmAlg.optimizer()
   .setNumIterations(200)
   .setRegParam(0.1)
   .setUpdater(new L1Updater());
-final SVMModel modelL1 = svmAlg.run(training.rdd());
+SVMModel modelL1 = svmAlg.run(training.rdd());
 {% endhighlight %}
 
 In order to run the above application, follow the instructions

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/mllib-statistics.md
----------------------------------------------------------------------
diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md
index 430c069..c29400a 100644
--- a/docs/mllib-statistics.md
+++ b/docs/mllib-statistics.md
@@ -317,12 +317,7 @@ JavaSparkContext jsc = ...
 // standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions.
 JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10);
 // Apply a transform to get a random double RDD following `N(1, 4)`.
-JavaDoubleRDD v = u.map(
-  new Function<Double, Double>() {
-    public Double call(Double x) {
-      return 1.0 + 2.0 * x;
-    }
-  });
+JavaDoubleRDD v = u.mapToDouble(x -> 1.0 + 2.0 * x);
 {% endhighlight %}
 </div>
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/programming-guide.md b/docs/programming-guide.md
index db8b048..6740dbe 100644
--- a/docs/programming-guide.md
+++ b/docs/programming-guide.md
@@ -54,12 +54,12 @@ import org.apache.spark.SparkConf
 
 <div data-lang="java"  markdown="1">
 
-Spark {{site.SPARK_VERSION}} works with Java 7 and higher. If you are using Java 8, Spark supports
+Spark {{site.SPARK_VERSION}} supports
 [lambda expressions](http://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html)
 for concisely writing functions, otherwise you can use the classes in the
 [org.apache.spark.api.java.function](api/java/index.html?org/apache/spark/api/java/function/package-summary.html) package.
 
-Note that support for Java 7 is deprecated as of Spark 2.0.0 and may be removed in Spark 2.2.0.
+Note that support for Java 7 was removed in Spark 2.2.0.
 
 To write a Spark application in Java, you need to add a dependency on Spark. Spark is available through Maven Central at:
 
@@ -295,11 +295,6 @@ JavaRDD<Integer> distData = sc.parallelize(data);
 Once created, the distributed dataset (`distData`) can be operated on in parallel. For example, we might call `distData.reduce((a, b) -> a + b)` to add up the elements of the list.
 We describe operations on distributed datasets later on.
 
-**Note:** *In this guide, we'll often use the concise Java 8 lambda syntax to specify Java functions, but
-in older versions of Java you can implement the interfaces in the
-[org.apache.spark.api.java.function](api/java/index.html?org/apache/spark/api/java/function/package-summary.html) package.
-We describe [passing functions to Spark](#passing-functions-to-spark) in more detail below.*
-
 </div>
 
 <div data-lang="python"  markdown="1">
@@ -658,7 +653,7 @@ There are two ways to create such functions:
 
 * Implement the Function interfaces in your own class, either as an anonymous inner class or a named one,
   and pass an instance of it to Spark.
-* In Java 8, use [lambda expressions](http://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html)
+* Use [lambda expressions](http://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html)
   to concisely define an implementation.
 
 While much of this guide uses lambda syntax for conciseness, it is easy to use all the same APIs

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/quick-start.md
----------------------------------------------------------------------
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 0836c60..04ac278 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -320,13 +320,8 @@ public class SimpleApp {
     JavaSparkContext sc = new JavaSparkContext(conf);
     JavaRDD<String> logData = sc.textFile(logFile).cache();
 
-    long numAs = logData.filter(new Function<String, Boolean>() {
-      public Boolean call(String s) { return s.contains("a"); }
-    }).count();
-
-    long numBs = logData.filter(new Function<String, Boolean>() {
-      public Boolean call(String s) { return s.contains("b"); }
-    }).count();
+    long numAs = logData.filter(s -> s.contains("a")).count();
+    long numBs = logData.filter(s -> s.contains("b")).count();
 
     System.out.println("Lines with a: " + numAs + ", lines with b: " + numBs);
     

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/streaming-custom-receivers.md
----------------------------------------------------------------------
diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md
index 117996d..d4ddcb1 100644
--- a/docs/streaming-custom-receivers.md
+++ b/docs/streaming-custom-receivers.md
@@ -113,15 +113,13 @@ public class JavaCustomReceiver extends Receiver<String> {
     port = port_;
   }
 
+  @Override
   public void onStart() {
     // Start the thread that receives data over a connection
-    new Thread()  {
-      @Override public void run() {
-        receive();
-      }
-    }.start();
+    new Thread(this::receive).start();
   }
 
+  @Override
   public void onStop() {
     // There is nothing much to do as the thread calling receive()
     // is designed to stop by itself if isStopped() returns false
@@ -189,7 +187,7 @@ The full source code is in the example [CustomReceiver.scala]({{site.SPARK_GITHU
 {% highlight java %}
 // Assuming ssc is the JavaStreamingContext
 JavaDStream<String> customReceiverStream = ssc.receiverStream(new JavaCustomReceiver(host, port));
-JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() { ... });
+JavaDStream<String> words = lines.flatMap(s -> ...);
 ...
 {% endhighlight %}
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/streaming-kafka-0-10-integration.md
----------------------------------------------------------------------
diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md
index 6ef54ac..e383701 100644
--- a/docs/streaming-kafka-0-10-integration.md
+++ b/docs/streaming-kafka-0-10-integration.md
@@ -68,20 +68,14 @@ kafkaParams.put("enable.auto.commit", false);
 
 Collection<String> topics = Arrays.asList("topicA", "topicB");
 
-final JavaInputDStream<ConsumerRecord<String, String>> stream =
+JavaInputDStream<ConsumerRecord<String, String>> stream =
   KafkaUtils.createDirectStream(
     streamingContext,
     LocationStrategies.PreferConsistent(),
     ConsumerStrategies.<String, String>Subscribe(topics, kafkaParams)
   );
 
-stream.mapToPair(
-  new PairFunction<ConsumerRecord<String, String>, String, String>() {
-    @Override
-    public Tuple2<String, String> call(ConsumerRecord<String, String> record) {
-      return new Tuple2<>(record.key(), record.value());
-    }
-  })
+stream.mapToPair(record -> new Tuple2<>(record.key(), record.value()));
 {% endhighlight %}
 </div>
 </div>
@@ -162,19 +156,13 @@ stream.foreachRDD { rdd =>
 </div>
 <div data-lang="java" markdown="1">
 {% highlight java %}
-stream.foreachRDD(new VoidFunction<JavaRDD<ConsumerRecord<String, String>>>() {
-  @Override
-  public void call(JavaRDD<ConsumerRecord<String, String>> rdd) {
-    final OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
-    rdd.foreachPartition(new VoidFunction<Iterator<ConsumerRecord<String, String>>>() {
-      @Override
-      public void call(Iterator<ConsumerRecord<String, String>> consumerRecords) {
-        OffsetRange o = offsetRanges[TaskContext.get().partitionId()];
-        System.out.println(
-          o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset());
-      }
-    });
-  }
+stream.foreachRDD(rdd -> {
+  OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
+  rdd.foreachPartition(consumerRecords -> {
+    OffsetRange o = offsetRanges[TaskContext.get().partitionId()];
+    System.out.println(
+      o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset());
+  });
 });
 {% endhighlight %}
 </div>
@@ -205,14 +193,11 @@ As with HasOffsetRanges, the cast to CanCommitOffsets will only succeed if calle
 </div>
 <div data-lang="java" markdown="1">
 {% highlight java %}
-stream.foreachRDD(new VoidFunction<JavaRDD<ConsumerRecord<String, String>>>() {
-  @Override
-  public void call(JavaRDD<ConsumerRecord<String, String>> rdd) {
-    OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
+stream.foreachRDD(rdd -> {
+  OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
 
-    // some time later, after outputs have completed
-    ((CanCommitOffsets) stream.inputDStream()).commitAsync(offsetRanges);
-  }
+  // some time later, after outputs have completed
+  ((CanCommitOffsets) stream.inputDStream()).commitAsync(offsetRanges);
 });
 {% endhighlight %}
 </div>
@@ -268,21 +253,18 @@ JavaInputDStream<ConsumerRecord<String, String>> stream = KafkaUtils.createDirec
   ConsumerStrategies.<String, String>Assign(fromOffsets.keySet(), kafkaParams, fromOffsets)
 );
 
-stream.foreachRDD(new VoidFunction<JavaRDD<ConsumerRecord<String, String>>>() {
-  @Override
-  public void call(JavaRDD<ConsumerRecord<String, String>> rdd) {
-    OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
-    
-    Object results = yourCalculation(rdd);
+stream.foreachRDD(rdd -> {
+  OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
+  
+  Object results = yourCalculation(rdd);
 
-    // begin your transaction
+  // begin your transaction
 
-    // update results
-    // update offsets where the end of existing offsets matches the beginning of this batch of offsets
-    // assert that offsets were updated correctly
+  // update results
+  // update offsets where the end of existing offsets matches the beginning of this batch of offsets
+  // assert that offsets were updated correctly
 
-    // end your transaction
-  }
+  // end your transaction
 });
 {% endhighlight %}
 </div>


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[7/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/test/java/org/apache/spark/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java
deleted file mode 100644
index 7bebe06..0000000
--- a/core/src/test/java/org/apache/spark/JavaAPISuite.java
+++ /dev/null
@@ -1,1836 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark;
-
-import java.io.*;
-import java.nio.channels.FileChannel;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.*;
-
-import scala.Tuple2;
-import scala.Tuple3;
-import scala.Tuple4;
-import scala.collection.JavaConverters;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-import com.google.common.base.Throwables;
-import com.google.common.io.Files;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.After;
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.spark.api.java.JavaDoubleRDD;
-import org.apache.spark.api.java.JavaFutureAction;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.Optional;
-import org.apache.spark.api.java.function.*;
-import org.apache.spark.input.PortableDataStream;
-import org.apache.spark.partial.BoundedDouble;
-import org.apache.spark.partial.PartialResult;
-import org.apache.spark.rdd.RDD;
-import org.apache.spark.serializer.KryoSerializer;
-import org.apache.spark.storage.StorageLevel;
-import org.apache.spark.util.LongAccumulator;
-import org.apache.spark.util.StatCounter;
-
-// The test suite itself is Serializable so that anonymous Function implementations can be
-// serialized, as an alternative to converting these anonymous classes to static inner classes;
-// see http://stackoverflow.com/questions/758570/.
-public class JavaAPISuite implements Serializable {
-  private transient JavaSparkContext sc;
-  private transient File tempDir;
-
-  @Before
-  public void setUp() {
-    sc = new JavaSparkContext("local", "JavaAPISuite");
-    tempDir = Files.createTempDir();
-    tempDir.deleteOnExit();
-  }
-
-  @After
-  public void tearDown() {
-    sc.stop();
-    sc = null;
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void sparkContextUnion() {
-    // Union of non-specialized JavaRDDs
-    List<String> strings = Arrays.asList("Hello", "World");
-    JavaRDD<String> s1 = sc.parallelize(strings);
-    JavaRDD<String> s2 = sc.parallelize(strings);
-    // Varargs
-    JavaRDD<String> sUnion = sc.union(s1, s2);
-    assertEquals(4, sUnion.count());
-    // List
-    List<JavaRDD<String>> list = new ArrayList<>();
-    list.add(s2);
-    sUnion = sc.union(s1, list);
-    assertEquals(4, sUnion.count());
-
-    // Union of JavaDoubleRDDs
-    List<Double> doubles = Arrays.asList(1.0, 2.0);
-    JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles);
-    JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles);
-    JavaDoubleRDD dUnion = sc.union(d1, d2);
-    assertEquals(4, dUnion.count());
-
-    // Union of JavaPairRDDs
-    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
-    pairs.add(new Tuple2<>(1, 2));
-    pairs.add(new Tuple2<>(3, 4));
-    JavaPairRDD<Integer, Integer> p1 = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> p2 = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> pUnion = sc.union(p1, p2);
-    assertEquals(4, pUnion.count());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void intersection() {
-    List<Integer> ints1 = Arrays.asList(1, 10, 2, 3, 4, 5);
-    List<Integer> ints2 = Arrays.asList(1, 6, 2, 3, 7, 8);
-    JavaRDD<Integer> s1 = sc.parallelize(ints1);
-    JavaRDD<Integer> s2 = sc.parallelize(ints2);
-
-    JavaRDD<Integer> intersections = s1.intersection(s2);
-    assertEquals(3, intersections.count());
-
-    JavaRDD<Integer> empty = sc.emptyRDD();
-    JavaRDD<Integer> emptyIntersection = empty.intersection(s2);
-    assertEquals(0, emptyIntersection.count());
-
-    List<Double> doubles = Arrays.asList(1.0, 2.0);
-    JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles);
-    JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles);
-    JavaDoubleRDD dIntersection = d1.intersection(d2);
-    assertEquals(2, dIntersection.count());
-
-    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
-    pairs.add(new Tuple2<>(1, 2));
-    pairs.add(new Tuple2<>(3, 4));
-    JavaPairRDD<Integer, Integer> p1 = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> p2 = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> pIntersection = p1.intersection(p2);
-    assertEquals(2, pIntersection.count());
-  }
-
-  @Test
-  public void sample() {
-    List<Integer> ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
-    JavaRDD<Integer> rdd = sc.parallelize(ints);
-    // the seeds here are "magic" to make this work out nicely
-    JavaRDD<Integer> sample20 = rdd.sample(true, 0.2, 8);
-    assertEquals(2, sample20.count());
-    JavaRDD<Integer> sample20WithoutReplacement = rdd.sample(false, 0.2, 2);
-    assertEquals(2, sample20WithoutReplacement.count());
-  }
-
-  @Test
-  public void randomSplit() {
-    List<Integer> ints = new ArrayList<>(1000);
-    for (int i = 0; i < 1000; i++) {
-      ints.add(i);
-    }
-    JavaRDD<Integer> rdd = sc.parallelize(ints);
-    JavaRDD<Integer>[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 31);
-    // the splits aren't perfect -- not enough data for them to be -- just check they're about right
-    assertEquals(3, splits.length);
-    long s0 = splits[0].count();
-    long s1 = splits[1].count();
-    long s2 = splits[2].count();
-    assertTrue(s0 + " not within expected range", s0 > 150 && s0 < 250);
-    assertTrue(s1 + " not within expected range", s1 > 250 && s0 < 350);
-    assertTrue(s2 + " not within expected range", s2 > 430 && s2 < 570);
-  }
-
-  @Test
-  public void sortByKey() {
-    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
-    pairs.add(new Tuple2<>(0, 4));
-    pairs.add(new Tuple2<>(3, 2));
-    pairs.add(new Tuple2<>(-1, 1));
-
-    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
-
-    // Default comparator
-    JavaPairRDD<Integer, Integer> sortedRDD = rdd.sortByKey();
-    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
-    List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect();
-    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1));
-    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2));
-
-    // Custom comparator
-    sortedRDD = rdd.sortByKey(Collections.<Integer>reverseOrder(), false);
-    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
-    sortedPairs = sortedRDD.collect();
-    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1));
-    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void repartitionAndSortWithinPartitions() {
-    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
-    pairs.add(new Tuple2<>(0, 5));
-    pairs.add(new Tuple2<>(3, 8));
-    pairs.add(new Tuple2<>(2, 6));
-    pairs.add(new Tuple2<>(0, 8));
-    pairs.add(new Tuple2<>(3, 8));
-    pairs.add(new Tuple2<>(1, 3));
-
-    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
-
-    Partitioner partitioner = new Partitioner() {
-      @Override
-      public int numPartitions() {
-        return 2;
-      }
-      @Override
-      public int getPartition(Object key) {
-        return (Integer) key % 2;
-      }
-    };
-
-    JavaPairRDD<Integer, Integer> repartitioned =
-        rdd.repartitionAndSortWithinPartitions(partitioner);
-    assertTrue(repartitioned.partitioner().isPresent());
-    assertEquals(repartitioned.partitioner().get(), partitioner);
-    List<List<Tuple2<Integer, Integer>>> partitions = repartitioned.glom().collect();
-    assertEquals(partitions.get(0),
-        Arrays.asList(new Tuple2<>(0, 5), new Tuple2<>(0, 8), new Tuple2<>(2, 6)));
-    assertEquals(partitions.get(1),
-        Arrays.asList(new Tuple2<>(1, 3), new Tuple2<>(3, 8), new Tuple2<>(3, 8)));
-  }
-
-  @Test
-  public void emptyRDD() {
-    JavaRDD<String> rdd = sc.emptyRDD();
-    assertEquals("Empty RDD shouldn't have any values", 0, rdd.count());
-  }
-
-  @Test
-  public void sortBy() {
-    List<Tuple2<Integer, Integer>> pairs = new ArrayList<>();
-    pairs.add(new Tuple2<>(0, 4));
-    pairs.add(new Tuple2<>(3, 2));
-    pairs.add(new Tuple2<>(-1, 1));
-
-    JavaRDD<Tuple2<Integer, Integer>> rdd = sc.parallelize(pairs);
-
-    // compare on first value
-    JavaRDD<Tuple2<Integer, Integer>> sortedRDD =
-        rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() {
-      @Override
-      public Integer call(Tuple2<Integer, Integer> t) {
-        return t._1();
-      }
-    }, true, 2);
-
-    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
-    List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect();
-    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1));
-    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2));
-
-    // compare on second value
-    sortedRDD = rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() {
-      @Override
-      public Integer call(Tuple2<Integer, Integer> t) {
-        return t._2();
-      }
-    }, true, 2);
-    assertEquals(new Tuple2<>(-1, 1), sortedRDD.first());
-    sortedPairs = sortedRDD.collect();
-    assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1));
-    assertEquals(new Tuple2<>(0, 4), sortedPairs.get(2));
-  }
-
-  @Test
-  public void foreach() {
-    final LongAccumulator accum = sc.sc().longAccumulator();
-    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
-    rdd.foreach(new VoidFunction<String>() {
-      @Override
-      public void call(String s) {
-        accum.add(1);
-      }
-    });
-    assertEquals(2, accum.value().intValue());
-  }
-
-  @Test
-  public void foreachPartition() {
-    final LongAccumulator accum = sc.sc().longAccumulator();
-    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
-    rdd.foreachPartition(new VoidFunction<Iterator<String>>() {
-      @Override
-      public void call(Iterator<String> iter) {
-        while (iter.hasNext()) {
-          iter.next();
-          accum.add(1);
-        }
-      }
-    });
-    assertEquals(2, accum.value().intValue());
-  }
-
-  @Test
-  public void toLocalIterator() {
-    List<Integer> correct = Arrays.asList(1, 2, 3, 4);
-    JavaRDD<Integer> rdd = sc.parallelize(correct);
-    List<Integer> result = Lists.newArrayList(rdd.toLocalIterator());
-    assertEquals(correct, result);
-  }
-
-  @Test
-  public void zipWithUniqueId() {
-    List<Integer> dataArray = Arrays.asList(1, 2, 3, 4);
-    JavaPairRDD<Integer, Long> zip = sc.parallelize(dataArray).zipWithUniqueId();
-    JavaRDD<Long> indexes = zip.values();
-    assertEquals(4, new HashSet<>(indexes.collect()).size());
-  }
-
-  @Test
-  public void zipWithIndex() {
-    List<Integer> dataArray = Arrays.asList(1, 2, 3, 4);
-    JavaPairRDD<Integer, Long> zip = sc.parallelize(dataArray).zipWithIndex();
-    JavaRDD<Long> indexes = zip.values();
-    List<Long> correctIndexes = Arrays.asList(0L, 1L, 2L, 3L);
-    assertEquals(correctIndexes, indexes.collect());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void lookup() {
-    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Apples", "Fruit"),
-      new Tuple2<>("Oranges", "Fruit"),
-      new Tuple2<>("Oranges", "Citrus")
-    ));
-    assertEquals(2, categories.lookup("Oranges").size());
-    assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0)));
-  }
-
-  @Test
-  public void groupBy() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    Function<Integer, Boolean> isOdd = new Function<Integer, Boolean>() {
-      @Override
-      public Boolean call(Integer x) {
-        return x % 2 == 0;
-      }
-    };
-    JavaPairRDD<Boolean, Iterable<Integer>> oddsAndEvens = rdd.groupBy(isOdd);
-    assertEquals(2, oddsAndEvens.count());
-    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
-    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
-
-    oddsAndEvens = rdd.groupBy(isOdd, 1);
-    assertEquals(2, oddsAndEvens.count());
-    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
-    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
-  }
-
-  @Test
-  public void groupByOnPairRDD() {
-    // Regression test for SPARK-4459
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    Function<Tuple2<Integer, Integer>, Boolean> areOdd =
-      new Function<Tuple2<Integer, Integer>, Boolean>() {
-        @Override
-        public Boolean call(Tuple2<Integer, Integer> x) {
-          return (x._1() % 2 == 0) && (x._2() % 2 == 0);
-        }
-      };
-    JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd);
-    JavaPairRDD<Boolean, Iterable<Tuple2<Integer, Integer>>> oddsAndEvens = pairRDD.groupBy(areOdd);
-    assertEquals(2, oddsAndEvens.count());
-    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
-    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
-
-    oddsAndEvens = pairRDD.groupBy(areOdd, 1);
-    assertEquals(2, oddsAndEvens.count());
-    assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
-    assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void keyByOnPairRDD() {
-    // Regression test for SPARK-4459
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    Function<Tuple2<Integer, Integer>, String> sumToString =
-      new Function<Tuple2<Integer, Integer>, String>() {
-        @Override
-        public String call(Tuple2<Integer, Integer> x) {
-          return String.valueOf(x._1() + x._2());
-        }
-      };
-    JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd);
-    JavaPairRDD<String, Tuple2<Integer, Integer>> keyed = pairRDD.keyBy(sumToString);
-    assertEquals(7, keyed.count());
-    assertEquals(1, (long) keyed.lookup("2").get(0)._1());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void cogroup() {
-    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Apples", "Fruit"),
-      new Tuple2<>("Oranges", "Fruit"),
-      new Tuple2<>("Oranges", "Citrus")
-      ));
-    JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Oranges", 2),
-      new Tuple2<>("Apples", 3)
-    ));
-    JavaPairRDD<String, Tuple2<Iterable<String>, Iterable<Integer>>> cogrouped =
-        categories.cogroup(prices);
-    assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1()));
-    assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2()));
-
-    cogrouped.collect();
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void cogroup3() {
-    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Apples", "Fruit"),
-      new Tuple2<>("Oranges", "Fruit"),
-      new Tuple2<>("Oranges", "Citrus")
-      ));
-    JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Oranges", 2),
-      new Tuple2<>("Apples", 3)
-    ));
-    JavaPairRDD<String, Integer> quantities = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Oranges", 21),
-      new Tuple2<>("Apples", 42)
-    ));
-
-    JavaPairRDD<String, Tuple3<Iterable<String>, Iterable<Integer>, Iterable<Integer>>> cogrouped =
-        categories.cogroup(prices, quantities);
-    assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1()));
-    assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2()));
-    assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3()));
-
-
-    cogrouped.collect();
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void cogroup4() {
-    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Apples", "Fruit"),
-      new Tuple2<>("Oranges", "Fruit"),
-      new Tuple2<>("Oranges", "Citrus")
-      ));
-    JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Oranges", 2),
-      new Tuple2<>("Apples", 3)
-    ));
-    JavaPairRDD<String, Integer> quantities = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Oranges", 21),
-      new Tuple2<>("Apples", 42)
-    ));
-    JavaPairRDD<String, String> countries = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>("Oranges", "BR"),
-      new Tuple2<>("Apples", "US")
-    ));
-
-    JavaPairRDD<String, Tuple4<Iterable<String>, Iterable<Integer>, Iterable<Integer>,
-        Iterable<String>>> cogrouped = categories.cogroup(prices, quantities, countries);
-    assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1()));
-    assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2()));
-    assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3()));
-    assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4()));
-
-    cogrouped.collect();
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void leftOuterJoin() {
-    JavaPairRDD<Integer, Integer> rdd1 = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>(1, 1),
-      new Tuple2<>(1, 2),
-      new Tuple2<>(2, 1),
-      new Tuple2<>(3, 1)
-      ));
-    JavaPairRDD<Integer, Character> rdd2 = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>(1, 'x'),
-      new Tuple2<>(2, 'y'),
-      new Tuple2<>(2, 'z'),
-      new Tuple2<>(4, 'w')
-    ));
-    List<Tuple2<Integer,Tuple2<Integer,Optional<Character>>>> joined =
-      rdd1.leftOuterJoin(rdd2).collect();
-    assertEquals(5, joined.size());
-    Tuple2<Integer,Tuple2<Integer,Optional<Character>>> firstUnmatched =
-      rdd1.leftOuterJoin(rdd2).filter(
-        new Function<Tuple2<Integer, Tuple2<Integer, Optional<Character>>>, Boolean>() {
-          @Override
-          public Boolean call(Tuple2<Integer, Tuple2<Integer, Optional<Character>>> tup) {
-            return !tup._2()._2().isPresent();
-          }
-      }).first();
-    assertEquals(3, firstUnmatched._1().intValue());
-  }
-
-  @Test
-  public void foldReduce() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() {
-      @Override
-      public Integer call(Integer a, Integer b) {
-        return a + b;
-      }
-    };
-
-    int sum = rdd.fold(0, add);
-    assertEquals(33, sum);
-
-    sum = rdd.reduce(add);
-    assertEquals(33, sum);
-  }
-
-  @Test
-  public void treeReduce() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10);
-    Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() {
-      @Override
-      public Integer call(Integer a, Integer b) {
-        return a + b;
-      }
-    };
-    for (int depth = 1; depth <= 10; depth++) {
-      int sum = rdd.treeReduce(add, depth);
-      assertEquals(-5, sum);
-    }
-  }
-
-  @Test
-  public void treeAggregate() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10);
-    Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() {
-      @Override
-      public Integer call(Integer a, Integer b) {
-        return a + b;
-      }
-    };
-    for (int depth = 1; depth <= 10; depth++) {
-      int sum = rdd.treeAggregate(0, add, add, depth);
-      assertEquals(-5, sum);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void aggregateByKey() {
-    JavaPairRDD<Integer, Integer> pairs = sc.parallelizePairs(
-      Arrays.asList(
-        new Tuple2<>(1, 1),
-        new Tuple2<>(1, 1),
-        new Tuple2<>(3, 2),
-        new Tuple2<>(5, 1),
-        new Tuple2<>(5, 3)), 2);
-
-    Map<Integer, Set<Integer>> sets = pairs.aggregateByKey(new HashSet<Integer>(),
-      new Function2<Set<Integer>, Integer, Set<Integer>>() {
-        @Override
-        public Set<Integer> call(Set<Integer> a, Integer b) {
-          a.add(b);
-          return a;
-        }
-      },
-      new Function2<Set<Integer>, Set<Integer>, Set<Integer>>() {
-        @Override
-        public Set<Integer> call(Set<Integer> a, Set<Integer> b) {
-          a.addAll(b);
-          return a;
-        }
-      }).collectAsMap();
-    assertEquals(3, sets.size());
-    assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1));
-    assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3));
-    assertEquals(new HashSet<>(Arrays.asList(1, 3)), sets.get(5));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void foldByKey() {
-    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
-      new Tuple2<>(2, 1),
-      new Tuple2<>(2, 1),
-      new Tuple2<>(1, 1),
-      new Tuple2<>(3, 2),
-      new Tuple2<>(3, 1)
-    );
-    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0,
-      new Function2<Integer, Integer, Integer>() {
-        @Override
-        public Integer call(Integer a, Integer b) {
-          return a + b;
-        }
-    });
-    assertEquals(1, sums.lookup(1).get(0).intValue());
-    assertEquals(2, sums.lookup(2).get(0).intValue());
-    assertEquals(3, sums.lookup(3).get(0).intValue());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void reduceByKey() {
-    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
-      new Tuple2<>(2, 1),
-      new Tuple2<>(2, 1),
-      new Tuple2<>(1, 1),
-      new Tuple2<>(3, 2),
-      new Tuple2<>(3, 1)
-    );
-    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> counts = rdd.reduceByKey(
-      new Function2<Integer, Integer, Integer>() {
-        @Override
-        public Integer call(Integer a, Integer b) {
-         return a + b;
-        }
-    });
-    assertEquals(1, counts.lookup(1).get(0).intValue());
-    assertEquals(2, counts.lookup(2).get(0).intValue());
-    assertEquals(3, counts.lookup(3).get(0).intValue());
-
-    Map<Integer, Integer> localCounts = counts.collectAsMap();
-    assertEquals(1, localCounts.get(1).intValue());
-    assertEquals(2, localCounts.get(2).intValue());
-    assertEquals(3, localCounts.get(3).intValue());
-
-    localCounts = rdd.reduceByKeyLocally(new Function2<Integer, Integer, Integer>() {
-      @Override
-      public Integer call(Integer a, Integer b) {
-        return a + b;
-      }
-    });
-    assertEquals(1, localCounts.get(1).intValue());
-    assertEquals(2, localCounts.get(2).intValue());
-    assertEquals(3, localCounts.get(3).intValue());
-  }
-
-  @Test
-  public void approximateResults() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    Map<Integer, Long> countsByValue = rdd.countByValue();
-    assertEquals(2, countsByValue.get(1).longValue());
-    assertEquals(1, countsByValue.get(13).longValue());
-
-    PartialResult<Map<Integer, BoundedDouble>> approx = rdd.countByValueApprox(1);
-    Map<Integer, BoundedDouble> finalValue = approx.getFinalValue();
-    assertEquals(2.0, finalValue.get(1).mean(), 0.01);
-    assertEquals(1.0, finalValue.get(13).mean(), 0.01);
-  }
-
-  @Test
-  public void take() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    assertEquals(1, rdd.first().intValue());
-    rdd.take(2);
-    rdd.takeSample(false, 2, 42);
-  }
-
-  @Test
-  public void isEmpty() {
-    assertTrue(sc.emptyRDD().isEmpty());
-    assertTrue(sc.parallelize(new ArrayList<Integer>()).isEmpty());
-    assertFalse(sc.parallelize(Arrays.asList(1)).isEmpty());
-    assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter(
-        new Function<Integer,Boolean>() {
-          @Override
-          public Boolean call(Integer i) {
-            return i < 0;
-          }
-        }).isEmpty());
-    assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter(
-        new Function<Integer, Boolean>() {
-          @Override
-          public Boolean call(Integer i) {
-            return i > 1;
-          }
-        }).isEmpty());
-  }
-
-  @Test
-  public void cartesian() {
-    JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
-    JavaRDD<String> stringRDD = sc.parallelize(Arrays.asList("Hello", "World"));
-    JavaPairRDD<String, Double> cartesian = stringRDD.cartesian(doubleRDD);
-    assertEquals(new Tuple2<>("Hello", 1.0), cartesian.first());
-  }
-
-  @Test
-  public void javaDoubleRDD() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
-    JavaDoubleRDD distinct = rdd.distinct();
-    assertEquals(5, distinct.count());
-    JavaDoubleRDD filter = rdd.filter(new Function<Double, Boolean>() {
-      @Override
-      public Boolean call(Double x) {
-        return x > 2.0;
-      }
-    });
-    assertEquals(3, filter.count());
-    JavaDoubleRDD union = rdd.union(rdd);
-    assertEquals(12, union.count());
-    union = union.cache();
-    assertEquals(12, union.count());
-
-    assertEquals(20, rdd.sum(), 0.01);
-    StatCounter stats = rdd.stats();
-    assertEquals(20, stats.sum(), 0.01);
-    assertEquals(20/6.0, rdd.mean(), 0.01);
-    assertEquals(20/6.0, rdd.mean(), 0.01);
-    assertEquals(6.22222, rdd.variance(), 0.01);
-    assertEquals(rdd.variance(), rdd.popVariance(), 1e-14);
-    assertEquals(7.46667, rdd.sampleVariance(), 0.01);
-    assertEquals(2.49444, rdd.stdev(), 0.01);
-    assertEquals(rdd.stdev(), rdd.popStdev(), 1e-14);
-    assertEquals(2.73252, rdd.sampleStdev(), 0.01);
-
-    rdd.first();
-    rdd.take(5);
-  }
-
-  @Test
-  public void javaDoubleRDDHistoGram() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
-    // Test using generated buckets
-    Tuple2<double[], long[]> results = rdd.histogram(2);
-    double[] expected_buckets = {1.0, 2.5, 4.0};
-    long[] expected_counts = {2, 2};
-    assertArrayEquals(expected_buckets, results._1(), 0.1);
-    assertArrayEquals(expected_counts, results._2());
-    // Test with provided buckets
-    long[] histogram = rdd.histogram(expected_buckets);
-    assertArrayEquals(expected_counts, histogram);
-    // SPARK-5744
-    assertArrayEquals(
-        new long[] {0},
-        sc.parallelizeDoubles(new ArrayList<Double>(0), 1).histogram(new double[]{0.0, 1.0}));
-  }
-
-  private static class DoubleComparator implements Comparator<Double>, Serializable {
-    @Override
-    public int compare(Double o1, Double o2) {
-      return o1.compareTo(o2);
-    }
-  }
-
-  @Test
-  public void max() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
-    double max = rdd.max(new DoubleComparator());
-    assertEquals(4.0, max, 0.001);
-  }
-
-  @Test
-  public void min() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
-    double max = rdd.min(new DoubleComparator());
-    assertEquals(1.0, max, 0.001);
-  }
-
-  @Test
-  public void naturalMax() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
-    double max = rdd.max();
-    assertEquals(4.0, max, 0.0);
-  }
-
-  @Test
-  public void naturalMin() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
-    double max = rdd.min();
-    assertEquals(1.0, max, 0.0);
-  }
-
-  @Test
-  public void takeOrdered() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
-    assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator()));
-    assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2));
-  }
-
-  @Test
-  public void top() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    List<Integer> top2 = rdd.top(2);
-    assertEquals(Arrays.asList(4, 3), top2);
-  }
-
-  private static class AddInts implements Function2<Integer, Integer, Integer> {
-    @Override
-    public Integer call(Integer a, Integer b) {
-      return a + b;
-    }
-  }
-
-  @Test
-  public void reduce() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    int sum = rdd.reduce(new AddInts());
-    assertEquals(10, sum);
-  }
-
-  @Test
-  public void reduceOnJavaDoubleRDD() {
-    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
-    double sum = rdd.reduce(new Function2<Double, Double, Double>() {
-      @Override
-      public Double call(Double v1, Double v2) {
-        return v1 + v2;
-      }
-    });
-    assertEquals(10.0, sum, 0.001);
-  }
-
-  @Test
-  public void fold() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    int sum = rdd.fold(0, new AddInts());
-    assertEquals(10, sum);
-  }
-
-  @Test
-  public void aggregate() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    int sum = rdd.aggregate(0, new AddInts(), new AddInts());
-    assertEquals(10, sum);
-  }
-
-  @Test
-  public void map() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-    JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction<Integer>() {
-      @Override
-      public double call(Integer x) {
-        return x.doubleValue();
-      }
-    }).cache();
-    doubles.collect();
-    JavaPairRDD<Integer, Integer> pairs = rdd.mapToPair(
-        new PairFunction<Integer, Integer, Integer>() {
-          @Override
-          public Tuple2<Integer, Integer> call(Integer x) {
-            return new Tuple2<>(x, x);
-          }
-        }).cache();
-    pairs.collect();
-    JavaRDD<String> strings = rdd.map(new Function<Integer, String>() {
-      @Override
-      public String call(Integer x) {
-        return x.toString();
-      }
-    }).cache();
-    strings.collect();
-  }
-
-  @Test
-  public void flatMap() {
-    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello World!",
-      "The quick brown fox jumps over the lazy dog."));
-    JavaRDD<String> words = rdd.flatMap(new FlatMapFunction<String, String>() {
-      @Override
-      public Iterator<String> call(String x) {
-        return Arrays.asList(x.split(" ")).iterator();
-      }
-    });
-    assertEquals("Hello", words.first());
-    assertEquals(11, words.count());
-
-    JavaPairRDD<String, String> pairsRDD = rdd.flatMapToPair(
-      new PairFlatMapFunction<String, String, String>() {
-        @Override
-        public Iterator<Tuple2<String, String>> call(String s) {
-          List<Tuple2<String, String>> pairs = new LinkedList<>();
-          for (String word : s.split(" ")) {
-            pairs.add(new Tuple2<>(word, word));
-          }
-          return pairs.iterator();
-        }
-      }
-    );
-    assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first());
-    assertEquals(11, pairsRDD.count());
-
-    JavaDoubleRDD doubles = rdd.flatMapToDouble(new DoubleFlatMapFunction<String>() {
-      @Override
-      public Iterator<Double> call(String s) {
-        List<Double> lengths = new LinkedList<>();
-        for (String word : s.split(" ")) {
-          lengths.add((double) word.length());
-        }
-        return lengths.iterator();
-      }
-    });
-    assertEquals(5.0, doubles.first(), 0.01);
-    assertEquals(11, pairsRDD.count());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void mapsFromPairsToPairs() {
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
-
-    // Regression test for SPARK-668:
-    JavaPairRDD<String, Integer> swapped = pairRDD.flatMapToPair(
-      new PairFlatMapFunction<Tuple2<Integer, String>, String, Integer>() {
-        @Override
-        public Iterator<Tuple2<String, Integer>> call(Tuple2<Integer, String> item) {
-          return Collections.singletonList(item.swap()).iterator();
-        }
-      });
-    swapped.collect();
-
-    // There was never a bug here, but it's worth testing:
-    pairRDD.mapToPair(new PairFunction<Tuple2<Integer, String>, String, Integer>() {
-      @Override
-      public Tuple2<String, Integer> call(Tuple2<Integer, String> item) {
-        return item.swap();
-      }
-    }).collect();
-  }
-
-  @Test
-  public void mapPartitions() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
-    JavaRDD<Integer> partitionSums = rdd.mapPartitions(
-      new FlatMapFunction<Iterator<Integer>, Integer>() {
-        @Override
-        public Iterator<Integer> call(Iterator<Integer> iter) {
-          int sum = 0;
-          while (iter.hasNext()) {
-            sum += iter.next();
-          }
-          return Collections.singletonList(sum).iterator();
-        }
-    });
-    assertEquals("[3, 7]", partitionSums.collect().toString());
-  }
-
-
-  @Test
-  public void mapPartitionsWithIndex() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
-    JavaRDD<Integer> partitionSums = rdd.mapPartitionsWithIndex(
-      new Function2<Integer, Iterator<Integer>, Iterator<Integer>>() {
-        @Override
-        public Iterator<Integer> call(Integer index, Iterator<Integer> iter) {
-          int sum = 0;
-          while (iter.hasNext()) {
-            sum += iter.next();
-          }
-          return Collections.singletonList(sum).iterator();
-        }
-    }, false);
-    assertEquals("[3, 7]", partitionSums.collect().toString());
-  }
-
-  @Test
-  public void getNumPartitions(){
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3);
-    JavaDoubleRDD rdd2 = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0), 2);
-    JavaPairRDD<String, Integer> rdd3 = sc.parallelizePairs(Arrays.asList(
-            new Tuple2<>("a", 1),
-            new Tuple2<>("aa", 2),
-            new Tuple2<>("aaa", 3)
-    ), 2);
-    assertEquals(3, rdd1.getNumPartitions());
-    assertEquals(2, rdd2.getNumPartitions());
-    assertEquals(2, rdd3.getNumPartitions());
-  }
-
-  @Test
-  public void repartition() {
-    // Shrinking number of partitions
-    JavaRDD<Integer> in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 2);
-    JavaRDD<Integer> repartitioned1 = in1.repartition(4);
-    List<List<Integer>> result1 = repartitioned1.glom().collect();
-    assertEquals(4, result1.size());
-    for (List<Integer> l : result1) {
-      assertFalse(l.isEmpty());
-    }
-
-    // Growing number of partitions
-    JavaRDD<Integer> in2 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 4);
-    JavaRDD<Integer> repartitioned2 = in2.repartition(2);
-    List<List<Integer>> result2 = repartitioned2.glom().collect();
-    assertEquals(2, result2.size());
-    for (List<Integer> l: result2) {
-      assertFalse(l.isEmpty());
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void persist() {
-    JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
-    doubleRDD = doubleRDD.persist(StorageLevel.DISK_ONLY());
-    assertEquals(20, doubleRDD.sum(), 0.1);
-
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
-    pairRDD = pairRDD.persist(StorageLevel.DISK_ONLY());
-    assertEquals("a", pairRDD.first()._2());
-
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-    rdd = rdd.persist(StorageLevel.DISK_ONLY());
-    assertEquals(1, rdd.first().intValue());
-  }
-
-  @Test
-  public void iterator() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
-    TaskContext context = TaskContext$.MODULE$.empty();
-    assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue());
-  }
-
-  @Test
-  public void glom() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
-    assertEquals("[1, 2]", rdd.glom().first().toString());
-  }
-
-  // File input / output tests are largely adapted from FileSuite:
-
-  @Test
-  public void textFiles() throws IOException {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    rdd.saveAsTextFile(outputDir);
-    // Read the plain text file and check it's OK
-    File outputFile = new File(outputDir, "part-00000");
-    String content = Files.toString(outputFile, StandardCharsets.UTF_8);
-    assertEquals("1\n2\n3\n4\n", content);
-    // Also try reading it in as a text file RDD
-    List<String> expected = Arrays.asList("1", "2", "3", "4");
-    JavaRDD<String> readRDD = sc.textFile(outputDir);
-    assertEquals(expected, readRDD.collect());
-  }
-
-  @Test
-  public void wholeTextFiles() throws Exception {
-    byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8);
-    byte[] content2 = "spark is also easy to use.\n".getBytes(StandardCharsets.UTF_8);
-
-    String tempDirName = tempDir.getAbsolutePath();
-    String path1 = new Path(tempDirName, "part-00000").toUri().getPath();
-    String path2 = new Path(tempDirName, "part-00001").toUri().getPath();
-
-    Files.write(content1, new File(path1));
-    Files.write(content2, new File(path2));
-
-    Map<String, String> container = new HashMap<>();
-    container.put(path1, new Text(content1).toString());
-    container.put(path2, new Text(content2).toString());
-
-    JavaPairRDD<String, String> readRDD = sc.wholeTextFiles(tempDirName, 3);
-    List<Tuple2<String, String>> result = readRDD.collect();
-
-    for (Tuple2<String, String> res : result) {
-      // Note that the paths from `wholeTextFiles` are in URI format on Windows,
-      // for example, file:/C:/a/b/c.
-      assertEquals(res._2(), container.get(new Path(res._1()).toUri().getPath()));
-    }
-  }
-
-  @Test
-  public void textFilesCompressed() throws IOException {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    rdd.saveAsTextFile(outputDir, DefaultCodec.class);
-
-    // Try reading it in as a text file RDD
-    List<String> expected = Arrays.asList("1", "2", "3", "4");
-    JavaRDD<String> readRDD = sc.textFile(outputDir);
-    assertEquals(expected, readRDD.collect());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void sequenceFile() {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
-
-    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
-      @Override
-      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
-        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
-      }
-    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
-
-    // Try reading the output back as an object file
-    JavaPairRDD<Integer, String> readRDD = sc.sequenceFile(outputDir, IntWritable.class,
-      Text.class).mapToPair(new PairFunction<Tuple2<IntWritable, Text>, Integer, String>() {
-      @Override
-      public Tuple2<Integer, String> call(Tuple2<IntWritable, Text> pair) {
-        return new Tuple2<>(pair._1().get(), pair._2().toString());
-      }
-    });
-    assertEquals(pairs, readRDD.collect());
-  }
-
-  @Test
-  public void binaryFiles() throws Exception {
-    // Reusing the wholeText files example
-    byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8);
-
-    String tempDirName = tempDir.getAbsolutePath();
-    File file1 = new File(tempDirName + "/part-00000");
-
-    FileOutputStream fos1 = new FileOutputStream(file1);
-
-    FileChannel channel1 = fos1.getChannel();
-    ByteBuffer bbuf = ByteBuffer.wrap(content1);
-    channel1.write(bbuf);
-    channel1.close();
-    JavaPairRDD<String, PortableDataStream> readRDD = sc.binaryFiles(tempDirName, 3);
-    List<Tuple2<String, PortableDataStream>> result = readRDD.collect();
-    for (Tuple2<String, PortableDataStream> res : result) {
-      assertArrayEquals(content1, res._2().toArray());
-    }
-  }
-
-  @Test
-  public void binaryFilesCaching() throws Exception {
-    // Reusing the wholeText files example
-    byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8);
-
-    String tempDirName = tempDir.getAbsolutePath();
-    File file1 = new File(tempDirName + "/part-00000");
-
-    FileOutputStream fos1 = new FileOutputStream(file1);
-
-    FileChannel channel1 = fos1.getChannel();
-    ByteBuffer bbuf = ByteBuffer.wrap(content1);
-    channel1.write(bbuf);
-    channel1.close();
-
-    JavaPairRDD<String, PortableDataStream> readRDD = sc.binaryFiles(tempDirName).cache();
-    readRDD.foreach(new VoidFunction<Tuple2<String,PortableDataStream>>() {
-      @Override
-      public void call(Tuple2<String, PortableDataStream> pair) {
-        pair._2().toArray(); // force the file to read
-      }
-    });
-
-    List<Tuple2<String, PortableDataStream>> result = readRDD.collect();
-    for (Tuple2<String, PortableDataStream> res : result) {
-      assertArrayEquals(content1, res._2().toArray());
-    }
-  }
-
-  @Test
-  public void binaryRecords() throws Exception {
-    // Reusing the wholeText files example
-    byte[] content1 = "spark isn't always easy to use.\n".getBytes(StandardCharsets.UTF_8);
-    int numOfCopies = 10;
-    String tempDirName = tempDir.getAbsolutePath();
-    File file1 = new File(tempDirName + "/part-00000");
-
-    FileOutputStream fos1 = new FileOutputStream(file1);
-
-    FileChannel channel1 = fos1.getChannel();
-
-    for (int i = 0; i < numOfCopies; i++) {
-      ByteBuffer bbuf = ByteBuffer.wrap(content1);
-      channel1.write(bbuf);
-    }
-    channel1.close();
-
-    JavaRDD<byte[]> readRDD = sc.binaryRecords(tempDirName, content1.length);
-    assertEquals(numOfCopies,readRDD.count());
-    List<byte[]> result = readRDD.collect();
-    for (byte[] res : result) {
-      assertArrayEquals(content1, res);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void writeWithNewAPIHadoopFile() {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
-
-    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
-      @Override
-      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
-        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
-      }
-    }).saveAsNewAPIHadoopFile(
-        outputDir, IntWritable.class, Text.class,
-        org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class);
-
-    JavaPairRDD<IntWritable, Text> output =
-        sc.sequenceFile(outputDir, IntWritable.class, Text.class);
-    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
-      @Override
-      public String call(Tuple2<IntWritable, Text> x) {
-        return x.toString();
-      }
-    }).collect().toString());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void readWithNewAPIHadoopFile() throws IOException {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
-
-    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
-      @Override
-      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
-        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
-      }
-    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
-
-    JavaPairRDD<IntWritable, Text> output = sc.newAPIHadoopFile(outputDir,
-        org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class,
-        IntWritable.class, Text.class, Job.getInstance().getConfiguration());
-    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
-      @Override
-      public String call(Tuple2<IntWritable, Text> x) {
-        return x.toString();
-      }
-    }).collect().toString());
-  }
-
-  @Test
-  public void objectFilesOfInts() {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    rdd.saveAsObjectFile(outputDir);
-    // Try reading the output back as an object file
-    List<Integer> expected = Arrays.asList(1, 2, 3, 4);
-    JavaRDD<Integer> readRDD = sc.objectFile(outputDir);
-    assertEquals(expected, readRDD.collect());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void objectFilesOfComplexTypes() {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
-    rdd.saveAsObjectFile(outputDir);
-    // Try reading the output back as an object file
-    JavaRDD<Tuple2<Integer, String>> readRDD = sc.objectFile(outputDir);
-    assertEquals(pairs, readRDD.collect());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void hadoopFile() {
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
-
-    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
-      @Override
-      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
-        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
-      }
-    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
-
-    JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir,
-        SequenceFileInputFormat.class, IntWritable.class, Text.class);
-    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
-      @Override
-      public String call(Tuple2<IntWritable, Text> x) {
-        return x.toString();
-      }
-    }).collect().toString());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void hadoopFileCompressed() {
-    String outputDir = new File(tempDir, "output_compressed").getAbsolutePath();
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
-
-    rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
-      @Override
-      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
-        return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()));
-      }
-    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class,
-        DefaultCodec.class);
-
-    JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir,
-        SequenceFileInputFormat.class, IntWritable.class, Text.class);
-
-    assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() {
-      @Override
-      public String call(Tuple2<IntWritable, Text> x) {
-        return x.toString();
-      }
-    }).collect().toString());
-  }
-
-  @Test
-  public void zip() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-    JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction<Integer>() {
-      @Override
-      public double call(Integer x) {
-        return x.doubleValue();
-      }
-    });
-    JavaPairRDD<Integer, Double> zipped = rdd.zip(doubles);
-    zipped.count();
-  }
-
-  @Test
-  public void zipPartitions() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2);
-    JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2);
-    FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer> sizesFn =
-      new FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer>() {
-        @Override
-        public Iterator<Integer> call(Iterator<Integer> i, Iterator<String> s) {
-          return Arrays.asList(Iterators.size(i), Iterators.size(s)).iterator();
-        }
-      };
-
-    JavaRDD<Integer> sizes = rdd1.zipPartitions(rdd2, sizesFn);
-    assertEquals("[3, 2, 3, 2]", sizes.collect().toString());
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void accumulators() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-
-    final Accumulator<Integer> intAccum = sc.intAccumulator(10);
-    rdd.foreach(new VoidFunction<Integer>() {
-      @Override
-      public void call(Integer x) {
-        intAccum.add(x);
-      }
-    });
-    assertEquals((Integer) 25, intAccum.value());
-
-    final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0);
-    rdd.foreach(new VoidFunction<Integer>() {
-      @Override
-      public void call(Integer x) {
-        doubleAccum.add((double) x);
-      }
-    });
-    assertEquals((Double) 25.0, doubleAccum.value());
-
-    // Try a custom accumulator type
-    AccumulatorParam<Float> floatAccumulatorParam = new AccumulatorParam<Float>() {
-      @Override
-      public Float addInPlace(Float r, Float t) {
-        return r + t;
-      }
-
-      @Override
-      public Float addAccumulator(Float r, Float t) {
-        return r + t;
-      }
-
-      @Override
-      public Float zero(Float initialValue) {
-        return 0.0f;
-      }
-    };
-
-    final Accumulator<Float> floatAccum = sc.accumulator(10.0f, floatAccumulatorParam);
-    rdd.foreach(new VoidFunction<Integer>() {
-      @Override
-      public void call(Integer x) {
-        floatAccum.add((float) x);
-      }
-    });
-    assertEquals((Float) 25.0f, floatAccum.value());
-
-    // Test the setValue method
-    floatAccum.setValue(5.0f);
-    assertEquals((Float) 5.0f, floatAccum.value());
-  }
-
-  @Test
-  public void keyBy() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2));
-    List<Tuple2<String, Integer>> s = rdd.keyBy(new Function<Integer, String>() {
-      @Override
-      public String call(Integer t) {
-        return t.toString();
-      }
-    }).collect();
-    assertEquals(new Tuple2<>("1", 1), s.get(0));
-    assertEquals(new Tuple2<>("2", 2), s.get(1));
-  }
-
-  @Test
-  public void checkpointAndComputation() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-    sc.setCheckpointDir(tempDir.getAbsolutePath());
-    assertFalse(rdd.isCheckpointed());
-    rdd.checkpoint();
-    rdd.count(); // Forces the DAG to cause a checkpoint
-    assertTrue(rdd.isCheckpointed());
-    assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect());
-  }
-
-  @Test
-  public void checkpointAndRestore() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-    sc.setCheckpointDir(tempDir.getAbsolutePath());
-    assertFalse(rdd.isCheckpointed());
-    rdd.checkpoint();
-    rdd.count(); // Forces the DAG to cause a checkpoint
-    assertTrue(rdd.isCheckpointed());
-
-    assertTrue(rdd.getCheckpointFile().isPresent());
-    JavaRDD<Integer> recovered = sc.checkpointFile(rdd.getCheckpointFile().get());
-    assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect());
-  }
-
-  @Test
-  public void combineByKey() {
-    JavaRDD<Integer> originalRDD = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6));
-    Function<Integer, Integer> keyFunction = new Function<Integer, Integer>() {
-      @Override
-      public Integer call(Integer v1) {
-        return v1 % 3;
-      }
-    };
-    Function<Integer, Integer> createCombinerFunction = new Function<Integer, Integer>() {
-      @Override
-      public Integer call(Integer v1) {
-        return v1;
-      }
-    };
-
-    Function2<Integer, Integer, Integer> mergeValueFunction =
-        new Function2<Integer, Integer, Integer>() {
-      @Override
-      public Integer call(Integer v1, Integer v2) {
-        return v1 + v2;
-      }
-    };
-
-    JavaPairRDD<Integer, Integer> combinedRDD = originalRDD.keyBy(keyFunction)
-        .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction);
-    Map<Integer, Integer> results = combinedRDD.collectAsMap();
-    ImmutableMap<Integer, Integer> expected = ImmutableMap.of(0, 9, 1, 5, 2, 7);
-    assertEquals(expected, results);
-
-    Partitioner defaultPartitioner = Partitioner.defaultPartitioner(
-        combinedRDD.rdd(),
-        JavaConverters.collectionAsScalaIterableConverter(
-            Collections.<RDD<?>>emptyList()).asScala().toSeq());
-    combinedRDD = originalRDD.keyBy(keyFunction)
-        .combineByKey(
-             createCombinerFunction,
-             mergeValueFunction,
-             mergeValueFunction,
-             defaultPartitioner,
-             false,
-             new KryoSerializer(new SparkConf()));
-    results = combinedRDD.collectAsMap();
-    assertEquals(expected, results);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void mapOnPairRDD() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1,2,3,4));
-    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
-        new PairFunction<Integer, Integer, Integer>() {
-          @Override
-          public Tuple2<Integer, Integer> call(Integer i) {
-            return new Tuple2<>(i, i % 2);
-          }
-        });
-    JavaPairRDD<Integer, Integer> rdd3 = rdd2.mapToPair(
-        new PairFunction<Tuple2<Integer, Integer>, Integer, Integer>() {
-          @Override
-          public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> in) {
-            return new Tuple2<>(in._2(), in._1());
-          }
-        });
-    assertEquals(Arrays.asList(
-        new Tuple2<>(1, 1),
-        new Tuple2<>(0, 2),
-        new Tuple2<>(1, 3),
-        new Tuple2<>(0, 4)), rdd3.collect());
-
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void collectPartitions() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3);
-
-    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
-        new PairFunction<Integer, Integer, Integer>() {
-          @Override
-          public Tuple2<Integer, Integer> call(Integer i) {
-            return new Tuple2<>(i, i % 2);
-          }
-        });
-
-    List<Integer>[] parts = rdd1.collectPartitions(new int[] {0});
-    assertEquals(Arrays.asList(1, 2), parts[0]);
-
-    parts = rdd1.collectPartitions(new int[] {1, 2});
-    assertEquals(Arrays.asList(3, 4), parts[0]);
-    assertEquals(Arrays.asList(5, 6, 7), parts[1]);
-
-    assertEquals(Arrays.asList(new Tuple2<>(1, 1),
-                                      new Tuple2<>(2, 0)),
-                        rdd2.collectPartitions(new int[] {0})[0]);
-
-    List<Tuple2<Integer,Integer>>[] parts2 = rdd2.collectPartitions(new int[] {1, 2});
-    assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]);
-    assertEquals(Arrays.asList(new Tuple2<>(5, 1),
-                                      new Tuple2<>(6, 0),
-                                      new Tuple2<>(7, 1)),
-                        parts2[1]);
-  }
-
-  @Test
-  public void countApproxDistinct() {
-    List<Integer> arrayData = new ArrayList<>();
-    int size = 100;
-    for (int i = 0; i < 100000; i++) {
-      arrayData.add(i % size);
-    }
-    JavaRDD<Integer> simpleRdd = sc.parallelize(arrayData, 10);
-    assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1);
-  }
-
-  @Test
-  public void countApproxDistinctByKey() {
-    List<Tuple2<Integer, Integer>> arrayData = new ArrayList<>();
-    for (int i = 10; i < 100; i++) {
-      for (int j = 0; j < i; j++) {
-        arrayData.add(new Tuple2<>(i, j));
-      }
-    }
-    double relativeSD = 0.001;
-    JavaPairRDD<Integer, Integer> pairRdd = sc.parallelizePairs(arrayData);
-    List<Tuple2<Integer, Long>> res =  pairRdd.countApproxDistinctByKey(relativeSD, 8).collect();
-    for (Tuple2<Integer, Long> resItem : res) {
-      double count = resItem._1();
-      long resCount = resItem._2();
-      double error = Math.abs((resCount - count) / count);
-      assertTrue(error < 0.1);
-    }
-
-  }
-
-  @Test
-  public void collectAsMapWithIntArrayValues() {
-    // Regression test for SPARK-1040
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1));
-    JavaPairRDD<Integer, int[]> pairRDD = rdd.mapToPair(
-        new PairFunction<Integer, Integer, int[]>() {
-          @Override
-          public Tuple2<Integer, int[]> call(Integer x) {
-            return new Tuple2<>(x, new int[]{x});
-          }
-        });
-    pairRDD.collect();  // Works fine
-    pairRDD.collectAsMap();  // Used to crash with ClassCastException
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void collectAsMapAndSerialize() throws Exception {
-    JavaPairRDD<String,Integer> rdd =
-        sc.parallelizePairs(Arrays.asList(new Tuple2<>("foo", 1)));
-    Map<String,Integer> map = rdd.collectAsMap();
-    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
-    new ObjectOutputStream(bytes).writeObject(map);
-    Map<String,Integer> deserializedMap = (Map<String,Integer>)
-        new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject();
-    assertEquals(1, deserializedMap.get("foo").intValue());
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void sampleByKey() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3);
-    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
-      new PairFunction<Integer, Integer, Integer>() {
-        @Override
-        public Tuple2<Integer, Integer> call(Integer i) {
-          return new Tuple2<>(i % 2, 1);
-        }
-      });
-    Map<Integer, Double> fractions = new HashMap<>();
-    fractions.put(0, 0.5);
-    fractions.put(1, 1.0);
-    JavaPairRDD<Integer, Integer> wr = rdd2.sampleByKey(true, fractions, 1L);
-    Map<Integer, Long> wrCounts = wr.countByKey();
-    assertEquals(2, wrCounts.size());
-    assertTrue(wrCounts.get(0) > 0);
-    assertTrue(wrCounts.get(1) > 0);
-    JavaPairRDD<Integer, Integer> wor = rdd2.sampleByKey(false, fractions, 1L);
-    Map<Integer, Long> worCounts = wor.countByKey();
-    assertEquals(2, worCounts.size());
-    assertTrue(worCounts.get(0) > 0);
-    assertTrue(worCounts.get(1) > 0);
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void sampleByKeyExact() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3);
-    JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(
-      new PairFunction<Integer, Integer, Integer>() {
-          @Override
-          public Tuple2<Integer, Integer> call(Integer i) {
-              return new Tuple2<>(i % 2, 1);
-          }
-      });
-    Map<Integer, Double> fractions = new HashMap<>();
-    fractions.put(0, 0.5);
-    fractions.put(1, 1.0);
-    JavaPairRDD<Integer, Integer> wrExact = rdd2.sampleByKeyExact(true, fractions, 1L);
-    Map<Integer, Long> wrExactCounts = wrExact.countByKey();
-    assertEquals(2, wrExactCounts.size());
-    assertTrue(wrExactCounts.get(0) == 2);
-    assertTrue(wrExactCounts.get(1) == 4);
-    JavaPairRDD<Integer, Integer> worExact = rdd2.sampleByKeyExact(false, fractions, 1L);
-    Map<Integer, Long> worExactCounts = worExact.countByKey();
-    assertEquals(2, worExactCounts.size());
-    assertTrue(worExactCounts.get(0) == 2);
-    assertTrue(worExactCounts.get(1) == 4);
-  }
-
-  private static class SomeCustomClass implements Serializable {
-    SomeCustomClass() {
-      // Intentionally left blank
-    }
-  }
-
-  @Test
-  public void collectUnderlyingScalaRDD() {
-    List<SomeCustomClass> data = new ArrayList<>();
-    for (int i = 0; i < 100; i++) {
-      data.add(new SomeCustomClass());
-    }
-    JavaRDD<SomeCustomClass> rdd = sc.parallelize(data);
-    SomeCustomClass[] collected =
-      (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect();
-    assertEquals(data.size(), collected.length);
-  }
-
-  private static final class BuggyMapFunction<T> implements Function<T, T> {
-
-    @Override
-    public T call(T x) {
-      throw new IllegalStateException("Custom exception!");
-    }
-  }
-
-  @Test
-  public void collectAsync() throws Exception {
-    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
-    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
-    JavaFutureAction<List<Integer>> future = rdd.collectAsync();
-    List<Integer> result = future.get();
-    assertEquals(data, result);
-    assertFalse(future.isCancelled());
-    assertTrue(future.isDone());
-    assertEquals(1, future.jobIds().size());
-  }
-
-  @Test
-  public void takeAsync() throws Exception {
-    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
-    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
-    JavaFutureAction<List<Integer>> future = rdd.takeAsync(1);
-    List<Integer> result = future.get();
-    assertEquals(1, result.size());
-    assertEquals((Integer) 1, result.get(0));
-    assertFalse(future.isCancelled());
-    assertTrue(future.isDone());
-    assertEquals(1, future.jobIds().size());
-  }
-
-  @Test
-  public void foreachAsync() throws Exception {
-    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
-    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
-    JavaFutureAction<Void> future = rdd.foreachAsync(
-        new VoidFunction<Integer>() {
-          @Override
-          public void call(Integer integer) {
-            // intentionally left blank.
-          }
-        }
-    );
-    future.get();
-    assertFalse(future.isCancelled());
-    assertTrue(future.isDone());
-    assertEquals(1, future.jobIds().size());
-  }
-
-  @Test
-  public void countAsync() throws Exception {
-    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
-    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
-    JavaFutureAction<Long> future = rdd.countAsync();
-    long count = future.get();
-    assertEquals(data.size(), count);
-    assertFalse(future.isCancelled());
-    assertTrue(future.isDone());
-    assertEquals(1, future.jobIds().size());
-  }
-
-  @Test
-  public void testAsyncActionCancellation() throws Exception {
-    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
-    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
-    JavaFutureAction<Void> future = rdd.foreachAsync(new VoidFunction<Integer>() {
-      @Override
-      public void call(Integer integer) throws InterruptedException {
-        Thread.sleep(10000);  // To ensure that the job won't finish before it's cancelled.
-      }
-    });
-    future.cancel(true);
-    assertTrue(future.isCancelled());
-    assertTrue(future.isDone());
-    try {
-      future.get(2000, TimeUnit.MILLISECONDS);
-      fail("Expected future.get() for cancelled job to throw CancellationException");
-    } catch (CancellationException ignored) {
-      // pass
-    }
-  }
-
-  @Test
-  public void testAsyncActionErrorWrapping() throws Exception {
-    List<Integer> data = Arrays.asList(1, 2, 3, 4, 5);
-    JavaRDD<Integer> rdd = sc.parallelize(data, 1);
-    JavaFutureAction<Long> future = rdd.map(new BuggyMapFunction<Integer>()).countAsync();
-    try {
-      future.get(2, TimeUnit.SECONDS);
-      fail("Expected future.get() for failed job to throw ExcecutionException");
-    } catch (ExecutionException ee) {
-      assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!"));
-    }
-    assertTrue(future.isDone());
-  }
-
-  static class Class1 {}
-  static class Class2 {}
-
-  @Test
-  public void testRegisterKryoClasses() {
-    SparkConf conf = new SparkConf();
-    conf.registerKryoClasses(new Class<?>[]{ Class1.class, Class2.class });
-    assertEquals(
-        Class1.class.getName() + "," + Class2.class.getName(),
-        conf.get("spark.kryo.classesToRegister"));
-  }
-
-  @Test
-  public void testGetPersistentRDDs() {
-    java.util.Map<Integer, JavaRDD<?>> cachedRddsMap = sc.getPersistentRDDs();
-    assertTrue(cachedRddsMap.isEmpty());
-    JavaRDD<String> rdd1 = sc.parallelize(Arrays.asList("a", "b")).setName("RDD1").cache();
-    JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("c", "d")).setName("RDD2").cache();
-    cachedRddsMap = sc.getPersistentRDDs();
-    assertEquals(2, cachedRddsMap.size());
-    assertEquals("RDD1", cachedRddsMap.get(0).name());
-    assertEquals("RDD2", cachedRddsMap.get(1).name());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java b/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java
new file mode 100644
index 0000000..e22ad89
--- /dev/null
+++ b/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.*;
+
+import scala.Tuple2;
+
+import com.google.common.collect.Iterables;
+import com.google.common.io.Files;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.spark.api.java.JavaDoubleRDD;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.Optional;
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.util.Utils;
+
+/**
+ * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8
+ * lambda syntax.
+ */
+public class Java8RDDAPISuite implements Serializable {
+  private static int foreachCalls = 0;
+  private transient JavaSparkContext sc;
+
+  @Before
+  public void setUp() {
+    sc = new JavaSparkContext("local", "JavaAPISuite");
+  }
+
+  @After
+  public void tearDown() {
+    sc.stop();
+    sc = null;
+  }
+
+  @Test
+  public void foreachWithAnonymousClass() {
+    foreachCalls = 0;
+    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
+    rdd.foreach(new VoidFunction<String>() {
+      @Override
+      public void call(String s) {
+        foreachCalls++;
+      }
+    });
+    Assert.assertEquals(2, foreachCalls);
+  }
+
+  @Test
+  public void foreach() {
+    foreachCalls = 0;
+    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
+    rdd.foreach(x -> foreachCalls++);
+    Assert.assertEquals(2, foreachCalls);
+  }
+
+  @Test
+  public void groupBy() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    Function<Integer, Boolean> isOdd = x -> x % 2 == 0;
+    JavaPairRDD<Boolean, Iterable<Integer>> oddsAndEvens = rdd.groupBy(isOdd);
+    Assert.assertEquals(2, oddsAndEvens.count());
+    Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
+    Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
+
+    oddsAndEvens = rdd.groupBy(isOdd, 1);
+    Assert.assertEquals(2, oddsAndEvens.count());
+    Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
+    Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
+  }
+
+  @Test
+  public void leftOuterJoin() {
+    JavaPairRDD<Integer, Integer> rdd1 = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>(1, 1),
+      new Tuple2<>(1, 2),
+      new Tuple2<>(2, 1),
+      new Tuple2<>(3, 1)
+    ));
+    JavaPairRDD<Integer, Character> rdd2 = sc.parallelizePairs(Arrays.asList(
+      new Tuple2<>(1, 'x'),
+      new Tuple2<>(2, 'y'),
+      new Tuple2<>(2, 'z'),
+      new Tuple2<>(4, 'w')
+    ));
+    List<Tuple2<Integer, Tuple2<Integer, Optional<Character>>>> joined =
+      rdd1.leftOuterJoin(rdd2).collect();
+    Assert.assertEquals(5, joined.size());
+    Tuple2<Integer, Tuple2<Integer, Optional<Character>>> firstUnmatched =
+      rdd1.leftOuterJoin(rdd2).filter(tup -> !tup._2()._2().isPresent()).first();
+    Assert.assertEquals(3, firstUnmatched._1().intValue());
+  }
+
+  @Test
+  public void foldReduce() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
+    Function2<Integer, Integer, Integer> add = (a, b) -> a + b;
+
+    int sum = rdd.fold(0, add);
+    Assert.assertEquals(33, sum);
+
+    sum = rdd.reduce(add);
+    Assert.assertEquals(33, sum);
+  }
+
+  @Test
+  public void foldByKey() {
+    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
+      new Tuple2<>(2, 1),
+      new Tuple2<>(2, 1),
+      new Tuple2<>(1, 1),
+      new Tuple2<>(3, 2),
+      new Tuple2<>(3, 1)
+    );
+    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0, (a, b) -> a + b);
+    Assert.assertEquals(1, sums.lookup(1).get(0).intValue());
+    Assert.assertEquals(2, sums.lookup(2).get(0).intValue());
+    Assert.assertEquals(3, sums.lookup(3).get(0).intValue());
+  }
+
+  @Test
+  public void reduceByKey() {
+    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
+      new Tuple2<>(2, 1),
+      new Tuple2<>(2, 1),
+      new Tuple2<>(1, 1),
+      new Tuple2<>(3, 2),
+      new Tuple2<>(3, 1)
+    );
+    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
+    JavaPairRDD<Integer, Integer> counts = rdd.reduceByKey((a, b) -> a + b);
+    Assert.assertEquals(1, counts.lookup(1).get(0).intValue());
+    Assert.assertEquals(2, counts.lookup(2).get(0).intValue());
+    Assert.assertEquals(3, counts.lookup(3).get(0).intValue());
+
+    Map<Integer, Integer> localCounts = counts.collectAsMap();
+    Assert.assertEquals(1, localCounts.get(1).intValue());
+    Assert.assertEquals(2, localCounts.get(2).intValue());
+    Assert.assertEquals(3, localCounts.get(3).intValue());
+
+    localCounts = rdd.reduceByKeyLocally((a, b) -> a + b);
+    Assert.assertEquals(1, localCounts.get(1).intValue());
+    Assert.assertEquals(2, localCounts.get(2).intValue());
+    Assert.assertEquals(3, localCounts.get(3).intValue());
+  }
+
+  @Test
+  public void map() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x).cache();
+    doubles.collect();
+    JavaPairRDD<Integer, Integer> pairs = rdd.mapToPair(x -> new Tuple2<>(x, x))
+      .cache();
+    pairs.collect();
+    JavaRDD<String> strings = rdd.map(Object::toString).cache();
+    strings.collect();
+  }
+
+  @Test
+  public void flatMap() {
+    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello World!",
+      "The quick brown fox jumps over the lazy dog."));
+    JavaRDD<String> words = rdd.flatMap(x -> Arrays.asList(x.split(" ")).iterator());
+
+    Assert.assertEquals("Hello", words.first());
+    Assert.assertEquals(11, words.count());
+
+    JavaPairRDD<String, String> pairs = rdd.flatMapToPair(s -> {
+      List<Tuple2<String, String>> pairs2 = new LinkedList<>();
+      for (String word : s.split(" ")) {
+        pairs2.add(new Tuple2<>(word, word));
+      }
+      return pairs2.iterator();
+    });
+
+    Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairs.first());
+    Assert.assertEquals(11, pairs.count());
+
+    JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> {
+      List<Double> lengths = new LinkedList<>();
+      for (String word : s.split(" ")) {
+        lengths.add((double) word.length());
+      }
+      return lengths.iterator();
+    });
+
+    Assert.assertEquals(5.0, doubles.first(), 0.01);
+    Assert.assertEquals(11, pairs.count());
+  }
+
+  @Test
+  public void mapsFromPairsToPairs() {
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
+
+    // Regression test for SPARK-668:
+    JavaPairRDD<String, Integer> swapped =
+      pairRDD.flatMapToPair(x -> Collections.singletonList(x.swap()).iterator());
+    swapped.collect();
+
+    // There was never a bug here, but it's worth testing:
+    pairRDD.map(Tuple2::swap).collect();
+  }
+
+  @Test
+  public void mapPartitions() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
+    JavaRDD<Integer> partitionSums = rdd.mapPartitions(iter -> {
+      int sum = 0;
+      while (iter.hasNext()) {
+        sum += iter.next();
+      }
+      return Collections.singletonList(sum).iterator();
+    });
+
+    Assert.assertEquals("[3, 7]", partitionSums.collect().toString());
+  }
+
+  @Test
+  public void sequenceFile() {
+    File tempDir = Files.createTempDir();
+    tempDir.deleteOnExit();
+    String outputDir = new File(tempDir, "output").getAbsolutePath();
+    List<Tuple2<Integer, String>> pairs = Arrays.asList(
+      new Tuple2<>(1, "a"),
+      new Tuple2<>(2, "aa"),
+      new Tuple2<>(3, "aaa")
+    );
+    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
+
+    rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())))
+      .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
+
+    // Try reading the output back as an object file
+    JavaPairRDD<Integer, String> readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class)
+      .mapToPair(pair -> new Tuple2<>(pair._1().get(), pair._2().toString()));
+    Assert.assertEquals(pairs, readRDD.collect());
+    Utils.deleteRecursively(tempDir);
+  }
+
+  @Test
+  public void zip() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x);
+    JavaPairRDD<Integer, Double> zipped = rdd.zip(doubles);
+    zipped.count();
+  }
+
+  @Test
+  public void zipPartitions() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2);
+    JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2);
+    FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer> sizesFn =
+      (Iterator<Integer> i, Iterator<String> s) -> {
+        int sizeI = 0;
+        while (i.hasNext()) {
+          sizeI += 1;
+          i.next();
+        }
+        int sizeS = 0;
+        while (s.hasNext()) {
+          sizeS += 1;
+          s.next();
+        }
+        return Arrays.asList(sizeI, sizeS).iterator();
+      };
+    JavaRDD<Integer> sizes = rdd1.zipPartitions(rdd2, sizesFn);
+    Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString());
+  }
+
+  @Test
+  public void keyBy() {
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2));
+    List<Tuple2<String, Integer>> s = rdd.keyBy(Object::toString).collect();
+    Assert.assertEquals(new Tuple2<>("1", 1), s.get(0));
+    Assert.assertEquals(new Tuple2<>("2", 2), s.get(1));
+  }
+
+  @Test
+  public void mapOnPairRDD() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4));
+    JavaPairRDD<Integer, Integer> rdd2 =
+      rdd1.mapToPair(i -> new Tuple2<>(i, i % 2));
+    JavaPairRDD<Integer, Integer> rdd3 =
+      rdd2.mapToPair(in -> new Tuple2<>(in._2(), in._1()));
+    Assert.assertEquals(Arrays.asList(
+      new Tuple2<>(1, 1),
+      new Tuple2<>(0, 2),
+      new Tuple2<>(1, 3),
+      new Tuple2<>(0, 4)), rdd3.collect());
+  }
+
+  @Test
+  public void collectPartitions() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3);
+
+    JavaPairRDD<Integer, Integer> rdd2 =
+      rdd1.mapToPair(i -> new Tuple2<>(i, i % 2));
+    List<Integer>[] parts = rdd1.collectPartitions(new int[]{0});
+    Assert.assertEquals(Arrays.asList(1, 2), parts[0]);
+
+    parts = rdd1.collectPartitions(new int[]{1, 2});
+    Assert.assertEquals(Arrays.asList(3, 4), parts[0]);
+    Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]);
+
+    Assert.assertEquals(Arrays.asList(new Tuple2<>(1, 1), new Tuple2<>(2, 0)),
+      rdd2.collectPartitions(new int[]{0})[0]);
+
+    List<Tuple2<Integer, Integer>>[] parts2 = rdd2.collectPartitions(new int[]{1, 2});
+    Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]);
+    Assert.assertEquals(Arrays.asList(new Tuple2<>(5, 1), new Tuple2<>(6, 0), new Tuple2<>(7, 1)),
+      parts2[1]);
+  }
+
+  @Test
+  public void collectAsMapWithIntArrayValues() {
+    // Regression test for SPARK-1040
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1));
+    JavaPairRDD<Integer, int[]> pairRDD =
+      rdd.mapToPair(x -> new Tuple2<>(x, new int[]{x}));
+    pairRDD.collect();  // Works fine
+    pairRDD.collectAsMap();  // Used to crash with ClassCastException
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[5/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/streaming-kafka-0-8-integration.md
----------------------------------------------------------------------
diff --git a/docs/streaming-kafka-0-8-integration.md b/docs/streaming-kafka-0-8-integration.md
index 58b17aa..24a3e4c 100644
--- a/docs/streaming-kafka-0-8-integration.md
+++ b/docs/streaming-kafka-0-8-integration.md
@@ -155,33 +155,22 @@ Next, we discuss how to use this approach in your streaming application.
 	</div>
 	<div data-lang="java" markdown="1">
 		// Hold a reference to the current offset ranges, so it can be used downstream
-		final AtomicReference<OffsetRange[]> offsetRanges = new AtomicReference<>();
-
-		directKafkaStream.transformToPair(
-		  new Function<JavaPairRDD<String, String>, JavaPairRDD<String, String>>() {
-		    @Override
-		    public JavaPairRDD<String, String> call(JavaPairRDD<String, String> rdd) throws Exception {
-		      OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
-		      offsetRanges.set(offsets);
-		      return rdd;
-		    }
-		  }
-		).map(
+		AtomicReference<OffsetRange[]> offsetRanges = new AtomicReference<>();
+
+		directKafkaStream.transformToPair(rdd -> {
+      OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges();
+      offsetRanges.set(offsets);
+      return rdd;
+		}).map(
 		  ...
-		).foreachRDD(
-		  new Function<JavaPairRDD<String, String>, Void>() {
-		    @Override
-		    public Void call(JavaPairRDD<String, String> rdd) throws IOException {
-		      for (OffsetRange o : offsetRanges.get()) {
-		        System.out.println(
-		          o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset()
-		        );
-		      }
-		      ...
-		      return null;
-		    }
-		  }
-		);
+		).foreachRDD(rdd -> {
+      for (OffsetRange o : offsetRanges.get()) {
+        System.out.println(
+          o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset()
+        );
+      }
+      ...
+		});
 	</div>
 	<div data-lang="python" markdown="1">
 		offsetRanges = []

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index a878971..abd4ac9 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -163,12 +163,7 @@ space into words.
 
 {% highlight java %}
 // Split each line into words
-JavaDStream<String> words = lines.flatMap(
-  new FlatMapFunction<String, String>() {
-    @Override public Iterator<String> call(String x) {
-      return Arrays.asList(x.split(" ")).iterator();
-    }
-  });
+JavaDStream<String> words = lines.flatMap(x -> Arrays.asList(x.split(" ")).iterator());
 {% endhighlight %}
 
 `flatMap` is a DStream operation that creates a new DStream by
@@ -183,18 +178,8 @@ Next, we want to count these words.
 
 {% highlight java %}
 // Count each word in each batch
-JavaPairDStream<String, Integer> pairs = words.mapToPair(
-  new PairFunction<String, String, Integer>() {
-    @Override public Tuple2<String, Integer> call(String s) {
-      return new Tuple2<>(s, 1);
-    }
-  });
-JavaPairDStream<String, Integer> wordCounts = pairs.reduceByKey(
-  new Function2<Integer, Integer, Integer>() {
-    @Override public Integer call(Integer i1, Integer i2) {
-      return i1 + i2;
-    }
-  });
+JavaPairDStream<String, Integer> pairs = words.mapToPair(s -> new Tuple2<>(s, 1));
+JavaPairDStream<String, Integer> wordCounts = pairs.reduceByKey((i1, i2) -> i1 + i2);
 
 // Print the first ten elements of each RDD generated in this DStream to the console
 wordCounts.print();
@@ -836,11 +821,9 @@ the `(word, 1)` pairs) and the `runningCount` having the previous count.
 
 {% highlight java %}
 Function2<List<Integer>, Optional<Integer>, Optional<Integer>> updateFunction =
-  new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
-    @Override public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
-      Integer newSum = ...  // add the new values with the previous running count to get the new count
-      return Optional.of(newSum);
-    }
+  (values, state) -> {
+    Integer newSum = ...  // add the new values with the previous running count to get the new count
+    return Optional.of(newSum);
   };
 {% endhighlight %}
 
@@ -915,15 +898,12 @@ val cleanedDStream = wordCounts.transform { rdd =>
 {% highlight java %}
 import org.apache.spark.streaming.api.java.*;
 // RDD containing spam information
-final JavaPairRDD<String, Double> spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...);
+JavaPairRDD<String, Double> spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...);
 
-JavaPairDStream<String, Integer> cleanedDStream = wordCounts.transform(
-  new Function<JavaPairRDD<String, Integer>, JavaPairRDD<String, Integer>>() {
-    @Override public JavaPairRDD<String, Integer> call(JavaPairRDD<String, Integer> rdd) throws Exception {
-      rdd.join(spamInfoRDD).filter(...); // join data stream with spam information to do data cleaning
-      ...
-    }
-  });
+JavaPairDStream<String, Integer> cleanedDStream = wordCounts.transform(rdd -> {
+  rdd.join(spamInfoRDD).filter(...); // join data stream with spam information to do data cleaning
+  ...
+});
 {% endhighlight %}
 
 </div>
@@ -986,15 +966,8 @@ val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Se
 <div data-lang="java" markdown="1">
 
 {% highlight java %}
-// Reduce function adding two integers, defined separately for clarity
-Function2<Integer, Integer, Integer> reduceFunc = new Function2<Integer, Integer, Integer>() {
-  @Override public Integer call(Integer i1, Integer i2) {
-    return i1 + i2;
-  }
-};
-
 // Reduce last 30 seconds of data, every 10 seconds
-JavaPairDStream<String, Integer> windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, Durations.seconds(30), Durations.seconds(10));
+JavaPairDStream<String, Integer> windowedWordCounts = pairs.reduceByKeyAndWindow((i1, i2) -> i1 + i2, Durations.seconds(30), Durations.seconds(10));
 {% endhighlight %}
 
 </div>
@@ -1141,14 +1114,7 @@ val joinedStream = windowedStream.transform { rdd => rdd.join(dataset) }
 {% highlight java %}
 JavaPairRDD<String, String> dataset = ...
 JavaPairDStream<String, String> windowedStream = stream.window(Durations.seconds(20));
-JavaPairDStream<String, String> joinedStream = windowedStream.transform(
-  new Function<JavaRDD<Tuple2<String, String>>, JavaRDD<Tuple2<String, String>>>() {
-    @Override
-    public JavaRDD<Tuple2<String, String>> call(JavaRDD<Tuple2<String, String>> rdd) {
-      return rdd.join(dataset);
-    }
-  }
-);
+JavaPairDStream<String, String> joinedStream = windowedStream.transform(rdd -> rdd.join(dataset));
 {% endhighlight %}
 </div>
 <div data-lang="python" markdown="1">
@@ -1248,17 +1214,11 @@ dstream.foreachRDD { rdd =>
 </div>
 <div data-lang="java" markdown="1">
 {% highlight java %}
-dstream.foreachRDD(new VoidFunction<JavaRDD<String>>() {
-  @Override
-  public void call(JavaRDD<String> rdd) {
-    final Connection connection = createNewConnection(); // executed at the driver
-    rdd.foreach(new VoidFunction<String>() {
-      @Override
-      public void call(String record) {
-        connection.send(record); // executed at the worker
-      }
-    });
-  }
+dstream.foreachRDD(rdd -> {
+  Connection connection = createNewConnection(); // executed at the driver
+  rdd.foreach(record -> {
+    connection.send(record); // executed at the worker
+  });
 });
 {% endhighlight %}
 </div>
@@ -1297,18 +1257,12 @@ dstream.foreachRDD { rdd =>
 </div>
 <div data-lang="java" markdown="1">
 {% highlight java %}
-dstream.foreachRDD(new VoidFunction<JavaRDD<String>>() {
-  @Override
-  public void call(JavaRDD<String> rdd) {
-    rdd.foreach(new VoidFunction<String>() {
-      @Override
-      public void call(String record) {
-        Connection connection = createNewConnection();
-        connection.send(record);
-        connection.close();
-      }
-    });
-  }
+dstream.foreachRDD(rdd -> {
+  rdd.foreach(record -> {
+    Connection connection = createNewConnection();
+    connection.send(record);
+    connection.close();
+  });
 });
 {% endhighlight %}
 </div>
@@ -1344,20 +1298,14 @@ dstream.foreachRDD { rdd =>
 </div>
 <div data-lang="java" markdown="1">
 {% highlight java %}
-dstream.foreachRDD(new VoidFunction<JavaRDD<String>>() {
-  @Override
-  public void call(JavaRDD<String> rdd) {
-    rdd.foreachPartition(new VoidFunction<Iterator<String>>() {
-      @Override
-      public void call(Iterator<String> partitionOfRecords) {
-        Connection connection = createNewConnection();
-        while (partitionOfRecords.hasNext()) {
-          connection.send(partitionOfRecords.next());
-        }
-        connection.close();
-      }
-    });
-  }
+dstream.foreachRDD(rdd -> {
+  rdd.foreachPartition(partitionOfRecords -> {
+    Connection connection = createNewConnection();
+    while (partitionOfRecords.hasNext()) {
+      connection.send(partitionOfRecords.next());
+    }
+    connection.close();
+  });
 });
 {% endhighlight %}
 </div>
@@ -1396,21 +1344,15 @@ dstream.foreachRDD { rdd =>
 
 <div data-lang="java" markdown="1">
 {% highlight java %}
-dstream.foreachRDD(new VoidFunction<JavaRDD<String>>() {
-  @Override
-  public void call(JavaRDD<String> rdd) {
-    rdd.foreachPartition(new VoidFunction<Iterator<String>>() {
-      @Override
-      public void call(Iterator<String> partitionOfRecords) {
-        // ConnectionPool is a static, lazily initialized pool of connections
-        Connection connection = ConnectionPool.getConnection();
-        while (partitionOfRecords.hasNext()) {
-          connection.send(partitionOfRecords.next());
-        }
-        ConnectionPool.returnConnection(connection); // return to the pool for future reuse
-      }
-    });
-  }
+dstream.foreachRDD(rdd -> {
+  rdd.foreachPartition(partitionOfRecords -> {
+    // ConnectionPool is a static, lazily initialized pool of connections
+    Connection connection = ConnectionPool.getConnection();
+    while (partitionOfRecords.hasNext()) {
+      connection.send(partitionOfRecords.next());
+    }
+    ConnectionPool.returnConnection(connection); // return to the pool for future reuse
+  });
 });
 {% endhighlight %}
 </div>
@@ -1495,35 +1437,26 @@ public class JavaRow implements java.io.Serializable {
 
 JavaDStream<String> words = ... 
 
-words.foreachRDD(
-  new Function2<JavaRDD<String>, Time, Void>() {
-    @Override
-    public Void call(JavaRDD<String> rdd, Time time) {
-
-      // Get the singleton instance of SparkSession
-      SparkSession spark = SparkSession.builder().config(rdd.sparkContext().getConf()).getOrCreate();
+words.foreachRDD((rdd, time) -> {
+  // Get the singleton instance of SparkSession
+  SparkSession spark = SparkSession.builder().config(rdd.sparkContext().getConf()).getOrCreate();
 
-      // Convert RDD[String] to RDD[case class] to DataFrame
-      JavaRDD<JavaRow> rowRDD = rdd.map(new Function<String, JavaRow>() {
-        public JavaRow call(String word) {
-          JavaRow record = new JavaRow();
-          record.setWord(word);
-          return record;
-        }
-      });
-      DataFrame wordsDataFrame = spark.createDataFrame(rowRDD, JavaRow.class);
+  // Convert RDD[String] to RDD[case class] to DataFrame
+  JavaRDD<JavaRow> rowRDD = rdd.map(word -> {
+    JavaRow record = new JavaRow();
+    record.setWord(word);
+    return record;
+  });
+  DataFrame wordsDataFrame = spark.createDataFrame(rowRDD, JavaRow.class);
 
-      // Creates a temporary view using the DataFrame
-      wordsDataFrame.createOrReplaceTempView("words");
+  // Creates a temporary view using the DataFrame
+  wordsDataFrame.createOrReplaceTempView("words");
 
-      // Do word count on table using SQL and print it
-      DataFrame wordCountsDataFrame =
-        spark.sql("select word, count(*) as total from words group by word");
-      wordCountsDataFrame.show();
-      return null;
-    }
-  }
-);
+  // Do word count on table using SQL and print it
+  DataFrame wordCountsDataFrame =
+    spark.sql("select word, count(*) as total from words group by word");
+  wordCountsDataFrame.show();
+});
 {% endhighlight %}
 
 See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java).
@@ -1883,27 +1816,21 @@ class JavaDroppedWordsCounter {
   }
 }
 
-wordCounts.foreachRDD(new Function2<JavaPairRDD<String, Integer>, Time, Void>() {
-  @Override
-  public Void call(JavaPairRDD<String, Integer> rdd, Time time) throws IOException {
-    // Get or register the blacklist Broadcast
-    final Broadcast<List<String>> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context()));
-    // Get or register the droppedWordsCounter Accumulator
-    final LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context()));
-    // Use blacklist to drop words and use droppedWordsCounter to count them
-    String counts = rdd.filter(new Function<Tuple2<String, Integer>, Boolean>() {
-      @Override
-      public Boolean call(Tuple2<String, Integer> wordCount) throws Exception {
-        if (blacklist.value().contains(wordCount._1())) {
-          droppedWordsCounter.add(wordCount._2());
-          return false;
-        } else {
-          return true;
-        }
-      }
-    }).collect().toString();
-    String output = "Counts at time " + time + " " + counts;
-  }
+wordCounts.foreachRDD((rdd, time) -> {
+  // Get or register the blacklist Broadcast
+  Broadcast<List<String>> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context()));
+  // Get or register the droppedWordsCounter Accumulator
+  LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context()));
+  // Use blacklist to drop words and use droppedWordsCounter to count them
+  String counts = rdd.filter(wordCount -> {
+    if (blacklist.value().contains(wordCount._1())) {
+      droppedWordsCounter.add(wordCount._2());
+      return false;
+    } else {
+      return true;
+    }
+  }).collect().toString();
+  String output = "Counts at time " + time + " " + counts;
 }
 
 {% endhighlight %}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/docs/structured-streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md
index b816072..ad3b2fb 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -103,13 +103,7 @@ Dataset<Row> lines = spark
 // Split the lines into words
 Dataset<String> words = lines
   .as(Encoders.STRING())
-  .flatMap(
-    new FlatMapFunction<String, String>() {
-      @Override
-      public Iterator<String> call(String x) {
-        return Arrays.asList(x.split(" ")).iterator();
-      }
-    }, Encoders.STRING());
+  .flatMap((FlatMapFunction<String, String>) x -> Arrays.asList(x.split(" ")).iterator(), Encoders.STRING());
 
 // Generate running word count
 Dataset<Row> wordCounts = words.groupBy("value").count();
@@ -517,7 +511,7 @@ val csvDF = spark
 SparkSession spark = ...
 
 // Read text from socket 
-Dataset[Row] socketDF = spark
+Dataset<Row> socketDF = spark
   .readStream()
   .format("socket")
   .option("host", "localhost")
@@ -530,7 +524,7 @@ socketDF.printSchema();
 
 // Read all the csv files written atomically in a directory
 StructType userSchema = new StructType().add("name", "string").add("age", "integer");
-Dataset[Row] csvDF = spark
+Dataset<Row> csvDF = spark
   .readStream()
   .option("sep", ";")
   .schema(userSchema)      // Specify schema of the csv files
@@ -625,33 +619,15 @@ Dataset<DeviceData> ds = df.as(ExpressionEncoder.javaBean(DeviceData.class)); //
 
 // Select the devices which have signal more than 10
 df.select("device").where("signal > 10"); // using untyped APIs
-ds.filter(new FilterFunction<DeviceData>() { // using typed APIs
-  @Override
-  public boolean call(DeviceData value) throws Exception {
-    return value.getSignal() > 10;
-  }
-}).map(new MapFunction<DeviceData, String>() {
-  @Override
-  public String call(DeviceData value) throws Exception {
-    return value.getDevice();
-  }
-}, Encoders.STRING());
+ds.filter((FilterFunction<DeviceData>) value -> value.getSignal() > 10)
+  .map((MapFunction<DeviceData, String>) value -> value.getDevice(), Encoders.STRING());
 
 // Running count of the number of updates for each device type
 df.groupBy("deviceType").count(); // using untyped API
 
 // Running average signal for each device type
-ds.groupByKey(new MapFunction<DeviceData, String>() { // using typed API
-  @Override
-  public String call(DeviceData value) throws Exception {
-    return value.getDeviceType();
-  }
-}, Encoders.STRING()).agg(typed.avg(new MapFunction<DeviceData, Double>() {
-  @Override
-  public Double call(DeviceData value) throws Exception {
-    return value.getSignal();
-  }
-}));
+ds.groupByKey((MapFunction<DeviceData, String>) value -> value.getDeviceType(), Encoders.STRING())
+  .agg(typed.avg((MapFunction<DeviceData, Double>) value -> value.getSignal()));
 {% endhighlight %}
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java
index f42fd33..004e9b1 100644
--- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java
+++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java
@@ -69,9 +69,9 @@ public class JavaTokenizerExample {
         .setOutputCol("words")
         .setPattern("\\W");  // alternatively .setPattern("\\w+").setGaps(false);
 
-    spark.udf().register("countTokens", new UDF1<WrappedArray, Integer>() {
+    spark.udf().register("countTokens", new UDF1<WrappedArray<String>, Integer>() {
       @Override
-      public Integer call(WrappedArray words) {
+      public Integer call(WrappedArray<String> words) {
         return words.size();
       }
     }, DataTypes.IntegerType);

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java
index 1860594..b687fae 100644
--- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java
+++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java
@@ -224,7 +224,7 @@ public class JavaSQLDataSourceExample {
             "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}");
     JavaRDD<String> anotherPeopleRDD =
             new JavaSparkContext(spark.sparkContext()).parallelize(jsonData);
-    Dataset anotherPeople = spark.read().json(anotherPeopleRDD);
+    Dataset<Row> anotherPeople = spark.read().json(anotherPeopleRDD);
     anotherPeople.show();
     // +---------------+----+
     // |        address|name|

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/java8-tests/README.md
----------------------------------------------------------------------
diff --git a/external/java8-tests/README.md b/external/java8-tests/README.md
deleted file mode 100644
index aa87901..0000000
--- a/external/java8-tests/README.md
+++ /dev/null
@@ -1,22 +0,0 @@
-# Java 8 Test Suites
-
-These tests require having Java 8 installed and are isolated from the main Spark build.
-If Java 8 is not your system's default Java version, you will need to point Spark's build
-to your Java location. The set-up depends a bit on the build system:
-
-* Sbt users can either set JAVA_HOME to the location of a Java 8 JDK or explicitly pass
-  `-java-home` to the sbt launch script. If a Java 8 JDK is detected sbt will automatically
-  include the Java 8 test project.
-
-  `$ JAVA_HOME=/opt/jdk1.8.0/ build/sbt clean java8-tests/test
-
-* For Maven users,
-
-  Maven users can also refer to their Java 8 directory using JAVA_HOME.
-
-  `$ JAVA_HOME=/opt/jdk1.8.0/ mvn clean install -DskipTests`
-  `$ JAVA_HOME=/opt/jdk1.8.0/ mvn -pl :java8-tests_2.11 test`
-
-  Note that the above command can only be run from project root directory since this module
-  depends on core and the test-jars of core and streaming. This means an install step is
-  required to make the test dependencies visible to the Java 8 sub-project.

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/java8-tests/pom.xml
----------------------------------------------------------------------
diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml
deleted file mode 100644
index 8fc46d7..0000000
--- a/external/java8-tests/pom.xml
+++ /dev/null
@@ -1,132 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-~ Licensed to the Apache Software Foundation (ASF) under one or more
-~ contributor license agreements.  See the NOTICE file distributed with
-~ this work for additional information regarding copyright ownership.
-~ The ASF licenses this file to You under the Apache License, Version 2.0
-~ (the "License"); you may not use this file except in compliance with
-~ the License.  You may obtain a copy of the License at
-~
-~    http://www.apache.org/licenses/LICENSE-2.0
-~
-~ Unless required by applicable law or agreed to in writing, software
-~ distributed under the License is distributed on an "AS IS" BASIS,
-~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-~ See the License for the specific language governing permissions and
-~ limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent_2.11</artifactId>
-    <version>2.2.0-SNAPSHOT</version>
-    <relativePath>../../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>java8-tests_2.11</artifactId>
-  <packaging>pom</packaging>
-  <name>Spark Project Java 8 Tests</name>
-
-  <properties>
-    <sbt.project.name>java8-tests</sbt.project.name>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-tags_${scala.binary.version}</artifactId>
-    </dependency>
-
-    <!--
-      This spark-tags test-dep is needed even though it isn't used in this module, otherwise testing-cmds that exclude
-      them will yield errors.
-    -->
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-tags_${scala.binary.version}</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-deploy-plugin</artifactId>
-        <configuration>
-          <skip>true</skip>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-install-plugin</artifactId>
-        <configuration>
-          <skip>true</skip>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <forceJavacCompilerUse>true</forceJavacCompilerUse>
-          <source>1.8</source>
-          <target>1.8</target>
-          <compilerVersion>1.8</compilerVersion>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>net.alchim31.maven</groupId>
-        <artifactId>scala-maven-plugin</artifactId>
-        <configuration>
-          <useZincServer>${useZincForJdk8}</useZincServer>
-          <javacArgs>
-            <javacArg>-source</javacArg>
-            <javacArg>1.8</javacArg>
-            <javacArg>-target</javacArg>
-            <javacArg>1.8</javacArg>
-            <javacArg>-Xlint:all,-serial,-path</javacArg>
-          </javacArgs>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java
----------------------------------------------------------------------
diff --git a/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java
deleted file mode 100644
index fa3a66e..0000000
--- a/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java
+++ /dev/null
@@ -1,356 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package test.org.apache.spark.java8;
-
-import java.io.File;
-import java.io.Serializable;
-import java.util.*;
-
-import scala.Tuple2;
-
-import com.google.common.collect.Iterables;
-import com.google.common.io.Files;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.spark.api.java.JavaDoubleRDD;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.Optional;
-import org.apache.spark.api.java.function.*;
-import org.apache.spark.util.Utils;
-
-/**
- * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8
- * lambda syntax.
- */
-public class Java8RDDAPISuite implements Serializable {
-  private static int foreachCalls = 0;
-  private transient JavaSparkContext sc;
-
-  @Before
-  public void setUp() {
-    sc = new JavaSparkContext("local", "JavaAPISuite");
-  }
-
-  @After
-  public void tearDown() {
-    sc.stop();
-    sc = null;
-  }
-
-  @Test
-  public void foreachWithAnonymousClass() {
-    foreachCalls = 0;
-    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
-    rdd.foreach(new VoidFunction<String>() {
-      @Override
-      public void call(String s) {
-        foreachCalls++;
-      }
-    });
-    Assert.assertEquals(2, foreachCalls);
-  }
-
-  @Test
-  public void foreach() {
-    foreachCalls = 0;
-    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
-    rdd.foreach(x -> foreachCalls++);
-    Assert.assertEquals(2, foreachCalls);
-  }
-
-  @Test
-  public void groupBy() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    Function<Integer, Boolean> isOdd = x -> x % 2 == 0;
-    JavaPairRDD<Boolean, Iterable<Integer>> oddsAndEvens = rdd.groupBy(isOdd);
-    Assert.assertEquals(2, oddsAndEvens.count());
-    Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
-    Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
-
-    oddsAndEvens = rdd.groupBy(isOdd, 1);
-    Assert.assertEquals(2, oddsAndEvens.count());
-    Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0)));  // Evens
-    Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
-  }
-
-  @Test
-  public void leftOuterJoin() {
-    JavaPairRDD<Integer, Integer> rdd1 = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>(1, 1),
-      new Tuple2<>(1, 2),
-      new Tuple2<>(2, 1),
-      new Tuple2<>(3, 1)
-    ));
-    JavaPairRDD<Integer, Character> rdd2 = sc.parallelizePairs(Arrays.asList(
-      new Tuple2<>(1, 'x'),
-      new Tuple2<>(2, 'y'),
-      new Tuple2<>(2, 'z'),
-      new Tuple2<>(4, 'w')
-    ));
-    List<Tuple2<Integer, Tuple2<Integer, Optional<Character>>>> joined =
-      rdd1.leftOuterJoin(rdd2).collect();
-    Assert.assertEquals(5, joined.size());
-    Tuple2<Integer, Tuple2<Integer, Optional<Character>>> firstUnmatched =
-      rdd1.leftOuterJoin(rdd2).filter(tup -> !tup._2()._2().isPresent()).first();
-    Assert.assertEquals(3, firstUnmatched._1().intValue());
-  }
-
-  @Test
-  public void foldReduce() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
-    Function2<Integer, Integer, Integer> add = (a, b) -> a + b;
-
-    int sum = rdd.fold(0, add);
-    Assert.assertEquals(33, sum);
-
-    sum = rdd.reduce(add);
-    Assert.assertEquals(33, sum);
-  }
-
-  @Test
-  public void foldByKey() {
-    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
-      new Tuple2<>(2, 1),
-      new Tuple2<>(2, 1),
-      new Tuple2<>(1, 1),
-      new Tuple2<>(3, 2),
-      new Tuple2<>(3, 1)
-    );
-    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0, (a, b) -> a + b);
-    Assert.assertEquals(1, sums.lookup(1).get(0).intValue());
-    Assert.assertEquals(2, sums.lookup(2).get(0).intValue());
-    Assert.assertEquals(3, sums.lookup(3).get(0).intValue());
-  }
-
-  @Test
-  public void reduceByKey() {
-    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
-      new Tuple2<>(2, 1),
-      new Tuple2<>(2, 1),
-      new Tuple2<>(1, 1),
-      new Tuple2<>(3, 2),
-      new Tuple2<>(3, 1)
-    );
-    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
-    JavaPairRDD<Integer, Integer> counts = rdd.reduceByKey((a, b) -> a + b);
-    Assert.assertEquals(1, counts.lookup(1).get(0).intValue());
-    Assert.assertEquals(2, counts.lookup(2).get(0).intValue());
-    Assert.assertEquals(3, counts.lookup(3).get(0).intValue());
-
-    Map<Integer, Integer> localCounts = counts.collectAsMap();
-    Assert.assertEquals(1, localCounts.get(1).intValue());
-    Assert.assertEquals(2, localCounts.get(2).intValue());
-    Assert.assertEquals(3, localCounts.get(3).intValue());
-
-    localCounts = rdd.reduceByKeyLocally((a, b) -> a + b);
-    Assert.assertEquals(1, localCounts.get(1).intValue());
-    Assert.assertEquals(2, localCounts.get(2).intValue());
-    Assert.assertEquals(3, localCounts.get(3).intValue());
-  }
-
-  @Test
-  public void map() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-    JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x).cache();
-    doubles.collect();
-    JavaPairRDD<Integer, Integer> pairs = rdd.mapToPair(x -> new Tuple2<>(x, x))
-      .cache();
-    pairs.collect();
-    JavaRDD<String> strings = rdd.map(Object::toString).cache();
-    strings.collect();
-  }
-
-  @Test
-  public void flatMap() {
-    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello World!",
-      "The quick brown fox jumps over the lazy dog."));
-    JavaRDD<String> words = rdd.flatMap(x -> Arrays.asList(x.split(" ")).iterator());
-
-    Assert.assertEquals("Hello", words.first());
-    Assert.assertEquals(11, words.count());
-
-    JavaPairRDD<String, String> pairs = rdd.flatMapToPair(s -> {
-      List<Tuple2<String, String>> pairs2 = new LinkedList<>();
-      for (String word : s.split(" ")) {
-        pairs2.add(new Tuple2<>(word, word));
-      }
-      return pairs2.iterator();
-    });
-
-    Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairs.first());
-    Assert.assertEquals(11, pairs.count());
-
-    JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> {
-      List<Double> lengths = new LinkedList<>();
-      for (String word : s.split(" ")) {
-        lengths.add((double) word.length());
-      }
-      return lengths.iterator();
-    });
-
-    Assert.assertEquals(5.0, doubles.first(), 0.01);
-    Assert.assertEquals(11, pairs.count());
-  }
-
-  @Test
-  public void mapsFromPairsToPairs() {
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
-
-    // Regression test for SPARK-668:
-    JavaPairRDD<String, Integer> swapped =
-      pairRDD.flatMapToPair(x -> Collections.singletonList(x.swap()).iterator());
-    swapped.collect();
-
-    // There was never a bug here, but it's worth testing:
-    pairRDD.map(Tuple2::swap).collect();
-  }
-
-  @Test
-  public void mapPartitions() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
-    JavaRDD<Integer> partitionSums = rdd.mapPartitions(iter -> {
-      int sum = 0;
-      while (iter.hasNext()) {
-        sum += iter.next();
-      }
-      return Collections.singletonList(sum).iterator();
-    });
-
-    Assert.assertEquals("[3, 7]", partitionSums.collect().toString());
-  }
-
-  @Test
-  public void sequenceFile() {
-    File tempDir = Files.createTempDir();
-    tempDir.deleteOnExit();
-    String outputDir = new File(tempDir, "output").getAbsolutePath();
-    List<Tuple2<Integer, String>> pairs = Arrays.asList(
-      new Tuple2<>(1, "a"),
-      new Tuple2<>(2, "aa"),
-      new Tuple2<>(3, "aaa")
-    );
-    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
-
-    rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())))
-      .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);
-
-    // Try reading the output back as an object file
-    JavaPairRDD<Integer, String> readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class)
-      .mapToPair(pair -> new Tuple2<>(pair._1().get(), pair._2().toString()));
-    Assert.assertEquals(pairs, readRDD.collect());
-    Utils.deleteRecursively(tempDir);
-  }
-
-  @Test
-  public void zip() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
-    JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x);
-    JavaPairRDD<Integer, Double> zipped = rdd.zip(doubles);
-    zipped.count();
-  }
-
-  @Test
-  public void zipPartitions() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2);
-    JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2);
-    FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer> sizesFn =
-      (Iterator<Integer> i, Iterator<String> s) -> {
-        int sizeI = 0;
-        while (i.hasNext()) {
-          sizeI += 1;
-          i.next();
-        }
-        int sizeS = 0;
-        while (s.hasNext()) {
-          sizeS += 1;
-          s.next();
-        }
-        return Arrays.asList(sizeI, sizeS).iterator();
-      };
-    JavaRDD<Integer> sizes = rdd1.zipPartitions(rdd2, sizesFn);
-    Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString());
-  }
-
-  @Test
-  public void keyBy() {
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2));
-    List<Tuple2<String, Integer>> s = rdd.keyBy(Object::toString).collect();
-    Assert.assertEquals(new Tuple2<>("1", 1), s.get(0));
-    Assert.assertEquals(new Tuple2<>("2", 2), s.get(1));
-  }
-
-  @Test
-  public void mapOnPairRDD() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4));
-    JavaPairRDD<Integer, Integer> rdd2 =
-      rdd1.mapToPair(i -> new Tuple2<>(i, i % 2));
-    JavaPairRDD<Integer, Integer> rdd3 =
-      rdd2.mapToPair(in -> new Tuple2<>(in._2(), in._1()));
-    Assert.assertEquals(Arrays.asList(
-      new Tuple2<>(1, 1),
-      new Tuple2<>(0, 2),
-      new Tuple2<>(1, 3),
-      new Tuple2<>(0, 4)), rdd3.collect());
-  }
-
-  @Test
-  public void collectPartitions() {
-    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3);
-
-    JavaPairRDD<Integer, Integer> rdd2 =
-      rdd1.mapToPair(i -> new Tuple2<>(i, i % 2));
-    List<Integer>[] parts = rdd1.collectPartitions(new int[]{0});
-    Assert.assertEquals(Arrays.asList(1, 2), parts[0]);
-
-    parts = rdd1.collectPartitions(new int[]{1, 2});
-    Assert.assertEquals(Arrays.asList(3, 4), parts[0]);
-    Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]);
-
-    Assert.assertEquals(Arrays.asList(new Tuple2<>(1, 1), new Tuple2<>(2, 0)),
-      rdd2.collectPartitions(new int[]{0})[0]);
-
-    List<Tuple2<Integer, Integer>>[] parts2 = rdd2.collectPartitions(new int[]{1, 2});
-    Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]);
-    Assert.assertEquals(Arrays.asList(new Tuple2<>(5, 1), new Tuple2<>(6, 0), new Tuple2<>(7, 1)),
-      parts2[1]);
-  }
-
-  @Test
-  public void collectAsMapWithIntArrayValues() {
-    // Regression test for SPARK-1040
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1));
-    JavaPairRDD<Integer, int[]> pairRDD =
-      rdd.mapToPair(x -> new Tuple2<>(x, new int[]{x}));
-    pairRDD.collect();  // Works fine
-    pairRDD.collectAsMap();  // Used to crash with ClassCastException
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java
----------------------------------------------------------------------
diff --git a/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java
deleted file mode 100644
index 338ca54..0000000
--- a/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java
+++ /dev/null
@@ -1,882 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package test.org.apache.spark.java8.dstream;
-
-import java.io.Serializable;
-import java.util.*;
-
-import scala.Tuple2;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.spark.HashPartitioner;
-import org.apache.spark.api.java.Optional;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.function.PairFunction;
-import org.apache.spark.streaming.*;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.JavaPairDStream;
-import org.apache.spark.streaming.api.java.JavaMapWithStateDStream;
-
-/**
- * Most of these tests replicate org.apache.spark.streaming.JavaAPISuite using java 8
- * lambda syntax.
- */
-@SuppressWarnings("unchecked")
-public class Java8APISuite extends LocalJavaStreamingContext implements Serializable {
-
-  @Test
-  public void testMap() {
-    List<List<String>> inputData = Arrays.asList(
-      Arrays.asList("hello", "world"),
-      Arrays.asList("goodnight", "moon"));
-
-    List<List<Integer>> expected = Arrays.asList(
-      Arrays.asList(5, 5),
-      Arrays.asList(9, 4));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> letterCount = stream.map(String::length);
-    JavaTestUtils.attachTestOutputStream(letterCount);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @Test
-  public void testFilter() {
-    List<List<String>> inputData = Arrays.asList(
-      Arrays.asList("giants", "dodgers"),
-      Arrays.asList("yankees", "red sox"));
-
-    List<List<String>> expected = Arrays.asList(
-      Arrays.asList("giants"),
-      Arrays.asList("yankees"));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<String> filtered = stream.filter(s -> s.contains("a"));
-    JavaTestUtils.attachTestOutputStream(filtered);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @Test
-  public void testMapPartitions() {
-    List<List<String>> inputData = Arrays.asList(
-      Arrays.asList("giants", "dodgers"),
-      Arrays.asList("yankees", "red sox"));
-
-    List<List<String>> expected = Arrays.asList(
-      Arrays.asList("GIANTSDODGERS"),
-      Arrays.asList("YANKEESRED SOX"));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<String> mapped = stream.mapPartitions(in -> {
-      String out = "";
-      while (in.hasNext()) {
-        out = out + in.next().toUpperCase();
-      }
-      return Lists.newArrayList(out).iterator();
-    });
-    JavaTestUtils.attachTestOutputStream(mapped);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testReduce() {
-    List<List<Integer>> inputData = Arrays.asList(
-      Arrays.asList(1, 2, 3),
-      Arrays.asList(4, 5, 6),
-      Arrays.asList(7, 8, 9));
-
-    List<List<Integer>> expected = Arrays.asList(
-      Arrays.asList(6),
-      Arrays.asList(15),
-      Arrays.asList(24));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> reduced = stream.reduce((x, y) -> x + y);
-    JavaTestUtils.attachTestOutputStream(reduced);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testReduceByWindow() {
-    List<List<Integer>> inputData = Arrays.asList(
-      Arrays.asList(1, 2, 3),
-      Arrays.asList(4, 5, 6),
-      Arrays.asList(7, 8, 9));
-
-    List<List<Integer>> expected = Arrays.asList(
-      Arrays.asList(6),
-      Arrays.asList(21),
-      Arrays.asList(39),
-      Arrays.asList(24));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> reducedWindowed = stream.reduceByWindow((x, y) -> x + y,
-      (x, y) -> x - y, new Duration(2000), new Duration(1000));
-    JavaTestUtils.attachTestOutputStream(reducedWindowed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testTransform() {
-    List<List<Integer>> inputData = Arrays.asList(
-      Arrays.asList(1, 2, 3),
-      Arrays.asList(4, 5, 6),
-      Arrays.asList(7, 8, 9));
-
-    List<List<Integer>> expected = Arrays.asList(
-      Arrays.asList(3, 4, 5),
-      Arrays.asList(6, 7, 8),
-      Arrays.asList(9, 10, 11));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> transformed = stream.transform(in -> in.map(i -> i + 2));
-
-    JavaTestUtils.attachTestOutputStream(transformed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @Test
-  public void testVariousTransform() {
-    // tests whether all variations of transform can be called from Java
-
-    List<List<Integer>> inputData = Arrays.asList(Arrays.asList(1));
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-
-    List<List<Tuple2<String, Integer>>> pairInputData =
-      Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(
-      JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1));
-
-    JavaDStream<Integer> transformed1 = stream.transform(in -> null);
-    JavaDStream<Integer> transformed2 = stream.transform((x, time) -> null);
-    JavaPairDStream<String, Integer> transformed3 = stream.transformToPair(x -> null);
-    JavaPairDStream<String, Integer> transformed4 = stream.transformToPair((x, time) -> null);
-    JavaDStream<Integer> pairTransformed1 = pairStream.transform(x -> null);
-    JavaDStream<Integer> pairTransformed2 = pairStream.transform((x, time) -> null);
-    JavaPairDStream<String, String> pairTransformed3 = pairStream.transformToPair(x -> null);
-    JavaPairDStream<String, String> pairTransformed4 =
-      pairStream.transformToPair((x, time) -> null);
-
-  }
-
-  @Test
-  public void testTransformWith() {
-    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>("california", "dodgers"),
-        new Tuple2<>("new york", "yankees")),
-      Arrays.asList(
-        new Tuple2<>("california", "sharks"),
-        new Tuple2<>("new york", "rangers")));
-
-    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>("california", "giants"),
-        new Tuple2<>("new york", "mets")),
-      Arrays.asList(
-        new Tuple2<>("california", "ducks"),
-        new Tuple2<>("new york", "islanders")));
-
-
-    List<Set<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
-      Sets.newHashSet(
-        new Tuple2<>("california",
-          new Tuple2<>("dodgers", "giants")),
-        new Tuple2<>("new york",
-          new Tuple2<>("yankees", "mets"))),
-      Sets.newHashSet(
-        new Tuple2<>("california",
-          new Tuple2<>("sharks", "ducks")),
-        new Tuple2<>("new york",
-          new Tuple2<>("rangers", "islanders"))));
-
-    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
-      ssc, stringStringKVStream1, 1);
-    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
-
-    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
-      ssc, stringStringKVStream2, 1);
-    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
-
-    JavaPairDStream<String, Tuple2<String, String>> joined =
-      pairStream1.transformWithToPair(pairStream2,(x, y, z) -> x.join(y));
-
-    JavaTestUtils.attachTestOutputStream(joined);
-    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-    List<Set<Tuple2<String, Tuple2<String, String>>>> unorderedResult = Lists.newArrayList();
-    for (List<Tuple2<String, Tuple2<String, String>>> res : result) {
-      unorderedResult.add(Sets.newHashSet(res));
-    }
-
-    Assert.assertEquals(expected, unorderedResult);
-  }
-
-
-  @Test
-  public void testVariousTransformWith() {
-    // tests whether all variations of transformWith can be called from Java
-
-    List<List<Integer>> inputData1 = Arrays.asList(Arrays.asList(1));
-    List<List<String>> inputData2 = Arrays.asList(Arrays.asList("x"));
-    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1);
-    JavaDStream<String> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1);
-
-    List<List<Tuple2<String, Integer>>> pairInputData1 =
-      Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
-    List<List<Tuple2<Double, Character>>> pairInputData2 =
-      Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x')));
-    JavaPairDStream<String, Integer> pairStream1 = JavaPairDStream.fromJavaDStream(
-      JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1));
-    JavaPairDStream<Double, Character> pairStream2 = JavaPairDStream.fromJavaDStream(
-      JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1));
-
-    JavaDStream<Double> transformed1 = stream1.transformWith(stream2, (x, y, z) -> null);
-    JavaDStream<Double> transformed2 = stream1.transformWith(pairStream1,(x, y, z) -> null);
-
-    JavaPairDStream<Double, Double> transformed3 =
-      stream1.transformWithToPair(stream2,(x, y, z) -> null);
-
-    JavaPairDStream<Double, Double> transformed4 =
-      stream1.transformWithToPair(pairStream1,(x, y, z) -> null);
-
-    JavaDStream<Double> pairTransformed1 = pairStream1.transformWith(stream2,(x, y, z) -> null);
-
-    JavaDStream<Double> pairTransformed2_ =
-      pairStream1.transformWith(pairStream1,(x, y, z) -> null);
-
-    JavaPairDStream<Double, Double> pairTransformed3 =
-      pairStream1.transformWithToPair(stream2,(x, y, z) -> null);
-
-    JavaPairDStream<Double, Double> pairTransformed4 =
-      pairStream1.transformWithToPair(pairStream2,(x, y, z) -> null);
-  }
-
-  @Test
-  public void testStreamingContextTransform() {
-    List<List<Integer>> stream1input = Arrays.asList(
-      Arrays.asList(1),
-      Arrays.asList(2)
-    );
-
-    List<List<Integer>> stream2input = Arrays.asList(
-      Arrays.asList(3),
-      Arrays.asList(4)
-    );
-
-    List<List<Tuple2<Integer, String>>> pairStream1input = Arrays.asList(
-      Arrays.asList(new Tuple2<>(1, "x")),
-      Arrays.asList(new Tuple2<>(2, "y"))
-    );
-
-    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> expected = Arrays.asList(
-      Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))),
-      Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y")))
-    );
-
-    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1);
-    JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1);
-    JavaPairDStream<Integer, String> pairStream1 = JavaPairDStream.fromJavaDStream(
-      JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1));
-
-    List<JavaDStream<?>> listOfDStreams1 = Arrays.<JavaDStream<?>>asList(stream1, stream2);
-
-    // This is just to test whether this transform to JavaStream compiles
-    JavaDStream<Long> transformed1 = ssc.transform(
-      listOfDStreams1, (List<JavaRDD<?>> listOfRDDs, Time time) -> {
-      Assert.assertEquals(2, listOfRDDs.size());
-      return null;
-    });
-
-    List<JavaDStream<?>> listOfDStreams2 =
-      Arrays.<JavaDStream<?>>asList(stream1, stream2, pairStream1.toJavaDStream());
-
-    JavaPairDStream<Integer, Tuple2<Integer, String>> transformed2 = ssc.transformToPair(
-      listOfDStreams2, (List<JavaRDD<?>> listOfRDDs, Time time) -> {
-      Assert.assertEquals(3, listOfRDDs.size());
-      JavaRDD<Integer> rdd1 = (JavaRDD<Integer>) listOfRDDs.get(0);
-      JavaRDD<Integer> rdd2 = (JavaRDD<Integer>) listOfRDDs.get(1);
-      JavaRDD<Tuple2<Integer, String>> rdd3 = (JavaRDD<Tuple2<Integer, String>>) listOfRDDs.get(2);
-      JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3);
-      PairFunction<Integer, Integer, Integer> mapToTuple =
-        (Integer i) -> new Tuple2<>(i, i);
-      return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3);
-    });
-    JavaTestUtils.attachTestOutputStream(transformed2);
-    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> result =
-      JavaTestUtils.runStreams(ssc, 2, 2);
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testFlatMap() {
-    List<List<String>> inputData = Arrays.asList(
-      Arrays.asList("go", "giants"),
-      Arrays.asList("boo", "dodgers"),
-      Arrays.asList("athletics"));
-
-    List<List<String>> expected = Arrays.asList(
-      Arrays.asList("g", "o", "g", "i", "a", "n", "t", "s"),
-      Arrays.asList("b", "o", "o", "d", "o", "d", "g", "e", "r", "s"),
-      Arrays.asList("a", "t", "h", "l", "e", "t", "i", "c", "s"));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<String> flatMapped = stream.flatMap(
-        s -> Lists.newArrayList(s.split("(?!^)")).iterator());
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @Test
-  public void testPairFlatMap() {
-    List<List<String>> inputData = Arrays.asList(
-      Arrays.asList("giants"),
-      Arrays.asList("dodgers"),
-      Arrays.asList("athletics"));
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>(6, "g"),
-        new Tuple2<>(6, "i"),
-        new Tuple2<>(6, "a"),
-        new Tuple2<>(6, "n"),
-        new Tuple2<>(6, "t"),
-        new Tuple2<>(6, "s")),
-      Arrays.asList(
-        new Tuple2<>(7, "d"),
-        new Tuple2<>(7, "o"),
-        new Tuple2<>(7, "d"),
-        new Tuple2<>(7, "g"),
-        new Tuple2<>(7, "e"),
-        new Tuple2<>(7, "r"),
-        new Tuple2<>(7, "s")),
-      Arrays.asList(
-        new Tuple2<>(9, "a"),
-        new Tuple2<>(9, "t"),
-        new Tuple2<>(9, "h"),
-        new Tuple2<>(9, "l"),
-        new Tuple2<>(9, "e"),
-        new Tuple2<>(9, "t"),
-        new Tuple2<>(9, "i"),
-        new Tuple2<>(9, "c"),
-        new Tuple2<>(9, "s")));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<Integer, String> flatMapped = stream.flatMapToPair(s -> {
-      List<Tuple2<Integer, String>> out = Lists.newArrayList();
-      for (String letter : s.split("(?!^)")) {
-        out.add(new Tuple2<>(s.length(), letter));
-      }
-      return out.iterator();
-    });
-
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  /*
-   * Performs an order-invariant comparison of lists representing two RDD streams. This allows
-   * us to account for ordering variation within individual RDD's which occurs during windowing.
-   */
-  public static <T extends Comparable<T>> void assertOrderInvariantEquals(
-    List<List<T>> expected, List<List<T>> actual) {
-    expected.forEach(list -> Collections.sort(list));
-    List<List<T>> sortedActual = new ArrayList<>();
-    actual.forEach(list -> {
-        List<T> sortedList = new ArrayList<>(list);
-        Collections.sort(sortedList);
-        sortedActual.add(sortedList);
-    });
-    Assert.assertEquals(expected, sortedActual);
-  }
-
-  @Test
-  public void testPairFilter() {
-    List<List<String>> inputData = Arrays.asList(
-      Arrays.asList("giants", "dodgers"),
-      Arrays.asList("yankees", "red sox"));
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-      Arrays.asList(new Tuple2<>("giants", 6)),
-      Arrays.asList(new Tuple2<>("yankees", 7)));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream =
-      stream.mapToPair(x -> new Tuple2<>(x, x.length()));
-    JavaPairDStream<String, Integer> filtered = pairStream.filter(x -> x._1().contains("a"));
-    JavaTestUtils.attachTestOutputStream(filtered);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  List<List<Tuple2<String, String>>> stringStringKVStream = Arrays.asList(
-    Arrays.asList(new Tuple2<>("california", "dodgers"),
-      new Tuple2<>("california", "giants"),
-      new Tuple2<>("new york", "yankees"),
-      new Tuple2<>("new york", "mets")),
-    Arrays.asList(new Tuple2<>("california", "sharks"),
-      new Tuple2<>("california", "ducks"),
-      new Tuple2<>("new york", "rangers"),
-      new Tuple2<>("new york", "islanders")));
-
-  List<List<Tuple2<String, Integer>>> stringIntKVStream = Arrays.asList(
-    Arrays.asList(
-      new Tuple2<>("california", 1),
-      new Tuple2<>("california", 3),
-      new Tuple2<>("new york", 4),
-      new Tuple2<>("new york", 1)),
-    Arrays.asList(
-      new Tuple2<>("california", 5),
-      new Tuple2<>("california", 5),
-      new Tuple2<>("new york", 3),
-      new Tuple2<>("new york", 1)));
-
-  @Test
-  public void testPairMap() { // Maps pair -> pair of different type
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>(1, "california"),
-        new Tuple2<>(3, "california"),
-        new Tuple2<>(4, "new york"),
-        new Tuple2<>(1, "new york")),
-      Arrays.asList(
-        new Tuple2<>(5, "california"),
-        new Tuple2<>(5, "california"),
-        new Tuple2<>(3, "new york"),
-        new Tuple2<>(1, "new york")));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaPairDStream<Integer, String> reversed = pairStream.mapToPair(x -> x.swap());
-    JavaTestUtils.attachTestOutputStream(reversed);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testPairMapPartitions() { // Maps pair -> pair of different type
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>(1, "california"),
-        new Tuple2<>(3, "california"),
-        new Tuple2<>(4, "new york"),
-        new Tuple2<>(1, "new york")),
-      Arrays.asList(
-        new Tuple2<>(5, "california"),
-        new Tuple2<>(5, "california"),
-        new Tuple2<>(3, "new york"),
-        new Tuple2<>(1, "new york")));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaPairDStream<Integer, String> reversed = pairStream.mapPartitionsToPair(in -> {
-      LinkedList<Tuple2<Integer, String>> out = new LinkedList<>();
-      while (in.hasNext()) {
-        Tuple2<String, Integer> next = in.next();
-        out.add(next.swap());
-      }
-      return out.iterator();
-    });
-
-    JavaTestUtils.attachTestOutputStream(reversed);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testPairMap2() { // Maps pair -> single
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Integer>> expected = Arrays.asList(
-      Arrays.asList(1, 3, 4, 1),
-      Arrays.asList(5, 5, 3, 1));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaDStream<Integer> reversed = pairStream.map(in -> in._2());
-    JavaTestUtils.attachTestOutputStream(reversed);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair
-    List<List<Tuple2<String, Integer>>> inputData = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>("hi", 1),
-        new Tuple2<>("ho", 2)),
-      Arrays.asList(
-        new Tuple2<>("hi", 1),
-        new Tuple2<>("ho", 2)));
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>(1, "h"),
-        new Tuple2<>(1, "i"),
-        new Tuple2<>(2, "h"),
-        new Tuple2<>(2, "o")),
-      Arrays.asList(
-        new Tuple2<>(1, "h"),
-        new Tuple2<>(1, "i"),
-        new Tuple2<>(2, "h"),
-        new Tuple2<>(2, "o")));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaPairDStream<Integer, String> flatMapped = pairStream.flatMapToPair(in -> {
-      List<Tuple2<Integer, String>> out = new LinkedList<>();
-      for (Character s : in._1().toCharArray()) {
-        out.add(new Tuple2<>(in._2(), s.toString()));
-      }
-      return out.iterator();
-    });
-
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testPairReduceByKey() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>("california", 4),
-        new Tuple2<>("new york", 5)),
-      Arrays.asList(
-        new Tuple2<>("california", 10),
-        new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
-      ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> reduced = pairStream.reduceByKey((x, y) -> x + y);
-
-    JavaTestUtils.attachTestOutputStream(reduced);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testCombineByKey() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>("california", 4),
-        new Tuple2<>("new york", 5)),
-      Arrays.asList(
-        new Tuple2<>("california", 10),
-        new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
-      ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> combined = pairStream.<Integer>combineByKey(i -> i,
-      (x, y) -> x + y, (x, y) -> x + y, new HashPartitioner(2));
-
-    JavaTestUtils.attachTestOutputStream(combined);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testReduceByKeyAndWindow() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-      Arrays.asList(new Tuple2<>("california", 4),
-        new Tuple2<>("new york", 5)),
-      Arrays.asList(new Tuple2<>("california", 14),
-        new Tuple2<>("new york", 9)),
-      Arrays.asList(new Tuple2<>("california", 10),
-        new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> reduceWindowed =
-      pairStream.reduceByKeyAndWindow((x, y) -> x + y, new Duration(2000), new Duration(1000));
-    JavaTestUtils.attachTestOutputStream(reduceWindowed);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testUpdateStateByKey() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-      Arrays.asList(new Tuple2<>("california", 4),
-        new Tuple2<>("new york", 5)),
-      Arrays.asList(new Tuple2<>("california", 14),
-        new Tuple2<>("new york", 9)),
-      Arrays.asList(new Tuple2<>("california", 14),
-        new Tuple2<>("new york", 9)));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey((values, state) -> {
-      int out = 0;
-      if (state.isPresent()) {
-        out = out + state.get();
-      }
-      for (Integer v : values) {
-        out = out + v;
-      }
-      return Optional.of(out);
-    });
-
-    JavaTestUtils.attachTestOutputStream(updated);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testReduceByKeyAndWindowWithInverse() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-      Arrays.asList(new Tuple2<>("california", 4),
-        new Tuple2<>("new york", 5)),
-      Arrays.asList(new Tuple2<>("california", 14),
-        new Tuple2<>("new york", 9)),
-      Arrays.asList(new Tuple2<>("california", 10),
-        new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> reduceWindowed =
-      pairStream.reduceByKeyAndWindow((x, y) -> x + y, (x, y) -> x - y, new Duration(2000),
-        new Duration(1000));
-    JavaTestUtils.attachTestOutputStream(reduceWindowed);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testPairTransform() {
-    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>(3, 5),
-        new Tuple2<>(1, 5),
-        new Tuple2<>(4, 5),
-        new Tuple2<>(2, 5)),
-      Arrays.asList(
-        new Tuple2<>(2, 5),
-        new Tuple2<>(3, 5),
-        new Tuple2<>(4, 5),
-        new Tuple2<>(1, 5)));
-
-    List<List<Tuple2<Integer, Integer>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>(1, 5),
-        new Tuple2<>(2, 5),
-        new Tuple2<>(3, 5),
-        new Tuple2<>(4, 5)),
-      Arrays.asList(
-        new Tuple2<>(1, 5),
-        new Tuple2<>(2, 5),
-        new Tuple2<>(3, 5),
-        new Tuple2<>(4, 5)));
-
-    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
-      ssc, inputData, 1);
-    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<Integer, Integer> sorted = pairStream.transformToPair(in -> in.sortByKey());
-
-    JavaTestUtils.attachTestOutputStream(sorted);
-    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testPairToNormalRDDTransform() {
-    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>(3, 5),
-        new Tuple2<>(1, 5),
-        new Tuple2<>(4, 5),
-        new Tuple2<>(2, 5)),
-      Arrays.asList(
-        new Tuple2<>(2, 5),
-        new Tuple2<>(3, 5),
-        new Tuple2<>(4, 5),
-        new Tuple2<>(1, 5)));
-
-    List<List<Integer>> expected = Arrays.asList(
-      Arrays.asList(3, 1, 4, 2),
-      Arrays.asList(2, 3, 4, 1));
-
-    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
-      ssc, inputData, 1);
-    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaDStream<Integer> firstParts = pairStream.transform(in -> in.map(x -> x._1()));
-    JavaTestUtils.attachTestOutputStream(firstParts);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testMapValues() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
-
-    List<List<Tuple2<String, String>>> expected = Arrays.asList(
-      Arrays.asList(new Tuple2<>("california", "DODGERS"),
-        new Tuple2<>("california", "GIANTS"),
-        new Tuple2<>("new york", "YANKEES"),
-        new Tuple2<>("new york", "METS")),
-      Arrays.asList(new Tuple2<>("california", "SHARKS"),
-        new Tuple2<>("california", "DUCKS"),
-        new Tuple2<>("new york", "RANGERS"),
-        new Tuple2<>("new york", "ISLANDERS")));
-
-    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
-      ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, String> mapped = pairStream.mapValues(String::toUpperCase);
-    JavaTestUtils.attachTestOutputStream(mapped);
-    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @Test
-  public void testFlatMapValues() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
-
-    List<List<Tuple2<String, String>>> expected = Arrays.asList(
-      Arrays.asList(new Tuple2<>("california", "dodgers1"),
-        new Tuple2<>("california", "dodgers2"),
-        new Tuple2<>("california", "giants1"),
-        new Tuple2<>("california", "giants2"),
-        new Tuple2<>("new york", "yankees1"),
-        new Tuple2<>("new york", "yankees2"),
-        new Tuple2<>("new york", "mets1"),
-        new Tuple2<>("new york", "mets2")),
-      Arrays.asList(new Tuple2<>("california", "sharks1"),
-        new Tuple2<>("california", "sharks2"),
-        new Tuple2<>("california", "ducks1"),
-        new Tuple2<>("california", "ducks2"),
-        new Tuple2<>("new york", "rangers1"),
-        new Tuple2<>("new york", "rangers2"),
-        new Tuple2<>("new york", "islanders1"),
-        new Tuple2<>("new york", "islanders2")));
-
-    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
-      ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, String> flatMapped =
-      pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2"));
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-    Assert.assertEquals(expected, result);
-  }
-
-  /**
-   * This test is only for testing the APIs. It's not necessary to run it.
-   */
-  public void testMapWithStateAPI() {
-    JavaPairRDD<String, Boolean> initialRDD = null;
-    JavaPairDStream<String, Integer> wordsDstream = null;
-
-    JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream =
-        wordsDstream.mapWithState(
-            StateSpec.<String, Integer, Boolean, Double>function((time, key, value, state) -> {
-              // Use all State's methods here
-              state.exists();
-              state.get();
-              state.isTimingOut();
-              state.remove();
-              state.update(true);
-              return Optional.of(2.0);
-            }).initialState(initialRDD)
-                .numPartitions(10)
-                .partitioner(new HashPartitioner(10))
-                .timeout(Durations.seconds(10)));
-
-    JavaPairDStream<String, Boolean> emittedRecords = stateDstream.stateSnapshots();
-
-    JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream2 =
-        wordsDstream.mapWithState(
-            StateSpec.<String, Integer, Boolean, Double>function((key, value, state) -> {
-              state.exists();
-              state.get();
-              state.isTimingOut();
-              state.remove();
-              state.update(true);
-              return 2.0;
-            }).initialState(initialRDD)
-                .numPartitions(10)
-                .partitioner(new HashPartitioner(10))
-                .timeout(Durations.seconds(10)));
-
-    JavaPairDStream<String, Boolean> mappedDStream = stateDstream2.stateSnapshots();
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/java8-tests/src/test/java/test/org/apache/spark/java8/sql/Java8DatasetAggregatorSuite.java
----------------------------------------------------------------------
diff --git a/external/java8-tests/src/test/java/test/org/apache/spark/java8/sql/Java8DatasetAggregatorSuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/sql/Java8DatasetAggregatorSuite.java
deleted file mode 100644
index 10d25fa..0000000
--- a/external/java8-tests/src/test/java/test/org/apache/spark/java8/sql/Java8DatasetAggregatorSuite.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package test.org.apache.spark.java8.sql;
-
-import java.util.Arrays;
-
-import org.junit.Assert;
-import org.junit.Test;
-import scala.Tuple2;
-
-import org.apache.spark.sql.Dataset;
-import org.apache.spark.sql.KeyValueGroupedDataset;
-import org.apache.spark.sql.expressions.javalang.typed;
-import test.org.apache.spark.sql.JavaDatasetAggregatorSuiteBase;
-
-/**
- * Suite that replicates tests in JavaDatasetAggregatorSuite using lambda syntax.
- */
-public class Java8DatasetAggregatorSuite extends JavaDatasetAggregatorSuiteBase {
-  @Test
-  public void testTypedAggregationAverage() {
-    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
-    Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.avg(v -> (double)(v._2() * 2)));
-    Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 6.0)), agged.collectAsList());
-  }
-
-  @Test
-  public void testTypedAggregationCount() {
-    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
-    Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.count(v -> v));
-    Assert.assertEquals(Arrays.asList(tuple2("a", 2L), tuple2("b", 1L)), agged.collectAsList());
-  }
-
-  @Test
-  public void testTypedAggregationSumDouble() {
-    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
-    Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.sum(v -> (double)v._2()));
-    Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 3.0)), agged.collectAsList());
-  }
-
-  @Test
-  public void testTypedAggregationSumLong() {
-    KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset();
-    Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.sumLong(v -> (long)v._2()));
-    Assert.assertEquals(Arrays.asList(tuple2("a", 3L), tuple2("b", 3L)), agged.collectAsList());
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/java8-tests/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/external/java8-tests/src/test/resources/log4j.properties b/external/java8-tests/src/test/resources/log4j.properties
deleted file mode 100644
index 3706a6e..0000000
--- a/external/java8-tests/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# Set everything to be logged to the file target/unit-tests.log
-log4j.rootCategory=INFO, file
-log4j.appender.file=org.apache.log4j.FileAppender
-log4j.appender.file.append=true
-log4j.appender.file.file=target/unit-tests.log
-log4j.appender.file.layout=org.apache.log4j.PatternLayout
-log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
-
-# Ignore messages below warning level from Jetty, because it's a bit verbose
-log4j.logger.org.spark_project.jetty=WARN

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/java8-tests/src/test/scala/test/org/apache/spark/java8/JDK8ScalaSuite.scala
----------------------------------------------------------------------
diff --git a/external/java8-tests/src/test/scala/test/org/apache/spark/java8/JDK8ScalaSuite.scala b/external/java8-tests/src/test/scala/test/org/apache/spark/java8/JDK8ScalaSuite.scala
deleted file mode 100644
index c4042e4..0000000
--- a/external/java8-tests/src/test/scala/test/org/apache/spark/java8/JDK8ScalaSuite.scala
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package test.org.apache.spark.java8
-
-import org.apache.spark.SharedSparkContext
-import org.apache.spark.SparkFunSuite
-
-/**
- * Test cases where JDK8-compiled Scala user code is used with Spark.
- */
-class JDK8ScalaSuite extends SparkFunSuite with SharedSparkContext {
-  test("basic RDD closure test (SPARK-6152)") {
-    sc.parallelize(1 to 1000).map(x => x * x).count()
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
index 02b2311..9c5dcec 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
@@ -259,7 +259,7 @@ private[kafka010] class KafkaSource(
       val preferredLoc = if (numExecutors > 0) {
         // This allows cached KafkaConsumers in the executors to be re-used to read the same
         // partition in every batch.
-        Some(sortedExecutors(floorMod(tp.hashCode, numExecutors)))
+        Some(sortedExecutors(Math.floorMod(tp.hashCode, numExecutors)))
       } else None
       KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc)
     }.filter { range =>
@@ -347,5 +347,4 @@ private[kafka010] object KafkaSource {
     if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host }
   }
 
-  def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
index bf8adbe..4c6e2ce 100644
--- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
+++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
@@ -145,11 +145,6 @@ private[spark] class KafkaRDD[K, V](
       a.host > b.host
     }
 
-  /**
-   * Non-negative modulus, from java 8 math
-   */
-  private def floorMod(a: Int, b: Int): Int = ((a % b) + b) % b
-
   override def getPreferredLocations(thePart: Partition): Seq[String] = {
     // The intention is best-effort consistent executor for a given topicpartition,
     // so that caching consumers can be effective.
@@ -164,7 +159,7 @@ private[spark] class KafkaRDD[K, V](
       Seq()
     } else {
       // execs is sorted, tp.hashCode depends only on topic and partition, so consistent index
-      val index = this.floorMod(tp.hashCode, execs.length)
+      val index = Math.floorMod(tp.hashCode, execs.length)
       val chosen = execs(index)
       Seq(chosen.toString)
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
index 0622fef..bc8d603 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
@@ -104,15 +104,12 @@ abstract class AbstractCommandBuilder {
     // Load extra JAVA_OPTS from conf/java-opts, if it exists.
     File javaOpts = new File(join(File.separator, getConfDir(), "java-opts"));
     if (javaOpts.isFile()) {
-      BufferedReader br = new BufferedReader(new InputStreamReader(
-          new FileInputStream(javaOpts), StandardCharsets.UTF_8));
-      try {
+      try (BufferedReader br = new BufferedReader(new InputStreamReader(
+          new FileInputStream(javaOpts), StandardCharsets.UTF_8))) {
         String line;
         while ((line = br.readLine()) != null) {
           addOptionString(cmd, line);
         }
-      } finally {
-        br.close();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
index c0779e1..12bf29d 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java
@@ -18,7 +18,6 @@
 package org.apache.spark.launcher;
 
 import java.io.IOException;
-import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.logging.Level;
@@ -103,14 +102,7 @@ class ChildProcAppHandle implements SparkAppHandle {
       try {
         childProc.exitValue();
       } catch (IllegalThreadStateException e) {
-        // Child is still alive. Try to use Java 8's "destroyForcibly()" if available,
-        // fall back to the old API if it's not there.
-        try {
-          Method destroy = childProc.getClass().getMethod("destroyForcibly");
-          destroy.invoke(childProc);
-        } catch (Exception inner) {
-          childProc.destroy();
-        }
+        childProc.destroyForcibly();
       } finally {
         childProc = null;
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
index 250b2a8..e14c8aa 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
@@ -313,27 +313,6 @@ class CommandBuilderUtils {
   }
 
   /**
-   * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't
-   * set it.
-   */
-  static void addPermGenSizeOpt(List<String> cmd) {
-    // Don't set MaxPermSize for IBM Java, or Oracle Java 8 and later.
-    if (getJavaVendor() == JavaVendor.IBM) {
-      return;
-    }
-    if (javaMajorVersion(System.getProperty("java.version")) > 7) {
-      return;
-    }
-    for (String arg : cmd) {
-      if (arg.contains("-XX:MaxPermSize=")) {
-        return;
-      }
-    }
-
-    cmd.add("-XX:MaxPermSize=256m");
-  }
-
-  /**
    * Get the major version of the java version string supplied. This method
    * accepts any JEP-223-compliant strings (9-ea, 9+100), as well as legacy
    * version strings such as 1.7.0_79

http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
----------------------------------------------------------------------
diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
index ae43f56..865d492 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
@@ -137,12 +137,7 @@ class LauncherServer implements Closeable {
       this.server = server;
       this.running = true;
 
-      this.serverThread = factory.newThread(new Runnable() {
-        @Override
-        public void run() {
-          acceptConnections();
-        }
-      });
+      this.serverThread = factory.newThread(this::acceptConnections);
       serverThread.start();
     } catch (IOException ioe) {
       close();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[3/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/0e240549/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
deleted file mode 100644
index 648a5ab..0000000
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ /dev/null
@@ -1,2000 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming;
-
-import java.io.*;
-import java.nio.charset.StandardCharsets;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import scala.Tuple2;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.google.common.io.Files;
-import com.google.common.collect.Sets;
-
-import org.apache.spark.HashPartitioner;
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.Optional;
-import org.apache.spark.api.java.function.*;
-import org.apache.spark.storage.StorageLevel;
-import org.apache.spark.streaming.api.java.*;
-import org.apache.spark.util.LongAccumulator;
-import org.apache.spark.util.Utils;
-
-// The test suite itself is Serializable so that anonymous Function implementations can be
-// serialized, as an alternative to converting these anonymous classes to static inner classes;
-// see http://stackoverflow.com/questions/758570/.
-public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable {
-
-  public static void equalIterator(Iterator<?> a, Iterator<?> b) {
-    while (a.hasNext() && b.hasNext()) {
-      Assert.assertEquals(a.next(), b.next());
-    }
-    Assert.assertEquals(a.hasNext(), b.hasNext());
-  }
-
-  public static void equalIterable(Iterable<?> a, Iterable<?> b) {
-      equalIterator(a.iterator(), b.iterator());
-  }
-
-  @Test
-  public void testInitialization() {
-    Assert.assertNotNull(ssc.sparkContext());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testContextState() {
-    List<List<Integer>> inputData = Arrays.asList(Arrays.asList(1, 2, 3, 4));
-    Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState());
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaTestUtils.attachTestOutputStream(stream);
-    Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState());
-    ssc.start();
-    Assert.assertEquals(StreamingContextState.ACTIVE, ssc.getState());
-    ssc.stop();
-    Assert.assertEquals(StreamingContextState.STOPPED, ssc.getState());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCount() {
-    List<List<Integer>> inputData = Arrays.asList(
-        Arrays.asList(1,2,3,4),
-        Arrays.asList(3,4,5),
-        Arrays.asList(3));
-
-    List<List<Long>> expected = Arrays.asList(
-        Arrays.asList(4L),
-        Arrays.asList(3L),
-        Arrays.asList(1L));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Long> count = stream.count();
-    JavaTestUtils.attachTestOutputStream(count);
-    List<List<Long>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testMap() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("hello", "world"),
-        Arrays.asList("goodnight", "moon"));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(5,5),
-        Arrays.asList(9,4));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
-        @Override
-        public Integer call(String s) {
-          return s.length();
-        }
-    });
-    JavaTestUtils.attachTestOutputStream(letterCount);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testWindow() {
-    List<List<Integer>> inputData = Arrays.asList(
-        Arrays.asList(1,2,3),
-        Arrays.asList(4,5,6),
-        Arrays.asList(7,8,9));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(1,2,3),
-        Arrays.asList(4,5,6,1,2,3),
-        Arrays.asList(7,8,9,4,5,6),
-        Arrays.asList(7,8,9));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> windowed = stream.window(new Duration(2000));
-    JavaTestUtils.attachTestOutputStream(windowed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testWindowWithSlideDuration() {
-    List<List<Integer>> inputData = Arrays.asList(
-        Arrays.asList(1,2,3),
-        Arrays.asList(4,5,6),
-        Arrays.asList(7,8,9),
-        Arrays.asList(10,11,12),
-        Arrays.asList(13,14,15),
-        Arrays.asList(16,17,18));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(1,2,3,4,5,6),
-        Arrays.asList(1,2,3,4,5,6,7,8,9,10,11,12),
-        Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18),
-        Arrays.asList(13,14,15,16,17,18));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> windowed = stream.window(new Duration(4000), new Duration(2000));
-    JavaTestUtils.attachTestOutputStream(windowed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 8, 4);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testFilter() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("giants", "dodgers"),
-        Arrays.asList("yankees", "red sox"));
-
-    List<List<String>> expected = Arrays.asList(
-        Arrays.asList("giants"),
-        Arrays.asList("yankees"));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<String> filtered = stream.filter(new Function<String, Boolean>() {
-      @Override
-      public Boolean call(String s) {
-        return s.contains("a");
-      }
-    });
-    JavaTestUtils.attachTestOutputStream(filtered);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testRepartitionMorePartitions() {
-    List<List<Integer>> inputData = Arrays.asList(
-      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
-      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
-    JavaDStream<Integer> stream =
-        JavaTestUtils.attachTestInputStream(ssc, inputData, 2);
-    JavaDStreamLike<Integer,JavaDStream<Integer>,JavaRDD<Integer>> repartitioned =
-        stream.repartition(4);
-    JavaTestUtils.attachTestOutputStream(repartitioned);
-    List<List<List<Integer>>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2);
-    Assert.assertEquals(2, result.size());
-    for (List<List<Integer>> rdd : result) {
-      Assert.assertEquals(4, rdd.size());
-      Assert.assertEquals(
-        10, rdd.get(0).size() + rdd.get(1).size() + rdd.get(2).size() + rdd.get(3).size());
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testRepartitionFewerPartitions() {
-    List<List<Integer>> inputData = Arrays.asList(
-      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
-      Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
-    JavaDStream<Integer> stream =
-        JavaTestUtils.attachTestInputStream(ssc, inputData, 4);
-    JavaDStreamLike<Integer,JavaDStream<Integer>,JavaRDD<Integer>> repartitioned =
-        stream.repartition(2);
-    JavaTestUtils.attachTestOutputStream(repartitioned);
-    List<List<List<Integer>>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2);
-    Assert.assertEquals(2, result.size());
-    for (List<List<Integer>> rdd : result) {
-      Assert.assertEquals(2, rdd.size());
-      Assert.assertEquals(10, rdd.get(0).size() + rdd.get(1).size());
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testGlom() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("giants", "dodgers"),
-        Arrays.asList("yankees", "red sox"));
-
-    List<List<List<String>>> expected = Arrays.asList(
-        Arrays.asList(Arrays.asList("giants", "dodgers")),
-        Arrays.asList(Arrays.asList("yankees", "red sox")));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<List<String>> glommed = stream.glom();
-    JavaTestUtils.attachTestOutputStream(glommed);
-    List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testMapPartitions() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("giants", "dodgers"),
-        Arrays.asList("yankees", "red sox"));
-
-    List<List<String>> expected = Arrays.asList(
-        Arrays.asList("GIANTSDODGERS"),
-        Arrays.asList("YANKEESRED SOX"));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<String> mapped = stream.mapPartitions(
-        new FlatMapFunction<Iterator<String>, String>() {
-          @Override
-          public Iterator<String> call(Iterator<String> in) {
-            StringBuilder out = new StringBuilder();
-            while (in.hasNext()) {
-              out.append(in.next().toUpperCase(Locale.ENGLISH));
-            }
-            return Arrays.asList(out.toString()).iterator();
-          }
-        });
-    JavaTestUtils.attachTestOutputStream(mapped);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  private static class IntegerSum implements Function2<Integer, Integer, Integer> {
-    @Override
-    public Integer call(Integer i1, Integer i2) {
-      return i1 + i2;
-    }
-  }
-
-  private static class IntegerDifference implements Function2<Integer, Integer, Integer> {
-    @Override
-    public Integer call(Integer i1, Integer i2) {
-      return i1 - i2;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testReduce() {
-    List<List<Integer>> inputData = Arrays.asList(
-        Arrays.asList(1,2,3),
-        Arrays.asList(4,5,6),
-        Arrays.asList(7,8,9));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(6),
-        Arrays.asList(15),
-        Arrays.asList(24));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> reduced = stream.reduce(new IntegerSum());
-    JavaTestUtils.attachTestOutputStream(reduced);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testReduceByWindowWithInverse() {
-    testReduceByWindow(true);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testReduceByWindowWithoutInverse() {
-    testReduceByWindow(false);
-  }
-
-  @SuppressWarnings("unchecked")
-  private void testReduceByWindow(boolean withInverse) {
-    List<List<Integer>> inputData = Arrays.asList(
-        Arrays.asList(1,2,3),
-        Arrays.asList(4,5,6),
-        Arrays.asList(7,8,9));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(6),
-        Arrays.asList(21),
-        Arrays.asList(39),
-        Arrays.asList(24));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> reducedWindowed;
-    if (withInverse) {
-      reducedWindowed = stream.reduceByWindow(new IntegerSum(),
-                                              new IntegerDifference(),
-                                              new Duration(2000),
-                                              new Duration(1000));
-    } else {
-      reducedWindowed = stream.reduceByWindow(new IntegerSum(),
-                                              new Duration(2000), new Duration(1000));
-    }
-    JavaTestUtils.attachTestOutputStream(reducedWindowed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testQueueStream() {
-    ssc.stop();
-    // Create a new JavaStreamingContext without checkpointing
-    SparkConf conf = new SparkConf()
-        .setMaster("local[2]")
-        .setAppName("test")
-        .set("spark.streaming.clock", "org.apache.spark.util.ManualClock");
-    ssc = new JavaStreamingContext(conf, new Duration(1000));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(1,2,3),
-        Arrays.asList(4,5,6),
-        Arrays.asList(7,8,9));
-
-    JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc());
-    JavaRDD<Integer> rdd1 = jsc.parallelize(Arrays.asList(1, 2, 3));
-    JavaRDD<Integer> rdd2 = jsc.parallelize(Arrays.asList(4, 5, 6));
-    JavaRDD<Integer> rdd3 = jsc.parallelize(Arrays.asList(7,8,9));
-
-    Queue<JavaRDD<Integer>> rdds = new LinkedList<>();
-    rdds.add(rdd1);
-    rdds.add(rdd2);
-    rdds.add(rdd3);
-
-    JavaDStream<Integer> stream = ssc.queueStream(rdds);
-    JavaTestUtils.attachTestOutputStream(stream);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testTransform() {
-    List<List<Integer>> inputData = Arrays.asList(
-        Arrays.asList(1,2,3),
-        Arrays.asList(4,5,6),
-        Arrays.asList(7,8,9));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(3,4,5),
-        Arrays.asList(6,7,8),
-        Arrays.asList(9,10,11));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> transformed = stream.transform(
-      new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
-        @Override
-        public JavaRDD<Integer> call(JavaRDD<Integer> in) {
-          return in.map(new Function<Integer, Integer>() {
-            @Override
-            public Integer call(Integer i) {
-              return i + 2;
-            }
-          });
-        }
-      });
-
-    JavaTestUtils.attachTestOutputStream(transformed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testVariousTransform() {
-    // tests whether all variations of transform can be called from Java
-
-    List<List<Integer>> inputData = Arrays.asList(Arrays.asList(1));
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-
-    List<List<Tuple2<String, Integer>>> pairInputData =
-        Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(
-        JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1));
-
-    stream.transform(
-        new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
-          @Override
-          public JavaRDD<Integer> call(JavaRDD<Integer> in) {
-            return null;
-          }
-        }
-    );
-
-    stream.transform(
-      new Function2<JavaRDD<Integer>, Time, JavaRDD<Integer>>() {
-        @Override public JavaRDD<Integer> call(JavaRDD<Integer> in, Time time) {
-          return null;
-        }
-      }
-    );
-
-    stream.transformToPair(
-        new Function<JavaRDD<Integer>, JavaPairRDD<String, Integer>>() {
-          @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in) {
-            return null;
-          }
-        }
-    );
-
-    stream.transformToPair(
-        new Function2<JavaRDD<Integer>, Time, JavaPairRDD<String, Integer>>() {
-          @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in, Time time) {
-            return null;
-          }
-        }
-    );
-
-    pairStream.transform(
-        new Function<JavaPairRDD<String, Integer>, JavaRDD<Integer>>() {
-          @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in) {
-            return null;
-          }
-        }
-    );
-
-    pairStream.transform(
-        new Function2<JavaPairRDD<String, Integer>, Time, JavaRDD<Integer>>() {
-          @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in, Time time) {
-            return null;
-          }
-        }
-    );
-
-    pairStream.transformToPair(
-        new Function<JavaPairRDD<String, Integer>, JavaPairRDD<String, String>>() {
-          @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in) {
-            return null;
-          }
-        }
-    );
-
-    pairStream.transformToPair(
-        new Function2<JavaPairRDD<String, Integer>, Time, JavaPairRDD<String, String>>() {
-          @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in,
-                                                            Time time) {
-            return null;
-          }
-        }
-    );
-
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testTransformWith() {
-    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("california", "dodgers"),
-            new Tuple2<>("new york", "yankees")),
-        Arrays.asList(
-            new Tuple2<>("california", "sharks"),
-            new Tuple2<>("new york", "rangers")));
-
-    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("california", "giants"),
-            new Tuple2<>("new york", "mets")),
-        Arrays.asList(
-            new Tuple2<>("california", "ducks"),
-            new Tuple2<>("new york", "islanders")));
-
-
-    List<HashSet<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
-        Sets.newHashSet(
-            new Tuple2<>("california",
-                         new Tuple2<>("dodgers", "giants")),
-            new Tuple2<>("new york",
-                         new Tuple2<>("yankees", "mets"))),
-        Sets.newHashSet(
-            new Tuple2<>("california",
-                         new Tuple2<>("sharks", "ducks")),
-            new Tuple2<>("new york",
-                         new Tuple2<>("rangers", "islanders"))));
-
-    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream1, 1);
-    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
-
-    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream2, 1);
-    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
-
-    JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.transformWithToPair(
-        pairStream2,
-        new Function3<
-            JavaPairRDD<String, String>,
-            JavaPairRDD<String, String>,
-            Time,
-            JavaPairRDD<String, Tuple2<String, String>>>() {
-          @Override
-          public JavaPairRDD<String, Tuple2<String, String>> call(
-              JavaPairRDD<String, String> rdd1,
-              JavaPairRDD<String, String> rdd2,
-              Time time) {
-            return rdd1.join(rdd2);
-          }
-        }
-    );
-
-    JavaTestUtils.attachTestOutputStream(joined);
-    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-    List<HashSet<Tuple2<String, Tuple2<String, String>>>> unorderedResult = new ArrayList<>();
-    for (List<Tuple2<String, Tuple2<String, String>>> res: result) {
-      unorderedResult.add(Sets.newHashSet(res));
-    }
-
-    Assert.assertEquals(expected, unorderedResult);
-  }
-
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testVariousTransformWith() {
-    // tests whether all variations of transformWith can be called from Java
-
-    List<List<Integer>> inputData1 = Arrays.asList(Arrays.asList(1));
-    List<List<String>> inputData2 = Arrays.asList(Arrays.asList("x"));
-    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1);
-    JavaDStream<String> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1);
-
-    List<List<Tuple2<String, Integer>>> pairInputData1 =
-        Arrays.asList(Arrays.asList(new Tuple2<>("x", 1)));
-    List<List<Tuple2<Double, Character>>> pairInputData2 =
-        Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x')));
-    JavaPairDStream<String, Integer> pairStream1 = JavaPairDStream.fromJavaDStream(
-        JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1));
-    JavaPairDStream<Double, Character> pairStream2 = JavaPairDStream.fromJavaDStream(
-        JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1));
-
-    stream1.transformWith(
-        stream2,
-        new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() {
-          @Override
-          public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2, Time time) {
-            return null;
-          }
-        }
-    );
-
-    stream1.transformWith(
-        pairStream1,
-        new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time, JavaRDD<Double>>() {
-          @Override
-          public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaPairRDD<String, Integer> rdd2,
-                                      Time time) {
-            return null;
-          }
-        }
-    );
-
-    stream1.transformWithToPair(
-        stream2,
-        new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaPairRDD<Double, Double>>() {
-          @Override
-          public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2,
-                                                  Time time) {
-            return null;
-          }
-        }
-    );
-
-    stream1.transformWithToPair(
-        pairStream1,
-        new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time,
-          JavaPairRDD<Double, Double>>() {
-          @Override
-          public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1,
-                                                  JavaPairRDD<String, Integer> rdd2,
-                                                  Time time) {
-            return null;
-          }
-        }
-    );
-
-    pairStream1.transformWith(
-        stream2,
-        new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() {
-          @Override
-          public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, JavaRDD<String> rdd2,
-                                      Time time) {
-            return null;
-          }
-        }
-    );
-
-    pairStream1.transformWith(
-        pairStream1,
-        new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<String, Integer>, Time,
-          JavaRDD<Double>>() {
-          @Override
-          public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1,
-                                      JavaPairRDD<String, Integer> rdd2,
-                                      Time time) {
-            return null;
-          }
-        }
-    );
-
-    pairStream1.transformWithToPair(
-        stream2,
-        new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time,
-          JavaPairRDD<Double, Double>>() {
-          @Override
-          public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1,
-                                                  JavaRDD<String> rdd2,
-                                                  Time time) {
-            return null;
-          }
-        }
-    );
-
-    pairStream1.transformWithToPair(
-        pairStream2,
-        new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<Double, Character>, Time,
-          JavaPairRDD<Double, Double>>() {
-          @Override
-          public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1,
-                                                  JavaPairRDD<Double, Character> rdd2,
-                                                  Time time) {
-            return null;
-          }
-        }
-    );
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testStreamingContextTransform(){
-    List<List<Integer>> stream1input = Arrays.asList(
-        Arrays.asList(1),
-        Arrays.asList(2)
-    );
-
-    List<List<Integer>> stream2input = Arrays.asList(
-        Arrays.asList(3),
-        Arrays.asList(4)
-    );
-
-    List<List<Tuple2<Integer, String>>> pairStream1input = Arrays.asList(
-        Arrays.asList(new Tuple2<>(1, "x")),
-        Arrays.asList(new Tuple2<>(2, "y"))
-    );
-
-    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))),
-        Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y")))
-    );
-
-    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1);
-    JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1);
-    JavaPairDStream<Integer, String> pairStream1 = JavaPairDStream.fromJavaDStream(
-        JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1));
-
-    List<JavaDStream<?>> listOfDStreams1 = Arrays.<JavaDStream<?>>asList(stream1, stream2);
-
-    // This is just to test whether this transform to JavaStream compiles
-    ssc.transform(
-      listOfDStreams1,
-      new Function2<List<JavaRDD<?>>, Time, JavaRDD<Long>>() {
-        @Override
-        public JavaRDD<Long> call(List<JavaRDD<?>> listOfRDDs, Time time) {
-          Assert.assertEquals(2, listOfRDDs.size());
-          return null;
-        }
-      }
-    );
-
-    List<JavaDStream<?>> listOfDStreams2 =
-        Arrays.<JavaDStream<?>>asList(stream1, stream2, pairStream1.toJavaDStream());
-
-    JavaPairDStream<Integer, Tuple2<Integer, String>> transformed2 = ssc.transformToPair(
-      listOfDStreams2,
-      new Function2<List<JavaRDD<?>>, Time, JavaPairRDD<Integer, Tuple2<Integer, String>>>() {
-        @Override
-        public JavaPairRDD<Integer, Tuple2<Integer, String>> call(List<JavaRDD<?>> listOfRDDs,
-                                                                  Time time) {
-          Assert.assertEquals(3, listOfRDDs.size());
-          JavaRDD<Integer> rdd1 = (JavaRDD<Integer>)listOfRDDs.get(0);
-          JavaRDD<Integer> rdd2 = (JavaRDD<Integer>)listOfRDDs.get(1);
-          JavaRDD<Tuple2<Integer, String>> rdd3 =
-            (JavaRDD<Tuple2<Integer, String>>)listOfRDDs.get(2);
-          JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3);
-          PairFunction<Integer, Integer, Integer> mapToTuple =
-            new PairFunction<Integer, Integer, Integer>() {
-            @Override
-            public Tuple2<Integer, Integer> call(Integer i) {
-              return new Tuple2<>(i, i);
-            }
-          };
-          return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3);
-        }
-      }
-    );
-    JavaTestUtils.attachTestOutputStream(transformed2);
-    List<List<Tuple2<Integer, Tuple2<Integer, String>>>> result =
-      JavaTestUtils.runStreams(ssc, 2, 2);
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testFlatMap() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("go", "giants"),
-        Arrays.asList("boo", "dodgers"),
-        Arrays.asList("athletics"));
-
-    List<List<String>> expected = Arrays.asList(
-        Arrays.asList("g","o","g","i","a","n","t","s"),
-        Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"),
-        Arrays.asList("a","t","h","l","e","t","i","c","s"));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<String> flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
-      @Override
-      public Iterator<String> call(String x) {
-        return Arrays.asList(x.split("(?!^)")).iterator();
-      }
-    });
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testForeachRDD() {
-    final LongAccumulator accumRdd = ssc.sparkContext().sc().longAccumulator();
-    final LongAccumulator accumEle = ssc.sparkContext().sc().longAccumulator();
-    List<List<Integer>> inputData = Arrays.asList(
-        Arrays.asList(1,1,1),
-        Arrays.asList(1,1,1));
-
-    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaTestUtils.attachTestOutputStream(stream.count()); // dummy output
-
-    stream.foreachRDD(new VoidFunction<JavaRDD<Integer>>() {
-      @Override
-      public void call(JavaRDD<Integer> rdd) {
-        accumRdd.add(1);
-        rdd.foreach(new VoidFunction<Integer>() {
-          @Override
-          public void call(Integer i) {
-            accumEle.add(1);
-          }
-        });
-      }
-    });
-
-    // This is a test to make sure foreachRDD(VoidFunction2) can be called from Java
-    stream.foreachRDD(new VoidFunction2<JavaRDD<Integer>, Time>() {
-      @Override
-      public void call(JavaRDD<Integer> rdd, Time time) {
-      }
-    });
-
-    JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(2, accumRdd.value().intValue());
-    Assert.assertEquals(6, accumEle.value().intValue());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairFlatMap() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("giants"),
-        Arrays.asList("dodgers"),
-        Arrays.asList("athletics"));
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>(6, "g"),
-            new Tuple2<>(6, "i"),
-            new Tuple2<>(6, "a"),
-            new Tuple2<>(6, "n"),
-            new Tuple2<>(6, "t"),
-            new Tuple2<>(6, "s")),
-        Arrays.asList(
-            new Tuple2<>(7, "d"),
-            new Tuple2<>(7, "o"),
-            new Tuple2<>(7, "d"),
-            new Tuple2<>(7, "g"),
-            new Tuple2<>(7, "e"),
-            new Tuple2<>(7, "r"),
-            new Tuple2<>(7, "s")),
-        Arrays.asList(
-            new Tuple2<>(9, "a"),
-            new Tuple2<>(9, "t"),
-            new Tuple2<>(9, "h"),
-            new Tuple2<>(9, "l"),
-            new Tuple2<>(9, "e"),
-            new Tuple2<>(9, "t"),
-            new Tuple2<>(9, "i"),
-            new Tuple2<>(9, "c"),
-            new Tuple2<>(9, "s")));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<Integer, String> flatMapped = stream.flatMapToPair(
-      new PairFlatMapFunction<String, Integer, String>() {
-        @Override
-        public Iterator<Tuple2<Integer, String>> call(String in) {
-          List<Tuple2<Integer, String>> out = new ArrayList<>();
-          for (String letter: in.split("(?!^)")) {
-            out.add(new Tuple2<>(in.length(), letter));
-          }
-          return out.iterator();
-        }
-      });
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testUnion() {
-    List<List<Integer>> inputData1 = Arrays.asList(
-        Arrays.asList(1,1),
-        Arrays.asList(2,2),
-        Arrays.asList(3,3));
-
-    List<List<Integer>> inputData2 = Arrays.asList(
-        Arrays.asList(4,4),
-        Arrays.asList(5,5),
-        Arrays.asList(6,6));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(1,1,4,4),
-        Arrays.asList(2,2,5,5),
-        Arrays.asList(3,3,6,6));
-
-    JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
-    JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
-
-    JavaDStream<Integer> unioned = stream1.union(stream2);
-    JavaTestUtils.attachTestOutputStream(unioned);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  /*
-   * Performs an order-invariant comparison of lists representing two RDD streams. This allows
-   * us to account for ordering variation within individual RDD's which occurs during windowing.
-   */
-  public static <T> void assertOrderInvariantEquals(
-      List<List<T>> expected, List<List<T>> actual) {
-    List<Set<T>> expectedSets = new ArrayList<>();
-    for (List<T> list: expected) {
-      expectedSets.add(Collections.unmodifiableSet(new HashSet<>(list)));
-    }
-    List<Set<T>> actualSets = new ArrayList<>();
-    for (List<T> list: actual) {
-      actualSets.add(Collections.unmodifiableSet(new HashSet<>(list)));
-    }
-    Assert.assertEquals(expectedSets, actualSets);
-  }
-
-
-  // PairDStream Functions
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairFilter() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("giants", "dodgers"),
-        Arrays.asList("yankees", "red sox"));
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>("giants", 6)),
-        Arrays.asList(new Tuple2<>("yankees", 7)));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = stream.mapToPair(
-        new PairFunction<String, String, Integer>() {
-          @Override
-          public Tuple2<String, Integer> call(String in) {
-            return new Tuple2<>(in, in.length());
-          }
-        });
-
-    JavaPairDStream<String, Integer> filtered = pairStream.filter(
-        new Function<Tuple2<String, Integer>, Boolean>() {
-      @Override
-      public Boolean call(Tuple2<String, Integer> in) {
-        return in._1().contains("a");
-      }
-    });
-    JavaTestUtils.attachTestOutputStream(filtered);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  private final List<List<Tuple2<String, String>>> stringStringKVStream = Arrays.asList(
-      Arrays.asList(new Tuple2<>("california", "dodgers"),
-                    new Tuple2<>("california", "giants"),
-                    new Tuple2<>("new york", "yankees"),
-                    new Tuple2<>("new york", "mets")),
-      Arrays.asList(new Tuple2<>("california", "sharks"),
-                    new Tuple2<>("california", "ducks"),
-                    new Tuple2<>("new york", "rangers"),
-                    new Tuple2<>("new york", "islanders")));
-
-  @SuppressWarnings("unchecked")
-  private final List<List<Tuple2<String, Integer>>> stringIntKVStream = Arrays.asList(
-      Arrays.asList(
-          new Tuple2<>("california", 1),
-          new Tuple2<>("california", 3),
-          new Tuple2<>("new york", 4),
-          new Tuple2<>("new york", 1)),
-      Arrays.asList(
-          new Tuple2<>("california", 5),
-          new Tuple2<>("california", 5),
-          new Tuple2<>("new york", 3),
-          new Tuple2<>("new york", 1)));
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairMap() { // Maps pair -> pair of different type
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>(1, "california"),
-            new Tuple2<>(3, "california"),
-            new Tuple2<>(4, "new york"),
-            new Tuple2<>(1, "new york")),
-        Arrays.asList(
-            new Tuple2<>(5, "california"),
-            new Tuple2<>(5, "california"),
-            new Tuple2<>(3, "new york"),
-            new Tuple2<>(1, "new york")));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaPairDStream<Integer, String> reversed = pairStream.mapToPair(
-        new PairFunction<Tuple2<String, Integer>, Integer, String>() {
-          @Override
-          public Tuple2<Integer, String> call(Tuple2<String, Integer> in) {
-            return in.swap();
-          }
-        });
-
-    JavaTestUtils.attachTestOutputStream(reversed);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairMapPartitions() { // Maps pair -> pair of different type
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>(1, "california"),
-            new Tuple2<>(3, "california"),
-            new Tuple2<>(4, "new york"),
-            new Tuple2<>(1, "new york")),
-        Arrays.asList(
-            new Tuple2<>(5, "california"),
-            new Tuple2<>(5, "california"),
-            new Tuple2<>(3, "new york"),
-            new Tuple2<>(1, "new york")));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaPairDStream<Integer, String> reversed = pairStream.mapPartitionsToPair(
-        new PairFlatMapFunction<Iterator<Tuple2<String, Integer>>, Integer, String>() {
-          @Override
-          public Iterator<Tuple2<Integer, String>> call(Iterator<Tuple2<String, Integer>> in) {
-            List<Tuple2<Integer, String>> out = new LinkedList<>();
-            while (in.hasNext()) {
-              Tuple2<String, Integer> next = in.next();
-              out.add(next.swap());
-            }
-            return out.iterator();
-          }
-        });
-
-    JavaTestUtils.attachTestOutputStream(reversed);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairMap2() { // Maps pair -> single
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Integer>> expected = Arrays.asList(
-            Arrays.asList(1, 3, 4, 1),
-            Arrays.asList(5, 5, 3, 1));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaDStream<Integer> reversed = pairStream.map(
-        new Function<Tuple2<String, Integer>, Integer>() {
-          @Override
-          public Integer call(Tuple2<String, Integer> in) {
-            return in._2();
-          }
-        });
-
-    JavaTestUtils.attachTestOutputStream(reversed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair
-    List<List<Tuple2<String, Integer>>> inputData = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("hi", 1),
-            new Tuple2<>("ho", 2)),
-        Arrays.asList(
-            new Tuple2<>("hi", 1),
-            new Tuple2<>("ho", 2)));
-
-    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>(1, "h"),
-            new Tuple2<>(1, "i"),
-            new Tuple2<>(2, "h"),
-            new Tuple2<>(2, "o")),
-        Arrays.asList(
-            new Tuple2<>(1, "h"),
-            new Tuple2<>(1, "i"),
-            new Tuple2<>(2, "h"),
-            new Tuple2<>(2, "o")));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-        JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-    JavaPairDStream<Integer, String> flatMapped = pairStream.flatMapToPair(
-        new PairFlatMapFunction<Tuple2<String, Integer>, Integer, String>() {
-          @Override
-          public Iterator<Tuple2<Integer, String>> call(Tuple2<String, Integer> in) {
-            List<Tuple2<Integer, String>> out = new LinkedList<>();
-            for (Character s : in._1().toCharArray()) {
-              out.add(new Tuple2<>(in._2(), s.toString()));
-            }
-            return out.iterator();
-          }
-        });
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairGroupByKey() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
-
-    List<List<Tuple2<String, List<String>>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("california", Arrays.asList("dodgers", "giants")),
-            new Tuple2<>("new york", Arrays.asList("yankees", "mets"))),
-        Arrays.asList(
-            new Tuple2<>("california", Arrays.asList("sharks", "ducks")),
-            new Tuple2<>("new york", Arrays.asList("rangers", "islanders"))));
-
-    JavaDStream<Tuple2<String, String>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Iterable<String>> grouped = pairStream.groupByKey();
-    JavaTestUtils.attachTestOutputStream(grouped);
-    List<List<Tuple2<String, Iterable<String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected.size(), result.size());
-    Iterator<List<Tuple2<String, Iterable<String>>>> resultItr = result.iterator();
-    Iterator<List<Tuple2<String, List<String>>>> expectedItr = expected.iterator();
-    while (resultItr.hasNext() && expectedItr.hasNext()) {
-      Iterator<Tuple2<String, Iterable<String>>> resultElements = resultItr.next().iterator();
-      Iterator<Tuple2<String, List<String>>> expectedElements = expectedItr.next().iterator();
-      while (resultElements.hasNext() && expectedElements.hasNext()) {
-        Tuple2<String, Iterable<String>> resultElement = resultElements.next();
-        Tuple2<String, List<String>> expectedElement = expectedElements.next();
-        Assert.assertEquals(expectedElement._1(), resultElement._1());
-        equalIterable(expectedElement._2(), resultElement._2());
-      }
-      Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext());
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairReduceByKey() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("california", 4),
-            new Tuple2<>("new york", 5)),
-        Arrays.asList(
-            new Tuple2<>("california", 10),
-            new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> reduced = pairStream.reduceByKey(new IntegerSum());
-
-    JavaTestUtils.attachTestOutputStream(reduced);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCombineByKey() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("california", 4),
-            new Tuple2<>("new york", 5)),
-        Arrays.asList(
-            new Tuple2<>("california", 10),
-            new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> combined = pairStream.combineByKey(
-        new Function<Integer, Integer>() {
-          @Override
-          public Integer call(Integer i) {
-            return i;
-          }
-        }, new IntegerSum(), new IntegerSum(), new HashPartitioner(2));
-
-    JavaTestUtils.attachTestOutputStream(combined);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCountByValue() {
-    List<List<String>> inputData = Arrays.asList(
-      Arrays.asList("hello", "world"),
-      Arrays.asList("hello", "moon"),
-      Arrays.asList("hello"));
-
-    List<List<Tuple2<String, Long>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("hello", 1L),
-            new Tuple2<>("world", 1L)),
-        Arrays.asList(
-            new Tuple2<>("hello", 1L),
-            new Tuple2<>("moon", 1L)),
-        Arrays.asList(
-            new Tuple2<>("hello", 1L)));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Long> counted = stream.countByValue();
-    JavaTestUtils.attachTestOutputStream(counted);
-    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testGroupByKeyAndWindow() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, List<Integer>>>> expected = Arrays.asList(
-      Arrays.asList(
-        new Tuple2<>("california", Arrays.asList(1, 3)),
-        new Tuple2<>("new york", Arrays.asList(1, 4))
-      ),
-      Arrays.asList(
-        new Tuple2<>("california", Arrays.asList(1, 3, 5, 5)),
-        new Tuple2<>("new york", Arrays.asList(1, 1, 3, 4))
-      ),
-      Arrays.asList(
-        new Tuple2<>("california", Arrays.asList(5, 5)),
-        new Tuple2<>("new york", Arrays.asList(1, 3))
-      )
-    );
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Iterable<Integer>> groupWindowed =
-        pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000));
-    JavaTestUtils.attachTestOutputStream(groupWindowed);
-    List<List<Tuple2<String, List<Integer>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected.size(), result.size());
-    for (int i = 0; i < result.size(); i++) {
-      Assert.assertEquals(convert(expected.get(i)), convert(result.get(i)));
-    }
-  }
-
-  private static Set<Tuple2<String, HashSet<Integer>>>
-    convert(List<Tuple2<String, List<Integer>>> listOfTuples) {
-    List<Tuple2<String, HashSet<Integer>>> newListOfTuples = new ArrayList<>();
-    for (Tuple2<String, List<Integer>> tuple: listOfTuples) {
-      newListOfTuples.add(convert(tuple));
-    }
-    return new HashSet<>(newListOfTuples);
-  }
-
-  private static Tuple2<String, HashSet<Integer>> convert(Tuple2<String, List<Integer>> tuple) {
-    return new Tuple2<>(tuple._1(), new HashSet<>(tuple._2()));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testReduceByKeyAndWindow() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", 4),
-                      new Tuple2<>("new york", 5)),
-        Arrays.asList(new Tuple2<>("california", 14),
-                      new Tuple2<>("new york", 9)),
-        Arrays.asList(new Tuple2<>("california", 10),
-                      new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> reduceWindowed =
-        pairStream.reduceByKeyAndWindow(new IntegerSum(), new Duration(2000), new Duration(1000));
-    JavaTestUtils.attachTestOutputStream(reduceWindowed);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testUpdateStateByKey() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", 4),
-                      new Tuple2<>("new york", 5)),
-        Arrays.asList(new Tuple2<>("california", 14),
-                      new Tuple2<>("new york", 9)),
-        Arrays.asList(new Tuple2<>("california", 14),
-                      new Tuple2<>("new york", 9)));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey(
-        new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
-          @Override
-          public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
-            int out = 0;
-            if (state.isPresent()) {
-              out += state.get();
-            }
-            for (Integer v : values) {
-              out += v;
-            }
-            return Optional.of(out);
-          }
-        });
-    JavaTestUtils.attachTestOutputStream(updated);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testUpdateStateByKeyWithInitial() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<Tuple2<String, Integer>> initial = Arrays.asList(
-        new Tuple2<>("california", 1),
-            new Tuple2<>("new york", 2));
-
-    JavaRDD<Tuple2<String, Integer>> tmpRDD = ssc.sparkContext().parallelize(initial);
-    JavaPairRDD<String, Integer> initialRDD = JavaPairRDD.fromJavaRDD(tmpRDD);
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", 5),
-                      new Tuple2<>("new york", 7)),
-        Arrays.asList(new Tuple2<>("california", 15),
-                      new Tuple2<>("new york", 11)),
-        Arrays.asList(new Tuple2<>("california", 15),
-                      new Tuple2<>("new york", 11)));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey(
-        new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
-          @Override
-          public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
-            int out = 0;
-            if (state.isPresent()) {
-              out += state.get();
-            }
-            for (Integer v : values) {
-              out += v;
-            }
-            return Optional.of(out);
-          }
-        }, new HashPartitioner(1), initialRDD);
-    JavaTestUtils.attachTestOutputStream(updated);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testReduceByKeyAndWindowWithInverse() {
-    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
-
-    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", 4),
-                      new Tuple2<>("new york", 5)),
-        Arrays.asList(new Tuple2<>("california", 14),
-                      new Tuple2<>("new york", 9)),
-        Arrays.asList(new Tuple2<>("california", 10),
-                      new Tuple2<>("new york", 4)));
-
-    JavaDStream<Tuple2<String, Integer>> stream =
-      JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, Integer> reduceWindowed =
-        pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(),
-                                        new Duration(2000), new Duration(1000));
-    JavaTestUtils.attachTestOutputStream(reduceWindowed);
-    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCountByValueAndWindow() {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("hello", "world"),
-        Arrays.asList("hello", "moon"),
-        Arrays.asList("hello"));
-
-    List<HashSet<Tuple2<String, Long>>> expected = Arrays.asList(
-        Sets.newHashSet(
-            new Tuple2<>("hello", 1L),
-            new Tuple2<>("world", 1L)),
-        Sets.newHashSet(
-            new Tuple2<>("hello", 2L),
-            new Tuple2<>("world", 1L),
-            new Tuple2<>("moon", 1L)),
-        Sets.newHashSet(
-            new Tuple2<>("hello", 2L),
-            new Tuple2<>("moon", 1L)));
-
-    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<String, Long> counted =
-      stream.countByValueAndWindow(new Duration(2000), new Duration(1000));
-    JavaTestUtils.attachTestOutputStream(counted);
-    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
-    List<Set<Tuple2<String, Long>>> unorderedResult = new ArrayList<>();
-    for (List<Tuple2<String, Long>> res: result) {
-      unorderedResult.add(Sets.newHashSet(res));
-    }
-
-    Assert.assertEquals(expected, unorderedResult);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairTransform() {
-    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>(3, 5),
-            new Tuple2<>(1, 5),
-            new Tuple2<>(4, 5),
-            new Tuple2<>(2, 5)),
-        Arrays.asList(
-            new Tuple2<>(2, 5),
-            new Tuple2<>(3, 5),
-            new Tuple2<>(4, 5),
-            new Tuple2<>(1, 5)));
-
-    List<List<Tuple2<Integer, Integer>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>(1, 5),
-            new Tuple2<>(2, 5),
-            new Tuple2<>(3, 5),
-            new Tuple2<>(4, 5)),
-        Arrays.asList(
-            new Tuple2<>(1, 5),
-            new Tuple2<>(2, 5),
-            new Tuple2<>(3, 5),
-            new Tuple2<>(4, 5)));
-
-    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<Integer, Integer> sorted = pairStream.transformToPair(
-        new Function<JavaPairRDD<Integer, Integer>, JavaPairRDD<Integer, Integer>>() {
-          @Override
-          public JavaPairRDD<Integer, Integer> call(JavaPairRDD<Integer, Integer> in) {
-            return in.sortByKey();
-          }
-        });
-
-    JavaTestUtils.attachTestOutputStream(sorted);
-    List<List<Tuple2<Integer, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPairToNormalRDDTransform() {
-    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>(3, 5),
-            new Tuple2<>(1, 5),
-            new Tuple2<>(4, 5),
-            new Tuple2<>(2, 5)),
-        Arrays.asList(
-            new Tuple2<>(2, 5),
-            new Tuple2<>(3, 5),
-            new Tuple2<>(4, 5),
-            new Tuple2<>(1, 5)));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(3,1,4,2),
-        Arrays.asList(2,3,4,1));
-
-    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaDStream<Integer> firstParts = pairStream.transform(
-        new Function<JavaPairRDD<Integer, Integer>, JavaRDD<Integer>>() {
-          @Override
-          public JavaRDD<Integer> call(JavaPairRDD<Integer, Integer> in) {
-            return in.map(new Function<Tuple2<Integer, Integer>, Integer>() {
-              @Override
-              public Integer call(Tuple2<Integer, Integer> in2) {
-                return in2._1();
-              }
-            });
-          }
-        });
-
-    JavaTestUtils.attachTestOutputStream(firstParts);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testMapValues() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
-
-    List<List<Tuple2<String, String>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "DODGERS"),
-                      new Tuple2<>("california", "GIANTS"),
-                      new Tuple2<>("new york", "YANKEES"),
-                      new Tuple2<>("new york", "METS")),
-        Arrays.asList(new Tuple2<>("california", "SHARKS"),
-                      new Tuple2<>("california", "DUCKS"),
-                      new Tuple2<>("new york", "RANGERS"),
-                      new Tuple2<>("new york", "ISLANDERS")));
-
-    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-    JavaPairDStream<String, String> mapped = pairStream.mapValues(new Function<String, String>() {
-      @Override
-      public String call(String s) {
-        return s.toUpperCase(Locale.ENGLISH);
-      }
-    });
-
-    JavaTestUtils.attachTestOutputStream(mapped);
-    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testFlatMapValues() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
-
-    List<List<Tuple2<String, String>>> expected = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "dodgers1"),
-                      new Tuple2<>("california", "dodgers2"),
-                      new Tuple2<>("california", "giants1"),
-                      new Tuple2<>("california", "giants2"),
-                      new Tuple2<>("new york", "yankees1"),
-                      new Tuple2<>("new york", "yankees2"),
-                      new Tuple2<>("new york", "mets1"),
-                      new Tuple2<>("new york", "mets2")),
-        Arrays.asList(new Tuple2<>("california", "sharks1"),
-                      new Tuple2<>("california", "sharks2"),
-                      new Tuple2<>("california", "ducks1"),
-                      new Tuple2<>("california", "ducks2"),
-                      new Tuple2<>("new york", "rangers1"),
-                      new Tuple2<>("new york", "rangers2"),
-                      new Tuple2<>("new york", "islanders1"),
-                      new Tuple2<>("new york", "islanders2")));
-
-    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
-
-    JavaPairDStream<String, String> flatMapped = pairStream.flatMapValues(
-        new Function<String, Iterable<String>>() {
-          @Override
-          public Iterable<String> call(String in) {
-            List<String> out = new ArrayList<>();
-            out.add(in + "1");
-            out.add(in + "2");
-            return out;
-          }
-        });
-
-    JavaTestUtils.attachTestOutputStream(flatMapped);
-    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCoGroup() {
-    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "dodgers"),
-                      new Tuple2<>("new york", "yankees")),
-        Arrays.asList(new Tuple2<>("california", "sharks"),
-                      new Tuple2<>("new york", "rangers")));
-
-    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "giants"),
-                      new Tuple2<>("new york", "mets")),
-        Arrays.asList(new Tuple2<>("california", "ducks"),
-                      new Tuple2<>("new york", "islanders")));
-
-
-    List<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("california",
-                         new Tuple2<>(Arrays.asList("dodgers"), Arrays.asList("giants"))),
-            new Tuple2<>("new york",
-                         new Tuple2<>(Arrays.asList("yankees"), Arrays.asList("mets")))),
-        Arrays.asList(
-            new Tuple2<>("california",
-                         new Tuple2<>(Arrays.asList("sharks"), Arrays.asList("ducks"))),
-            new Tuple2<>("new york",
-                         new Tuple2<>(Arrays.asList("rangers"), Arrays.asList("islanders")))));
-
-
-    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream1, 1);
-    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
-
-    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream2, 1);
-    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
-
-    JavaPairDStream<String, Tuple2<Iterable<String>, Iterable<String>>> grouped =
-        pairStream1.cogroup(pairStream2);
-    JavaTestUtils.attachTestOutputStream(grouped);
-    List<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> result =
-        JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected.size(), result.size());
-    Iterator<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> resultItr =
-        result.iterator();
-    Iterator<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expectedItr =
-        expected.iterator();
-    while (resultItr.hasNext() && expectedItr.hasNext()) {
-      Iterator<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>> resultElements =
-          resultItr.next().iterator();
-      Iterator<Tuple2<String, Tuple2<List<String>, List<String>>>> expectedElements =
-          expectedItr.next().iterator();
-      while (resultElements.hasNext() && expectedElements.hasNext()) {
-        Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>> resultElement =
-            resultElements.next();
-        Tuple2<String, Tuple2<List<String>, List<String>>> expectedElement =
-            expectedElements.next();
-        Assert.assertEquals(expectedElement._1(), resultElement._1());
-        equalIterable(expectedElement._2()._1(), resultElement._2()._1());
-        equalIterable(expectedElement._2()._2(), resultElement._2()._2());
-      }
-      Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext());
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testJoin() {
-    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "dodgers"),
-                      new Tuple2<>("new york", "yankees")),
-        Arrays.asList(new Tuple2<>("california", "sharks"),
-                      new Tuple2<>("new york", "rangers")));
-
-    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "giants"),
-                      new Tuple2<>("new york", "mets")),
-        Arrays.asList(new Tuple2<>("california", "ducks"),
-                      new Tuple2<>("new york", "islanders")));
-
-
-    List<List<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
-        Arrays.asList(
-            new Tuple2<>("california",
-                         new Tuple2<>("dodgers", "giants")),
-            new Tuple2<>("new york",
-                         new Tuple2<>("yankees", "mets"))),
-        Arrays.asList(
-            new Tuple2<>("california",
-                         new Tuple2<>("sharks", "ducks")),
-            new Tuple2<>("new york",
-                         new Tuple2<>("rangers", "islanders"))));
-
-
-    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream1, 1);
-    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
-
-    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream2, 1);
-    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
-
-    JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.join(pairStream2);
-    JavaTestUtils.attachTestOutputStream(joined);
-    List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testLeftOuterJoin() {
-    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "dodgers"),
-                      new Tuple2<>("new york", "yankees")),
-        Arrays.asList(new Tuple2<>("california", "sharks") ));
-
-    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
-        Arrays.asList(new Tuple2<>("california", "giants") ),
-        Arrays.asList(new Tuple2<>("new york", "islanders") )
-
-    );
-
-    List<List<Long>> expected = Arrays.asList(Arrays.asList(2L), Arrays.asList(1L));
-
-    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream1, 1);
-    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
-
-    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
-        ssc, stringStringKVStream2, 1);
-    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
-
-    JavaPairDStream<String, Tuple2<String, Optional<String>>> joined =
-        pairStream1.leftOuterJoin(pairStream2);
-    JavaDStream<Long> counted = joined.count();
-    JavaTestUtils.attachTestOutputStream(counted);
-    List<List<Long>> result = JavaTestUtils.runStreams(ssc, 2, 2);
-
-    Assert.assertEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCheckpointMasterRecovery() throws InterruptedException {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("this", "is"),
-        Arrays.asList("a", "test"),
-        Arrays.asList("counting", "letters"));
-
-    List<List<Integer>> expectedInitial = Arrays.asList(
-        Arrays.asList(4,2));
-    List<List<Integer>> expectedFinal = Arrays.asList(
-        Arrays.asList(1,4),
-        Arrays.asList(8,7));
-
-    File tempDir = Files.createTempDir();
-    tempDir.deleteOnExit();
-    ssc.checkpoint(tempDir.getAbsolutePath());
-
-    JavaDStream<String> stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
-      @Override
-      public Integer call(String s) {
-        return s.length();
-      }
-    });
-    JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
-    List<List<Integer>> initialResult = JavaTestUtils.runStreams(ssc, 1, 1);
-
-    assertOrderInvariantEquals(expectedInitial, initialResult);
-    Thread.sleep(1000);
-    ssc.stop();
-
-    ssc = new JavaStreamingContext(tempDir.getAbsolutePath());
-    // Tweak to take into consideration that the last batch before failure
-    // will be re-processed after recovery
-    List<List<Integer>> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3);
-    assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3));
-    ssc.stop();
-    Utils.deleteRecursively(tempDir);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testContextGetOrCreate() throws InterruptedException {
-    ssc.stop();
-
-    final SparkConf conf = new SparkConf()
-        .setMaster("local[2]")
-        .setAppName("test")
-        .set("newContext", "true");
-
-    File emptyDir = Files.createTempDir();
-    emptyDir.deleteOnExit();
-    StreamingContextSuite contextSuite = new StreamingContextSuite();
-    String corruptedCheckpointDir = contextSuite.createCorruptedCheckpoint();
-    String checkpointDir = contextSuite.createValidCheckpoint();
-
-    // Function to create JavaStreamingContext without any output operations
-    // (used to detect the new context)
-    final AtomicBoolean newContextCreated = new AtomicBoolean(false);
-    Function0<JavaStreamingContext> creatingFunc = new Function0<JavaStreamingContext>() {
-      @Override
-      public JavaStreamingContext call() {
-        newContextCreated.set(true);
-        return new JavaStreamingContext(conf, Seconds.apply(1));
-      }
-    };
-
-    newContextCreated.set(false);
-    ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc);
-    Assert.assertTrue("new context not created", newContextCreated.get());
-    ssc.stop();
-
-    newContextCreated.set(false);
-    ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc,
-        new Configuration(), true);
-    Assert.assertTrue("new context not created", newContextCreated.get());
-    ssc.stop();
-
-    newContextCreated.set(false);
-    ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc,
-        new Configuration());
-    Assert.assertTrue("old context not recovered", !newContextCreated.get());
-    ssc.stop();
-
-    newContextCreated.set(false);
-    JavaSparkContext sc = new JavaSparkContext(conf);
-    ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc,
-        new Configuration());
-    Assert.assertTrue("old context not recovered", !newContextCreated.get());
-    ssc.stop();
-  }
-
-  /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCheckpointofIndividualStream() throws InterruptedException {
-    List<List<String>> inputData = Arrays.asList(
-        Arrays.asList("this", "is"),
-        Arrays.asList("a", "test"),
-        Arrays.asList("counting", "letters"));
-
-    List<List<Integer>> expected = Arrays.asList(
-        Arrays.asList(4,2),
-        Arrays.asList(1,4),
-        Arrays.asList(8,7));
-
-    JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream letterCount = stream.map(new Function<String, Integer>() {
-      @Override
-      public Integer call(String s) {
-        return s.length();
-      }
-    });
-    JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
-
-    letterCount.checkpoint(new Duration(1000));
-
-    List<List<Integer>> result1 = JavaCheckpointTestUtils.runStreams(ssc, 3, 3);
-    assertOrderInvariantEquals(expected, result1);
-  }
-  */
-
-  // Input stream tests. These mostly just test that we can instantiate a given InputStream with
-  // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the
-  // InputStream functionality is deferred to the existing Scala tests.
-  @Test
-  public void testSocketTextStream() {
-    ssc.socketTextStream("localhost", 12345);
-  }
-
-  @Test
-  public void testSocketString() {
-    ssc.socketStream(
-      "localhost",
-      12345,
-      new Function<InputStream, Iterable<String>>() {
-        @Override
-        public Iterable<String> call(InputStream in) throws IOException {
-          List<String> out = new ArrayList<>();
-          try (BufferedReader reader = new BufferedReader(
-              new InputStreamReader(in, StandardCharsets.UTF_8))) {
-            for (String line; (line = reader.readLine()) != null;) {
-              out.add(line);
-            }
-          }
-          return out;
-        }
-      },
-      StorageLevel.MEMORY_ONLY());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testTextFileStream() throws IOException {
-    File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark");
-    List<List<String>> expected = fileTestPrepare(testDir);
-
-    JavaDStream<String> input = ssc.textFileStream(testDir.toString());
-    JavaTestUtils.attachTestOutputStream(input);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 1, 1);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testFileStream() throws IOException {
-    File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark");
-    List<List<String>> expected = fileTestPrepare(testDir);
-
-    JavaPairInputDStream<LongWritable, Text> inputStream = ssc.fileStream(
-      testDir.toString(),
-      LongWritable.class,
-      Text.class,
-      TextInputFormat.class,
-      new Function<Path, Boolean>() {
-        @Override
-        public Boolean call(Path v1) {
-          return Boolean.TRUE;
-        }
-      },
-      true);
-
-    JavaDStream<String> test = inputStream.map(
-      new Function<Tuple2<LongWritable, Text>, String>() {
-        @Override
-        public String call(Tuple2<LongWritable, Text> v1) {
-          return v1._2().toString();
-        }
-    });
-
-    JavaTestUtils.attachTestOutputStream(test);
-    List<List<String>> result = JavaTestUtils.runStreams(ssc, 1, 1);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
-  @Test
-  public void testRawSocketStream() {
-    ssc.rawSocketStream("localhost", 12345);
-  }
-
-  private static List<List<String>> fileTestPrepare(File testDir) throws IOException {
-    File existingFile = new File(testDir, "0");
-    Files.write("0\n", existingFile, StandardCharsets.UTF_8);
-    Assert.assertTrue(existingFile.setLastModified(1000));
-    Assert.assertEquals(1000, existingFile.lastModified());
-    return Arrays.asList(Arrays.asList("0"));
-  }
-
-  @SuppressWarnings("unchecked")
-  // SPARK-5795: no logic assertions, just testing that intended API invocations compile
-  private void compileSaveAsJavaAPI(JavaPairDStream<LongWritable,Text> pds) {
-    pds.saveAsNewAPIHadoopFiles(
-        "", "", LongWritable.class, Text.class,
-        org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class);
-    pds.saveAsHadoopFiles(
-        "", "", LongWritable.class, Text.class,
-        org.apache.hadoop.mapred.SequenceFileOutputFormat.class);
-    // Checks that a previous common workaround for this API still compiles
-    pds.saveAsNewAPIHadoopFiles(
-        "", "", LongWritable.class, Text.class,
-        (Class) org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class);
-    pds.saveAsHadoopFiles(
-        "", "", LongWritable.class, Text.class,
-        (Class) org.apache.hadoop.mapred.SequenceFileOutputFormat.class);
-  }
-
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org