diff options
author | Sean Owen <sowen@cloudera.com> | 2015-09-12 10:40:10 +0100 |
---|---|---|
committer | Sean Owen <sowen@cloudera.com> | 2015-09-12 10:40:10 +0100 |
commit | 22730ad54d681ad30e63fe910e8d89360853177d (patch) | |
tree | 81194034499a6d391a0949e865fc0aa6dd5fc4ec /core/src | |
parent | 8285e3b0d3dc0eff669eba993742dfe0401116f9 (diff) | |
download | spark-22730ad54d681ad30e63fe910e8d89360853177d.tar.gz spark-22730ad54d681ad30e63fe910e8d89360853177d.tar.bz2 spark-22730ad54d681ad30e63fe910e8d89360853177d.zip |
[SPARK-10547] [TEST] Streamline / improve style of Java API tests
Fix a few Java API test style issues: unused generic types, exceptions, wrong assert argument order
Author: Sean Owen <sowen@cloudera.com>
Closes #8706 from srowen/SPARK-10547.
Diffstat (limited to 'core/src')
-rw-r--r-- | core/src/test/java/org/apache/spark/JavaAPISuite.java | 451 |
1 files changed, 224 insertions, 227 deletions
diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index ebd3d61ae7..fd8f7f39b7 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -90,7 +90,7 @@ public class JavaAPISuite implements Serializable { JavaRDD<String> sUnion = sc.union(s1, s2); Assert.assertEquals(4, sUnion.count()); // List - List<JavaRDD<String>> list = new ArrayList<JavaRDD<String>>(); + List<JavaRDD<String>> list = new ArrayList<>(); list.add(s2); sUnion = sc.union(s1, list); Assert.assertEquals(4, sUnion.count()); @@ -103,9 +103,9 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(4, dUnion.count()); // Union of JavaPairRDDs - List<Tuple2<Integer, Integer>> pairs = new ArrayList<Tuple2<Integer, Integer>>(); - pairs.add(new Tuple2<Integer, Integer>(1, 2)); - pairs.add(new Tuple2<Integer, Integer>(3, 4)); + 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); @@ -133,9 +133,9 @@ public class JavaAPISuite implements Serializable { JavaDoubleRDD dIntersection = d1.intersection(d2); Assert.assertEquals(2, dIntersection.count()); - List<Tuple2<Integer, Integer>> pairs = new ArrayList<Tuple2<Integer, Integer>>(); - pairs.add(new Tuple2<Integer, Integer>(1, 2)); - pairs.add(new Tuple2<Integer, Integer>(3, 4)); + 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); @@ -165,47 +165,49 @@ public class JavaAPISuite implements Serializable { @Test public void sortByKey() { - List<Tuple2<Integer, Integer>> pairs = new ArrayList<Tuple2<Integer, Integer>>(); - pairs.add(new Tuple2<Integer, Integer>(0, 4)); - pairs.add(new Tuple2<Integer, Integer>(3, 2)); - pairs.add(new Tuple2<Integer, Integer>(-1, 1)); + 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(); - Assert.assertEquals(new Tuple2<Integer, Integer>(-1, 1), sortedRDD.first()); + Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<Integer, Integer>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(2)); + Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); // Custom comparator sortedRDD = rdd.sortByKey(Collections.<Integer>reverseOrder(), false); - Assert.assertEquals(new Tuple2<Integer, Integer>(-1, 1), sortedRDD.first()); + Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<Integer, Integer>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(2)); + Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); } @SuppressWarnings("unchecked") @Test public void repartitionAndSortWithinPartitions() { - List<Tuple2<Integer, Integer>> pairs = new ArrayList<Tuple2<Integer, Integer>>(); - pairs.add(new Tuple2<Integer, Integer>(0, 5)); - pairs.add(new Tuple2<Integer, Integer>(3, 8)); - pairs.add(new Tuple2<Integer, Integer>(2, 6)); - pairs.add(new Tuple2<Integer, Integer>(0, 8)); - pairs.add(new Tuple2<Integer, Integer>(3, 8)); - pairs.add(new Tuple2<Integer, Integer>(1, 3)); + 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).intValue() % 2; + return (Integer) key % 2; } }; @@ -214,10 +216,10 @@ public class JavaAPISuite implements Serializable { Assert.assertTrue(repartitioned.partitioner().isPresent()); Assert.assertEquals(repartitioned.partitioner().get(), partitioner); List<List<Tuple2<Integer, Integer>>> partitions = repartitioned.glom().collect(); - Assert.assertEquals(partitions.get(0), Arrays.asList(new Tuple2<Integer, Integer>(0, 5), - new Tuple2<Integer, Integer>(0, 8), new Tuple2<Integer, Integer>(2, 6))); - Assert.assertEquals(partitions.get(1), Arrays.asList(new Tuple2<Integer, Integer>(1, 3), - new Tuple2<Integer, Integer>(3, 8), new Tuple2<Integer, Integer>(3, 8))); + Assert.assertEquals(partitions.get(0), + Arrays.asList(new Tuple2<>(0, 5), new Tuple2<>(0, 8), new Tuple2<>(2, 6))); + Assert.assertEquals(partitions.get(1), + Arrays.asList(new Tuple2<>(1, 3), new Tuple2<>(3, 8), new Tuple2<>(3, 8))); } @Test @@ -228,35 +230,37 @@ public class JavaAPISuite implements Serializable { @Test public void sortBy() { - List<Tuple2<Integer, Integer>> pairs = new ArrayList<Tuple2<Integer, Integer>>(); - pairs.add(new Tuple2<Integer, Integer>(0, 4)); - pairs.add(new Tuple2<Integer, Integer>(3, 2)); - pairs.add(new Tuple2<Integer, Integer>(-1, 1)); + 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>() { - public Integer call(Tuple2<Integer, Integer> t) throws Exception { + @Override + public Integer call(Tuple2<Integer, Integer> t) { return t._1(); } }, true, 2); - Assert.assertEquals(new Tuple2<Integer, Integer>(-1, 1), sortedRDD.first()); + Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<Integer, Integer>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(2)); + Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); // compare on second value sortedRDD = rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() { - public Integer call(Tuple2<Integer, Integer> t) throws Exception { + @Override + public Integer call(Tuple2<Integer, Integer> t) { return t._2(); } }, true, 2); - Assert.assertEquals(new Tuple2<Integer, Integer>(-1, 1), sortedRDD.first()); + Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<Integer, Integer>(0, 4), sortedPairs.get(2)); + Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1)); + Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(2)); } @Test @@ -265,7 +269,7 @@ public class JavaAPISuite implements Serializable { JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World")); rdd.foreach(new VoidFunction<String>() { @Override - public void call(String s) throws IOException { + public void call(String s) { accum.add(1); } }); @@ -278,7 +282,7 @@ public class JavaAPISuite implements Serializable { JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World")); rdd.foreachPartition(new VoidFunction<Iterator<String>>() { @Override - public void call(Iterator<String> iter) throws IOException { + public void call(Iterator<String> iter) { while (iter.hasNext()) { iter.next(); accum.add(1); @@ -301,7 +305,7 @@ public class JavaAPISuite implements Serializable { List<Integer> dataArray = Arrays.asList(1, 2, 3, 4); JavaPairRDD<Integer, Long> zip = sc.parallelize(dataArray).zipWithUniqueId(); JavaRDD<Long> indexes = zip.values(); - Assert.assertEquals(4, new HashSet<Long>(indexes.collect()).size()); + Assert.assertEquals(4, new HashSet<>(indexes.collect()).size()); } @Test @@ -317,10 +321,10 @@ public class JavaAPISuite implements Serializable { @Test public void lookup() { JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, String>("Apples", "Fruit"), - new Tuple2<String, String>("Oranges", "Fruit"), - new Tuple2<String, String>("Oranges", "Citrus") - )); + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") + )); Assert.assertEquals(2, categories.lookup("Oranges").size()); Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); } @@ -390,18 +394,17 @@ public class JavaAPISuite implements Serializable { @Test public void cogroup() { JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, String>("Apples", "Fruit"), - new Tuple2<String, String>("Oranges", "Fruit"), - new Tuple2<String, String>("Oranges", "Citrus") + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") )); JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, Integer>("Oranges", 2), - new Tuple2<String, Integer>("Apples", 3) + new Tuple2<>("Oranges", 2), + new Tuple2<>("Apples", 3) )); JavaPairRDD<String, Tuple2<Iterable<String>, Iterable<Integer>>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", - Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); @@ -411,23 +414,22 @@ public class JavaAPISuite implements Serializable { @Test public void cogroup3() { JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, String>("Apples", "Fruit"), - new Tuple2<String, String>("Oranges", "Fruit"), - new Tuple2<String, String>("Oranges", "Citrus") + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") )); JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, Integer>("Oranges", 2), - new Tuple2<String, Integer>("Apples", 3) + new Tuple2<>("Oranges", 2), + new Tuple2<>("Apples", 3) )); JavaPairRDD<String, Integer> quantities = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, Integer>("Oranges", 21), - new Tuple2<String, Integer>("Apples", 42) + new Tuple2<>("Oranges", 21), + new Tuple2<>("Apples", 42) )); JavaPairRDD<String, Tuple3<Iterable<String>, Iterable<Integer>, Iterable<Integer>>> cogrouped = categories.cogroup(prices, quantities); - Assert.assertEquals("[Fruit, Citrus]", - Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); @@ -439,27 +441,26 @@ public class JavaAPISuite implements Serializable { @Test public void cogroup4() { JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, String>("Apples", "Fruit"), - new Tuple2<String, String>("Oranges", "Fruit"), - new Tuple2<String, String>("Oranges", "Citrus") + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") )); JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, Integer>("Oranges", 2), - new Tuple2<String, Integer>("Apples", 3) + new Tuple2<>("Oranges", 2), + new Tuple2<>("Apples", 3) )); JavaPairRDD<String, Integer> quantities = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, Integer>("Oranges", 21), - new Tuple2<String, Integer>("Apples", 42) + new Tuple2<>("Oranges", 21), + new Tuple2<>("Apples", 42) )); JavaPairRDD<String, String> countries = sc.parallelizePairs(Arrays.asList( - new Tuple2<String, String>("Oranges", "BR"), - new Tuple2<String, String>("Apples", "US") + 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); - Assert.assertEquals("[Fruit, Citrus]", - Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); Assert.assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4())); @@ -471,16 +472,16 @@ public class JavaAPISuite implements Serializable { @Test public void leftOuterJoin() { JavaPairRDD<Integer, Integer> rdd1 = sc.parallelizePairs(Arrays.asList( - new Tuple2<Integer, Integer>(1, 1), - new Tuple2<Integer, Integer>(1, 2), - new Tuple2<Integer, Integer>(2, 1), - new Tuple2<Integer, Integer>(3, 1) + 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<Integer, Character>(1, 'x'), - new Tuple2<Integer, Character>(2, 'y'), - new Tuple2<Integer, Character>(2, 'z'), - new Tuple2<Integer, Character>(4, 'w') + 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(); @@ -548,11 +549,11 @@ public class JavaAPISuite implements Serializable { public void aggregateByKey() { JavaPairRDD<Integer, Integer> pairs = sc.parallelizePairs( Arrays.asList( - new Tuple2<Integer, Integer>(1, 1), - new Tuple2<Integer, Integer>(1, 1), - new Tuple2<Integer, Integer>(3, 2), - new Tuple2<Integer, Integer>(5, 1), - new Tuple2<Integer, Integer>(5, 3)), 2); + 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>>() { @@ -570,20 +571,20 @@ public class JavaAPISuite implements Serializable { } }).collectAsMap(); Assert.assertEquals(3, sets.size()); - Assert.assertEquals(new HashSet<Integer>(Arrays.asList(1)), sets.get(1)); - Assert.assertEquals(new HashSet<Integer>(Arrays.asList(2)), sets.get(3)); - Assert.assertEquals(new HashSet<Integer>(Arrays.asList(1, 3)), sets.get(5)); + Assert.assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1)); + Assert.assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3)); + Assert.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<Integer, Integer>(2, 1), - new Tuple2<Integer, Integer>(2, 1), - new Tuple2<Integer, Integer>(1, 1), - new Tuple2<Integer, Integer>(3, 2), - new Tuple2<Integer, Integer>(3, 1) + 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, @@ -602,11 +603,11 @@ public class JavaAPISuite implements Serializable { @Test public void reduceByKey() { List<Tuple2<Integer, Integer>> pairs = Arrays.asList( - new Tuple2<Integer, Integer>(2, 1), - new Tuple2<Integer, Integer>(2, 1), - new Tuple2<Integer, Integer>(1, 1), - new Tuple2<Integer, Integer>(3, 2), - new Tuple2<Integer, Integer>(3, 1) + 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( @@ -690,7 +691,7 @@ public class JavaAPISuite implements Serializable { 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); - Assert.assertEquals(new Tuple2<String, Double>("Hello", 1.0), cartesian.first()); + Assert.assertEquals(new Tuple2<>("Hello", 1.0), cartesian.first()); } @Test @@ -743,6 +744,7 @@ public class JavaAPISuite implements Serializable { } private static class DoubleComparator implements Comparator<Double>, Serializable { + @Override public int compare(Double o1, Double o2) { return o1.compareTo(o2); } @@ -766,14 +768,14 @@ public class JavaAPISuite implements Serializable { public void naturalMax() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); double max = rdd.max(); - Assert.assertTrue(4.0 == max); + Assert.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(); - Assert.assertTrue(1.0 == max); + Assert.assertEquals(1.0, max, 0.0); } @Test @@ -809,7 +811,7 @@ public class JavaAPISuite implements Serializable { 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) throws Exception { + public Double call(Double v1, Double v2) { return v1 + v2; } }); @@ -844,7 +846,7 @@ public class JavaAPISuite implements Serializable { new PairFunction<Integer, Integer, Integer>() { @Override public Tuple2<Integer, Integer> call(Integer x) { - return new Tuple2<Integer, Integer>(x, x); + return new Tuple2<>(x, x); } }).cache(); pairs.collect(); @@ -870,26 +872,25 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals("Hello", words.first()); Assert.assertEquals(11, words.count()); - JavaPairRDD<String, String> pairs = rdd.flatMapToPair( + JavaPairRDD<String, String> pairsRDD = rdd.flatMapToPair( new PairFlatMapFunction<String, String, String>() { - @Override public Iterable<Tuple2<String, String>> call(String s) { - List<Tuple2<String, String>> pairs = new LinkedList<Tuple2<String, String>>(); + List<Tuple2<String, String>> pairs = new LinkedList<>(); for (String word : s.split(" ")) { - pairs.add(new Tuple2<String, String>(word, word)); + pairs.add(new Tuple2<>(word, word)); } return pairs; } } ); - Assert.assertEquals(new Tuple2<String, String>("Hello", "Hello"), pairs.first()); - Assert.assertEquals(11, pairs.count()); + Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first()); + Assert.assertEquals(11, pairsRDD.count()); JavaDoubleRDD doubles = rdd.flatMapToDouble(new DoubleFlatMapFunction<String>() { @Override public Iterable<Double> call(String s) { - List<Double> lengths = new LinkedList<Double>(); + List<Double> lengths = new LinkedList<>(); for (String word : s.split(" ")) { lengths.add((double) word.length()); } @@ -897,36 +898,36 @@ public class JavaAPISuite implements Serializable { } }); Assert.assertEquals(5.0, doubles.first(), 0.01); - Assert.assertEquals(11, pairs.count()); + Assert.assertEquals(11, pairsRDD.count()); } @SuppressWarnings("unchecked") @Test public void mapsFromPairsToPairs() { - List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(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 Iterable<Tuple2<String, Integer>> call(Tuple2<Integer, String> item) { - return Collections.singletonList(item.swap()); - } + 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 Iterable<Tuple2<String, Integer>> call(Tuple2<Integer, String> item) { + return Collections.singletonList(item.swap()); + } }); - swapped.collect(); + 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(); + // 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 @@ -953,7 +954,7 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> partitionSums = rdd.mapPartitionsWithIndex( new Function2<Integer, Iterator<Integer>, Iterator<Integer>>() { @Override - public Iterator<Integer> call(Integer index, Iterator<Integer> iter) throws Exception { + public Iterator<Integer> call(Integer index, Iterator<Integer> iter) { int sum = 0; while (iter.hasNext()) { sum += iter.next(); @@ -972,8 +973,8 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> repartitioned1 = in1.repartition(4); List<List<Integer>> result1 = repartitioned1.glom().collect(); Assert.assertEquals(4, result1.size()); - for (List<Integer> l: result1) { - Assert.assertTrue(l.size() > 0); + for (List<Integer> l : result1) { + Assert.assertFalse(l.isEmpty()); } // Growing number of partitions @@ -982,7 +983,7 @@ public class JavaAPISuite implements Serializable { List<List<Integer>> result2 = repartitioned2.glom().collect(); Assert.assertEquals(2, result2.size()); for (List<Integer> l: result2) { - Assert.assertTrue(l.size() > 0); + Assert.assertFalse(l.isEmpty()); } } @@ -994,9 +995,9 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(20, doubleRDD.sum(), 0.1); List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(3, "aaa") + 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()); @@ -1046,7 +1047,7 @@ public class JavaAPISuite implements Serializable { Files.write(content1, new File(tempDirName + "/part-00000")); Files.write(content2, new File(tempDirName + "/part-00001")); - Map<String, String> container = new HashMap<String, String>(); + Map<String, String> container = new HashMap<>(); container.put(tempDirName+"/part-00000", new Text(content1).toString()); container.put(tempDirName+"/part-00001", new Text(content2).toString()); @@ -1075,16 +1076,16 @@ public class JavaAPISuite implements Serializable { public void sequenceFile() { String outputDir = new File(tempDir, "output").getAbsolutePath(); List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(3, "aaa") + 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<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2())); + return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); } }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); @@ -1093,7 +1094,7 @@ public class JavaAPISuite implements Serializable { Text.class).mapToPair(new PairFunction<Tuple2<IntWritable, Text>, Integer, String>() { @Override public Tuple2<Integer, String> call(Tuple2<IntWritable, Text> pair) { - return new Tuple2<Integer, String>(pair._1().get(), pair._2().toString()); + return new Tuple2<>(pair._1().get(), pair._2().toString()); } }); Assert.assertEquals(pairs, readRDD.collect()); @@ -1110,7 +1111,7 @@ public class JavaAPISuite implements Serializable { FileOutputStream fos1 = new FileOutputStream(file1); FileChannel channel1 = fos1.getChannel(); - ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + ByteBuffer bbuf = ByteBuffer.wrap(content1); channel1.write(bbuf); channel1.close(); JavaPairRDD<String, PortableDataStream> readRDD = sc.binaryFiles(tempDirName, 3); @@ -1131,14 +1132,14 @@ public class JavaAPISuite implements Serializable { FileOutputStream fos1 = new FileOutputStream(file1); FileChannel channel1 = fos1.getChannel(); - ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + 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) throws Exception { + public void call(Tuple2<String, PortableDataStream> pair) { pair._2().toArray(); // force the file to read } }); @@ -1162,7 +1163,7 @@ public class JavaAPISuite implements Serializable { FileChannel channel1 = fos1.getChannel(); for (int i = 0; i < numOfCopies; i++) { - ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + ByteBuffer bbuf = ByteBuffer.wrap(content1); channel1.write(bbuf); } channel1.close(); @@ -1180,24 +1181,23 @@ public class JavaAPISuite implements Serializable { public void writeWithNewAPIHadoopFile() { String outputDir = new File(tempDir, "output").getAbsolutePath(); List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(3, "aaa") + 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<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2())); + 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); + }).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); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, - String>() { + JavaPairRDD<IntWritable, Text> output = sc.sequenceFile(outputDir, IntWritable.class, Text.class); + Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1210,24 +1210,23 @@ public class JavaAPISuite implements Serializable { public void readWithNewAPIHadoopFile() throws IOException { String outputDir = new File(tempDir, "output").getAbsolutePath(); List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(3, "aaa") + 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<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2())); + 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, new Job().getConfiguration()); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, - String>() { + org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class, + IntWritable.class, Text.class, new Job().getConfiguration()); + Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1251,9 +1250,9 @@ public class JavaAPISuite implements Serializable { public void objectFilesOfComplexTypes() { String outputDir = new File(tempDir, "output").getAbsolutePath(); List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(3, "aaa") + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") ); JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs); rdd.saveAsObjectFile(outputDir); @@ -1267,23 +1266,22 @@ public class JavaAPISuite implements Serializable { public void hadoopFile() { String outputDir = new File(tempDir, "output").getAbsolutePath(); List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(3, "aaa") + 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<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2())); + 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); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, - String>() { + SequenceFileInputFormat.class, IntWritable.class, Text.class); + Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1296,16 +1294,16 @@ public class JavaAPISuite implements Serializable { public void hadoopFileCompressed() { String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); List<Tuple2<Integer, String>> pairs = Arrays.asList( - new Tuple2<Integer, String>(1, "a"), - new Tuple2<Integer, String>(2, "aa"), - new Tuple2<Integer, String>(3, "aaa") + 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<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2())); + return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); } }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, DefaultCodec.class); @@ -1313,8 +1311,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir, SequenceFileInputFormat.class, IntWritable.class, Text.class); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, - String>() { + Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1414,8 +1411,8 @@ public class JavaAPISuite implements Serializable { return t.toString(); } }).collect(); - Assert.assertEquals(new Tuple2<String, Integer>("1", 1), s.get(0)); - Assert.assertEquals(new Tuple2<String, Integer>("2", 2), s.get(1)); + Assert.assertEquals(new Tuple2<>("1", 1), s.get(0)); + Assert.assertEquals(new Tuple2<>("2", 2), s.get(1)); } @Test @@ -1448,20 +1445,20 @@ public class JavaAPISuite implements Serializable { 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) throws Exception { + public Integer call(Integer v1) { return v1 % 3; } }; Function<Integer, Integer> createCombinerFunction = new Function<Integer, Integer>() { @Override - public Integer call(Integer v1) throws Exception { + 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) throws Exception { + public Integer call(Integer v1, Integer v2) { return v1 + v2; } }; @@ -1496,21 +1493,21 @@ public class JavaAPISuite implements Serializable { new PairFunction<Integer, Integer, Integer>() { @Override public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<Integer, Integer>(i, i % 2); + 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<Integer, Integer>(in._2(), in._1()); - } - }); + @Override + public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> in) { + return new Tuple2<>(in._2(), in._1()); + } + }); Assert.assertEquals(Arrays.asList( - new Tuple2<Integer, Integer>(1, 1), - new Tuple2<Integer, Integer>(0, 2), - new Tuple2<Integer, Integer>(1, 3), - new Tuple2<Integer, Integer>(0, 4)), rdd3.collect()); + new Tuple2<>(1, 1), + new Tuple2<>(0, 2), + new Tuple2<>(1, 3), + new Tuple2<>(0, 4)), rdd3.collect()); } @@ -1523,7 +1520,7 @@ public class JavaAPISuite implements Serializable { new PairFunction<Integer, Integer, Integer>() { @Override public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<Integer, Integer>(i, i % 2); + return new Tuple2<>(i, i % 2); } }); @@ -1534,23 +1531,23 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(Arrays.asList(3, 4), parts[0]); Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); - Assert.assertEquals(Arrays.asList(new Tuple2<Integer, Integer>(1, 1), - new Tuple2<Integer, Integer>(2, 0)), + 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<Integer, Integer>(3, 1), - new Tuple2<Integer, Integer>(4, 0)), + Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), + new Tuple2<>(4, 0)), parts2[0]); - Assert.assertEquals(Arrays.asList(new Tuple2<Integer, Integer>(5, 1), - new Tuple2<Integer, Integer>(6, 0), - new Tuple2<Integer, Integer>(7, 1)), + Assert.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<Integer>(); + List<Integer> arrayData = new ArrayList<>(); int size = 100; for (int i = 0; i < 100000; i++) { arrayData.add(i % size); @@ -1561,15 +1558,15 @@ public class JavaAPISuite implements Serializable { @Test public void countApproxDistinctByKey() { - List<Tuple2<Integer, Integer>> arrayData = new ArrayList<Tuple2<Integer, Integer>>(); + 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<Integer, Integer>(i, j)); + arrayData.add(new Tuple2<>(i, j)); } } double relativeSD = 0.001; JavaPairRDD<Integer, Integer> pairRdd = sc.parallelizePairs(arrayData); - List<Tuple2<Integer, Object>> res = pairRdd.countApproxDistinctByKey(8, 0).collect(); + List<Tuple2<Integer, Object>> res = pairRdd.countApproxDistinctByKey(relativeSD, 8).collect(); for (Tuple2<Integer, Object> resItem : res) { double count = (double)resItem._1(); Long resCount = (Long)resItem._2(); @@ -1587,7 +1584,7 @@ public class JavaAPISuite implements Serializable { new PairFunction<Integer, Integer, int[]>() { @Override public Tuple2<Integer, int[]> call(Integer x) { - return new Tuple2<Integer, int[]>(x, new int[] { x }); + return new Tuple2<>(x, new int[]{x}); } }); pairRDD.collect(); // Works fine @@ -1598,7 +1595,7 @@ public class JavaAPISuite implements Serializable { @Test public void collectAsMapAndSerialize() throws Exception { JavaPairRDD<String,Integer> rdd = - sc.parallelizePairs(Arrays.asList(new Tuple2<String,Integer>("foo", 1))); + sc.parallelizePairs(Arrays.asList(new Tuple2<>("foo", 1))); Map<String,Integer> map = rdd.collectAsMap(); ByteArrayOutputStream bytes = new ByteArrayOutputStream(); new ObjectOutputStream(bytes).writeObject(map); @@ -1615,7 +1612,7 @@ public class JavaAPISuite implements Serializable { new PairFunction<Integer, Integer, Integer>() { @Override public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<Integer, Integer>(i % 2, 1); + return new Tuple2<>(i % 2, 1); } }); Map<Integer, Object> fractions = Maps.newHashMap(); @@ -1623,12 +1620,12 @@ public class JavaAPISuite implements Serializable { fractions.put(1, 1.0); JavaPairRDD<Integer, Integer> wr = rdd2.sampleByKey(true, fractions, 1L); Map<Integer, Long> wrCounts = (Map<Integer, Long>) (Object) wr.countByKey(); - Assert.assertTrue(wrCounts.size() == 2); + Assert.assertEquals(2, wrCounts.size()); Assert.assertTrue(wrCounts.get(0) > 0); Assert.assertTrue(wrCounts.get(1) > 0); JavaPairRDD<Integer, Integer> wor = rdd2.sampleByKey(false, fractions, 1L); Map<Integer, Long> worCounts = (Map<Integer, Long>) (Object) wor.countByKey(); - Assert.assertTrue(worCounts.size() == 2); + Assert.assertEquals(2, worCounts.size()); Assert.assertTrue(worCounts.get(0) > 0); Assert.assertTrue(worCounts.get(1) > 0); } @@ -1641,7 +1638,7 @@ public class JavaAPISuite implements Serializable { new PairFunction<Integer, Integer, Integer>() { @Override public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<Integer, Integer>(i % 2, 1); + return new Tuple2<>(i % 2, 1); } }); Map<Integer, Object> fractions = Maps.newHashMap(); @@ -1649,25 +1646,25 @@ public class JavaAPISuite implements Serializable { fractions.put(1, 1.0); JavaPairRDD<Integer, Integer> wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); Map<Integer, Long> wrExactCounts = (Map<Integer, Long>) (Object) wrExact.countByKey(); - Assert.assertTrue(wrExactCounts.size() == 2); + Assert.assertEquals(2, wrExactCounts.size()); Assert.assertTrue(wrExactCounts.get(0) == 2); Assert.assertTrue(wrExactCounts.get(1) == 4); JavaPairRDD<Integer, Integer> worExact = rdd2.sampleByKeyExact(false, fractions, 1L); Map<Integer, Long> worExactCounts = (Map<Integer, Long>) (Object) worExact.countByKey(); - Assert.assertTrue(worExactCounts.size() == 2); + Assert.assertEquals(2, worExactCounts.size()); Assert.assertTrue(worExactCounts.get(0) == 2); Assert.assertTrue(worExactCounts.get(1) == 4); } private static class SomeCustomClass implements Serializable { - public SomeCustomClass() { + SomeCustomClass() { // Intentionally left blank } } @Test public void collectUnderlyingScalaRDD() { - List<SomeCustomClass> data = new ArrayList<SomeCustomClass>(); + List<SomeCustomClass> data = new ArrayList<>(); for (int i = 0; i < 100; i++) { data.add(new SomeCustomClass()); } @@ -1679,7 +1676,7 @@ public class JavaAPISuite implements Serializable { private static final class BuggyMapFunction<T> implements Function<T, T> { @Override - public T call(T x) throws Exception { + public T call(T x) { throw new IllegalStateException("Custom exception!"); } } @@ -1716,7 +1713,7 @@ public class JavaAPISuite implements Serializable { JavaFutureAction<Void> future = rdd.foreachAsync( new VoidFunction<Integer>() { @Override - public void call(Integer integer) throws Exception { + public void call(Integer integer) { // intentionally left blank. } } @@ -1745,7 +1742,7 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> rdd = sc.parallelize(data, 1); JavaFutureAction<Void> future = rdd.foreachAsync(new VoidFunction<Integer>() { @Override - public void call(Integer integer) throws Exception { + public void call(Integer integer) throws InterruptedException { Thread.sleep(10000); // To ensure that the job won't finish before it's cancelled. } }); |