diff options
author | Matei Zaharia <matei@eecs.berkeley.edu> | 2012-12-24 15:28:05 -0800 |
---|---|---|
committer | Matei Zaharia <matei@eecs.berkeley.edu> | 2012-12-24 15:28:05 -0800 |
commit | 84587a9bf3c734be151251b97ac5af48eb03f4d9 (patch) | |
tree | 00c7bd83e157fde54435b87c59dc99e7131ecf13 /core | |
parent | b575cbe0691df5d17c84acfaa36ea82b74324222 (diff) | |
parent | 903f3518dfcd686cda2256b07fbc1dde6aec0178 (diff) | |
download | spark-84587a9bf3c734be151251b97ac5af48eb03f4d9.tar.gz spark-84587a9bf3c734be151251b97ac5af48eb03f4d9.tar.bz2 spark-84587a9bf3c734be151251b97ac5af48eb03f4d9.zip |
Merge pull request #343 from markhamstra/spark-601
lookup() needn't fail when there is no partitioner
Diffstat (limited to 'core')
-rw-r--r-- | core/src/main/scala/spark/PairRDDFunctions.scala | 2 | ||||
-rw-r--r-- | core/src/test/scala/spark/JavaAPISuite.java | 11 |
2 files changed, 12 insertions, 1 deletions
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 08ae06e865..d3e206b353 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -438,7 +438,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val res = self.context.runJob(self, process _, Array(index), false) res(0) case None => - throw new UnsupportedOperationException("lookup() called on an RDD without a partitioner") + self.filter(_._1 == key).map(_._2).collect } } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 46a0b68f89..33d5fc2d89 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -131,6 +131,17 @@ 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") + )); + Assert.assertEquals(2, categories.lookup("Oranges").size()); + Assert.assertEquals(2, categories.groupByKey().lookup("Oranges").get(0).size()); + } + + @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>() { |