Skip to content

Commit

Permalink
maybe I should have had a coffee first. use correct import for guava …
Browse files Browse the repository at this point in the history
…iterables
  • Loading branch information
holdenk committed Apr 8, 2014
1 parent c5075aa commit eb06216
Show file tree
Hide file tree
Showing 3 changed files with 10 additions and 21 deletions.
27 changes: 8 additions & 19 deletions core/src/test/java/org/apache/spark/JavaAPISuite.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@

import scala.Tuple2;

import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.collect.Iterables;
import com.google.common.base.Optional;
import com.google.common.base.Charsets;
import com.google.common.io.Files;
Expand Down Expand Up @@ -79,17 +79,6 @@ public int compare(Integer a, Integer b) {
}
}


private int iterableSize(Iterable<?> a) {
return Iterables.size(a.iterator());
}


private String iterableStr(Iterable<?> a) {
return Iterables.toString(a.iterator());
}


@SuppressWarnings("unchecked")
@Test
public void sparkContextUnion() {
Expand Down Expand Up @@ -210,7 +199,7 @@ public void lookup() {
new Tuple2<String, String>("Oranges", "Citrus")
));
Assert.assertEquals(2, categories.lookup("Oranges").size());
Assert.assertEquals(2, iterableSize(categories.groupByKey().lookup("Oranges").get(0)));
Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0)));
}

@Test
Expand All @@ -224,13 +213,13 @@ public Boolean call(Integer x) {
};
JavaPairRDD<Boolean, Iterable<Integer>> oddsAndEvens = rdd.groupBy(isOdd);
Assert.assertEquals(2, oddsAndEvens.count());
Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens
Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds
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, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens
Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds
Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens
Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds
}

@SuppressWarnings("unchecked")
Expand All @@ -246,8 +235,8 @@ public void cogroup() {
new Tuple2<String, Integer>("Apples", 3)
));
JavaPairRDD<String, Tuple2<Iterable<String>, Iterable<Integer>>> cogrouped = categories.cogroup(prices);
Assert.assertEquals("[Fruit, Citrus]", iterableStr(cogrouped.lookup("Oranges").get(0)._1()));
Assert.assertEquals("[2]", iterableStr(cogrouped.lookup("Oranges").get(0)._2()));
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();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import scala.Tuple2;

import com.google.collect.Iterables;
import com.google.common.collect.Iterables;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@

import scala.Tuple2;

import com.google.collections.Iterables;
import com.google.common.collections.Iterables;
import com.google.common.base.Optional;
import com.google.common.io.Files;
import org.apache.hadoop.io.IntWritable;
Expand Down

0 comments on commit eb06216

Please sign in to comment.