Skip to content

Commit

Permalink
Suppress warnings about legitimate unchecked array creations, or chan…
Browse files Browse the repository at this point in the history
…ge code to avoid it
  • Loading branch information
srowen committed Feb 27, 2014
1 parent f35b833 commit 6c4a32c
Show file tree
Hide file tree
Showing 2 changed files with 112 additions and 47 deletions.
35 changes: 25 additions & 10 deletions core/src/test/java/org/apache/spark/JavaAPISuite.java
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -148,6 +149,7 @@ public void call(String s) {
Assert.assertEquals(2, foreachCalls);
}

@SuppressWarnings("unchecked")
@Test
public void lookup() {
JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
Expand Down Expand Up @@ -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<String, String> categories = sc.parallelizePairs(Arrays.asList(
Expand All @@ -197,6 +200,7 @@ public void cogroup() {
cogrouped.collect();
}

@SuppressWarnings("unchecked")
@Test
public void leftOuterJoin() {
JavaPairRDD<Integer, Integer> rdd1 = sc.parallelizePairs(Arrays.asList(
Expand Down Expand Up @@ -243,6 +247,7 @@ public Integer call(Integer a, Integer b) {
Assert.assertEquals(33, sum);
}

@SuppressWarnings("unchecked")
@Test
public void foldByKey() {
List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
Expand All @@ -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<Tuple2<Integer, Integer>> pairs = Arrays.asList(
Expand Down Expand Up @@ -320,8 +326,8 @@ public void approximateResults() {
public void take() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
Assert.assertEquals(1, rdd.first().intValue());
List<Integer> firstTwo = rdd.take(2);
List<Integer> sample = rdd.takeSample(false, 2, 42);
rdd.take(2);
rdd.takeSample(false, 2, 42);
}

@Test
Expand Down Expand Up @@ -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<Double> take = rdd.take(5);
rdd.first();
rdd.take(5);
}

@Test
Expand Down Expand Up @@ -438,11 +444,11 @@ public Iterable<Double> 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<Tuple2<Integer, String>> pairs = Arrays.asList(
Expand Down Expand Up @@ -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));
Expand Down Expand Up @@ -573,6 +580,7 @@ public void textFilesCompressed() throws IOException {
Assert.assertEquals(expected, readRDD.collect());
}

@SuppressWarnings("unchecked")
@Test
public void sequenceFile() {
File tempDir = Files.createTempDir();
Expand Down Expand Up @@ -602,6 +610,7 @@ public Tuple2<Integer, String> call(Tuple2<IntWritable, Text> pair) {
Assert.assertEquals(pairs, readRDD.collect());
}

@SuppressWarnings("unchecked")
@Test
public void writeWithNewAPIHadoopFile() {
File tempDir = Files.createTempDir();
Expand Down Expand Up @@ -632,6 +641,7 @@ public String call(Tuple2<IntWritable, Text> x) {
}).collect().toString());
}

@SuppressWarnings("unchecked")
@Test
public void readWithNewAPIHadoopFile() throws IOException {
File tempDir = Files.createTempDir();
Expand Down Expand Up @@ -674,6 +684,7 @@ public void objectFilesOfInts() {
Assert.assertEquals(expected, readRDD.collect());
}

@SuppressWarnings("unchecked")
@Test
public void objectFilesOfComplexTypes() {
File tempDir = Files.createTempDir();
Expand All @@ -690,6 +701,7 @@ public void objectFilesOfComplexTypes() {
Assert.assertEquals(pairs, readRDD.collect());
}

@SuppressWarnings("unchecked")
@Test
public void hadoopFile() {
File tempDir = Files.createTempDir();
Expand Down Expand Up @@ -719,6 +731,7 @@ public String call(Tuple2<IntWritable, Text> x) {
}).collect().toString());
}

@SuppressWarnings("unchecked")
@Test
public void hadoopFileCompressed() {
File tempDir = Files.createTempDir();
Expand Down Expand Up @@ -824,7 +837,7 @@ public Float zero(Float initialValue) {
}
};

final Accumulator<Float> floatAccum = sc.accumulator((Float) 10.0f, floatAccumulatorParam);
final Accumulator<Float> floatAccum = sc.accumulator(10.0f, floatAccumulatorParam);
rdd.foreach(new VoidFunction<Integer>() {
public void call(Integer x) {
floatAccum.add((float) x);
Expand Down Expand Up @@ -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<Integer> rdd1 = sc.parallelize(Arrays.asList(1,2,3,4));
Expand All @@ -900,6 +914,7 @@ public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> in) throws Excepti

}

@SuppressWarnings("unchecked")
@Test
public void collectPartitions() {
JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3);
Expand Down Expand Up @@ -968,14 +983,14 @@ public void countApproxDistinctByKey() {
@Test
public void collectAsMapWithIntArrayValues() {
// Regression test for SPARK-1040
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(new Integer[] { 1 }));
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1));
JavaPairRDD<Integer, int[]> pairRDD = rdd.map(new PairFunction<Integer, Integer, int[]>() {
@Override
public Tuple2<Integer, int[]> call(Integer x) throws Exception {
return new Tuple2<Integer, int[]>(x, new int[] { x });
}
});
pairRDD.collect(); // Works fine
Map<Integer, int[]> map = pairRDD.collectAsMap(); // Used to crash with ClassCastException
pairRDD.collectAsMap(); // Used to crash with ClassCastException
}
}
Loading

0 comments on commit 6c4a32c

Please sign in to comment.