From 6c4a32c2e2b9bc57d249bb72fad9fc2adc12cb69 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 27 Feb 2014 08:05:54 +0000 Subject: [PATCH] Suppress warnings about legitimate unchecked array creations, or change code to avoid it --- .../java/org/apache/spark/JavaAPISuite.java | 35 +++-- .../apache/spark/streaming/JavaAPISuite.java | 124 ++++++++++++------ 2 files changed, 112 insertions(+), 47 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 20232e9fbb8d0..aa5079c159830 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -75,8 +75,9 @@ public int compare(Integer a, Integer b) { else if (a < b) return 1; else return 0; } - }; + } + @SuppressWarnings("unchecked") @Test public void sparkContextUnion() { // Union of non-specialized JavaRDDs @@ -148,6 +149,7 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } + @SuppressWarnings("unchecked") @Test public void lookup() { JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( @@ -179,6 +181,7 @@ public Boolean call(Integer x) { Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds } + @SuppressWarnings("unchecked") @Test public void cogroup() { JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( @@ -197,6 +200,7 @@ public void cogroup() { cogrouped.collect(); } + @SuppressWarnings("unchecked") @Test public void leftOuterJoin() { JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( @@ -243,6 +247,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(33, sum); } + @SuppressWarnings("unchecked") @Test public void foldByKey() { List> pairs = Arrays.asList( @@ -265,6 +270,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); } + @SuppressWarnings("unchecked") @Test public void reduceByKey() { List> pairs = Arrays.asList( @@ -320,8 +326,8 @@ public void approximateResults() { public void take() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Assert.assertEquals(1, rdd.first().intValue()); - List firstTwo = rdd.take(2); - List sample = rdd.takeSample(false, 2, 42); + rdd.take(2); + rdd.takeSample(false, 2, 42); } @Test @@ -359,8 +365,8 @@ public Boolean call(Double x) { Assert.assertEquals(2.49444, rdd.stdev(), 0.01); Assert.assertEquals(2.73252, rdd.sampleStdev(), 0.01); - Double first = rdd.first(); - List take = rdd.take(5); + rdd.first(); + rdd.take(5); } @Test @@ -438,11 +444,11 @@ public Iterable call(String s) { return lengths; } }); - Double x = doubles.first(); - Assert.assertEquals(5.0, doubles.first().doubleValue(), 0.01); + Assert.assertEquals(5.0, doubles.first(), 0.01); Assert.assertEquals(11, pairs.count()); } + @SuppressWarnings("unchecked") @Test public void mapsFromPairsToPairs() { List> pairs = Arrays.asList( @@ -509,6 +515,7 @@ public void repartition() { } } + @SuppressWarnings("unchecked") @Test public void persist() { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); @@ -573,6 +580,7 @@ public void textFilesCompressed() throws IOException { Assert.assertEquals(expected, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void sequenceFile() { File tempDir = Files.createTempDir(); @@ -602,6 +610,7 @@ public Tuple2 call(Tuple2 pair) { Assert.assertEquals(pairs, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { File tempDir = Files.createTempDir(); @@ -632,6 +641,7 @@ public String call(Tuple2 x) { }).collect().toString()); } + @SuppressWarnings("unchecked") @Test public void readWithNewAPIHadoopFile() throws IOException { File tempDir = Files.createTempDir(); @@ -674,6 +684,7 @@ public void objectFilesOfInts() { Assert.assertEquals(expected, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void objectFilesOfComplexTypes() { File tempDir = Files.createTempDir(); @@ -690,6 +701,7 @@ public void objectFilesOfComplexTypes() { Assert.assertEquals(pairs, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void hadoopFile() { File tempDir = Files.createTempDir(); @@ -719,6 +731,7 @@ public String call(Tuple2 x) { }).collect().toString()); } + @SuppressWarnings("unchecked") @Test public void hadoopFileCompressed() { File tempDir = Files.createTempDir(); @@ -824,7 +837,7 @@ public Float zero(Float initialValue) { } }; - final Accumulator floatAccum = sc.accumulator((Float) 10.0f, floatAccumulatorParam); + final Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); rdd.foreach(new VoidFunction() { public void call(Integer x) { floatAccum.add((float) x); @@ -876,6 +889,7 @@ public void checkpointAndRestore() { Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); } + @SuppressWarnings("unchecked") @Test public void mapOnPairRDD() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); @@ -900,6 +914,7 @@ public Tuple2 call(Tuple2 in) throws Excepti } + @SuppressWarnings("unchecked") @Test public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); @@ -968,7 +983,7 @@ public void countApproxDistinctByKey() { @Test public void collectAsMapWithIntArrayValues() { // Regression test for SPARK-1040 - JavaRDD rdd = sc.parallelize(Arrays.asList(new Integer[] { 1 })); + JavaRDD rdd = sc.parallelize(Arrays.asList(1)); JavaPairRDD pairRDD = rdd.map(new PairFunction() { @Override public Tuple2 call(Integer x) throws Exception { @@ -976,6 +991,6 @@ public Tuple2 call(Integer x) throws Exception { } }); pairRDD.collect(); // Works fine - Map map = pairRDD.collectAsMap(); // Used to crash with ClassCastException + pairRDD.collectAsMap(); // Used to crash with ClassCastException } } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 4fbbce9b8b90e..54a0791d04ea4 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -19,7 +19,6 @@ import scala.Tuple2; -import org.junit.After; import org.junit.Assert; import org.junit.Test; import java.io.*; @@ -30,7 +29,6 @@ import com.google.common.io.Files; import com.google.common.collect.Sets; -import org.apache.spark.SparkConf; import org.apache.spark.HashPartitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -38,6 +36,7 @@ import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaDStreamLike; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; @@ -45,6 +44,8 @@ // 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 { + + @SuppressWarnings("unchecked") @Test public void testCount() { List> inputData = Arrays.asList( @@ -64,6 +65,7 @@ public void testCount() { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testMap() { List> inputData = Arrays.asList( @@ -87,6 +89,7 @@ public Integer call(String s) throws Exception { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testWindow() { List> inputData = Arrays.asList( @@ -108,6 +111,7 @@ public void testWindow() { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testWindowWithSlideDuration() { List> inputData = Arrays.asList( @@ -132,6 +136,7 @@ public void testWindowWithSlideDuration() { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testFilter() { List> inputData = Arrays.asList( @@ -155,13 +160,16 @@ public Boolean call(String s) throws Exception { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testRepartitionMorePartitions() { List> 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 stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 2); - JavaDStream repartitioned = stream.repartition(4); + JavaDStream stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 2); + JavaDStreamLike,JavaRDD> repartitioned = + stream.repartition(4); JavaTestUtils.attachTestOutputStream(repartitioned); List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); Assert.assertEquals(2, result.size()); @@ -172,13 +180,16 @@ public void testRepartitionMorePartitions() { } } + @SuppressWarnings("unchecked") @Test public void testRepartitionFewerPartitions() { List> 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 stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 4); - JavaDStream repartitioned = stream.repartition(2); + JavaDStream stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 4); + JavaDStreamLike,JavaRDD> repartitioned = + stream.repartition(2); JavaTestUtils.attachTestOutputStream(repartitioned); List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); Assert.assertEquals(2, result.size()); @@ -188,6 +199,7 @@ public void testRepartitionFewerPartitions() { } } + @SuppressWarnings("unchecked") @Test public void testGlom() { List> inputData = Arrays.asList( @@ -206,6 +218,7 @@ public void testGlom() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testMapPartitions() { List> inputData = Arrays.asList( @@ -217,16 +230,17 @@ public void testMapPartitions() { Arrays.asList("YANKEESRED SOCKS")); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream mapped = stream.mapPartitions(new FlatMapFunction, String>() { - @Override - public Iterable call(Iterator in) { - String out = ""; - while (in.hasNext()) { - out = out + in.next().toUpperCase(); - } - return Lists.newArrayList(out); - } - }); + JavaDStream mapped = stream.mapPartitions( + new FlatMapFunction, String>() { + @Override + public Iterable call(Iterator in) { + String out = ""; + while (in.hasNext()) { + out = out + in.next().toUpperCase(); + } + return Lists.newArrayList(out); + } + }); JavaTestUtils.attachTestOutputStream(mapped); List> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -247,6 +261,7 @@ public Integer call(Integer i1, Integer i2) throws Exception { } } + @SuppressWarnings("unchecked") @Test public void testReduce() { List> inputData = Arrays.asList( @@ -267,6 +282,7 @@ public void testReduce() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testReduceByWindow() { List> inputData = Arrays.asList( @@ -289,6 +305,7 @@ public void testReduceByWindow() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testQueueStream() { List> expected = Arrays.asList( @@ -312,6 +329,7 @@ public void testQueueStream() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testTransform() { List> inputData = Arrays.asList( @@ -344,6 +362,7 @@ public Integer call(Integer i) throws Exception { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testVariousTransform() { // tests whether all variations of transform can be called from Java @@ -423,6 +442,7 @@ public JavaRDD call(JavaRDD in) throws Exception { } + @SuppressWarnings("unchecked") @Test public void testTransformWith() { List>> stringStringKVStream1 = Arrays.asList( @@ -492,6 +512,7 @@ public JavaPairRDD> call( } + @SuppressWarnings("unchecked") @Test public void testVariousTransformWith() { // tests whether all variations of transformWith can be called from Java @@ -591,6 +612,7 @@ public JavaPairRDD call(JavaPairRDD rdd1, JavaP ); } + @SuppressWarnings("unchecked") @Test public void testStreamingContextTransform(){ List> stream1input = Arrays.asList( @@ -658,6 +680,7 @@ public Tuple2 call(Integer i) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testFlatMap() { List> inputData = Arrays.asList( @@ -683,6 +706,7 @@ public Iterable call(String x) { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairFlatMap() { List> inputData = Arrays.asList( @@ -718,22 +742,24 @@ public void testPairFlatMap() { new Tuple2(9, "s"))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { - @Override - public Iterable> call(String in) throws Exception { - List> out = Lists.newArrayList(); - for (String letter: in.split("(?!^)")) { - out.add(new Tuple2(in.length(), letter)); - } - return out; - } - }); + JavaPairDStream flatMapped = stream.flatMap( + new PairFlatMapFunction() { + @Override + public Iterable> call(String in) throws Exception { + List> out = Lists.newArrayList(); + for (String letter: in.split("(?!^)")) { + out.add(new Tuple2(in.length(), letter)); + } + return out; + } + }); JavaTestUtils.attachTestOutputStream(flatMapped); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testUnion() { List> inputData1 = Arrays.asList( @@ -778,6 +804,7 @@ public static > void assertOrderInvariantEquals( // PairDStream Functions + @SuppressWarnings("unchecked") @Test public void testPairFilter() { List> inputData = Arrays.asList( @@ -810,7 +837,8 @@ public Boolean call(Tuple2 in) throws Exception { Assert.assertEquals(expected, result); } - List>> stringStringKVStream = Arrays.asList( + @SuppressWarnings("unchecked") + private List>> stringStringKVStream = Arrays.asList( Arrays.asList(new Tuple2("california", "dodgers"), new Tuple2("california", "giants"), new Tuple2("new york", "yankees"), @@ -820,7 +848,8 @@ public Boolean call(Tuple2 in) throws Exception { new Tuple2("new york", "rangers"), new Tuple2("new york", "islanders"))); - List>> stringIntKVStream = Arrays.asList( + @SuppressWarnings("unchecked") + private List>> stringIntKVStream = Arrays.asList( Arrays.asList( new Tuple2("california", 1), new Tuple2("california", 3), @@ -832,6 +861,7 @@ public Boolean call(Tuple2 in) throws Exception { new Tuple2("new york", 3), new Tuple2("new york", 1))); + @SuppressWarnings("unchecked") @Test public void testPairMap() { // Maps pair -> pair of different type List>> inputData = stringIntKVStream; @@ -864,6 +894,7 @@ public Tuple2 call(Tuple2 in) throws Exception Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairMapPartitions() { // Maps pair -> pair of different type List>> inputData = stringIntKVStream; @@ -901,6 +932,7 @@ public Iterable> call(Iterator> Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairMap2() { // Maps pair -> single List>> inputData = stringIntKVStream; @@ -925,6 +957,7 @@ public Integer call(Tuple2 in) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair List>> inputData = Arrays.asList( @@ -967,6 +1000,7 @@ public Iterable> call(Tuple2 in) throws Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairGroupByKey() { List>> inputData = stringStringKVStream; @@ -989,6 +1023,7 @@ public void testPairGroupByKey() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairReduceByKey() { List>> inputData = stringIntKVStream; @@ -1013,6 +1048,7 @@ public void testPairReduceByKey() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCombineByKey() { List>> inputData = stringIntKVStream; @@ -1043,6 +1079,7 @@ public Integer call(Integer i) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCountByValue() { List> inputData = Arrays.asList( @@ -1068,6 +1105,7 @@ public void testCountByValue() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testGroupByKeyAndWindow() { List>> inputData = stringIntKVStream; @@ -1113,6 +1151,7 @@ private Tuple2> convert(Tuple2> t return new Tuple2>(tuple._1(), new HashSet(tuple._2())); } + @SuppressWarnings("unchecked") @Test public void testReduceByKeyAndWindow() { List>> inputData = stringIntKVStream; @@ -1136,6 +1175,7 @@ public void testReduceByKeyAndWindow() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testUpdateStateByKey() { List>> inputData = stringIntKVStream; @@ -1171,6 +1211,7 @@ public Optional call(List values, Optional state) { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testReduceByKeyAndWindowWithInverse() { List>> inputData = stringIntKVStream; @@ -1194,6 +1235,7 @@ public void testReduceByKeyAndWindowWithInverse() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCountByValueAndWindow() { List> inputData = Arrays.asList( @@ -1227,6 +1269,7 @@ public void testCountByValueAndWindow() { Assert.assertEquals(expected, unorderedResult); } + @SuppressWarnings("unchecked") @Test public void testPairTransform() { List>> inputData = Arrays.asList( @@ -1271,6 +1314,7 @@ public JavaPairRDD call(JavaPairRDD in) thro Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairToNormalRDDTransform() { List>> inputData = Arrays.asList( @@ -1312,6 +1356,8 @@ public Integer call(Tuple2 in) { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") + @Test public void testMapValues() { List>> inputData = stringStringKVStream; @@ -1342,6 +1388,7 @@ public String call(String s) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testFlatMapValues() { List>> inputData = stringStringKVStream; @@ -1386,6 +1433,7 @@ public Iterable call(String in) { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCoGroup() { List>> stringStringKVStream1 = Arrays.asList( @@ -1429,6 +1477,7 @@ public void testCoGroup() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testJoin() { List>> stringStringKVStream1 = Arrays.asList( @@ -1472,6 +1521,7 @@ public void testJoin() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testLeftOuterJoin() { List>> stringStringKVStream1 = Arrays.asList( @@ -1503,6 +1553,7 @@ public void testLeftOuterJoin() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCheckpointMasterRecovery() throws InterruptedException { List> inputData = Arrays.asList( @@ -1541,7 +1592,8 @@ public Integer call(String s) throws Exception { } - /** TEST DISABLED: Pending a discussion about checkpoint() semantics with TD + /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD + @SuppressWarnings("unchecked") @Test public void testCheckpointofIndividualStream() throws InterruptedException { List> inputData = Arrays.asList( @@ -1581,16 +1633,14 @@ public void testSocketTextStream() { @Test public void testSocketString() { class Converter extends Function> { - public Iterable call(InputStream in) { + public Iterable call(InputStream in) throws IOException { BufferedReader reader = new BufferedReader(new InputStreamReader(in)); List out = new ArrayList(); - try { - while (true) { - String line = reader.readLine(); - if (line == null) { break; } - out.add(line); - } - } catch (IOException e) { } + while (true) { + String line = reader.readLine(); + if (line == null) { break; } + out.add(line); + } return out; } }