aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/test/java
diff options
context:
space:
mode:
authorAaron Davidson <aaron@databricks.com>2013-12-26 16:19:25 -0800
committerAndrew Or <andrewor14@gmail.com>2013-12-26 23:40:07 -0800
commit1ffe26c7c03b5d128952e7d3ea7f130cd242a468 (patch)
tree50846950251baa9238b2c88ef2dc86819c4887bb /streaming/src/test/java
parent0289eb752abfc71ac0cc6796b57f1d21603cfd90 (diff)
downloadspark-1ffe26c7c03b5d128952e7d3ea7f130cd242a468.tar.gz
spark-1ffe26c7c03b5d128952e7d3ea7f130cd242a468.tar.bz2
spark-1ffe26c7c03b5d128952e7d3ea7f130cd242a468.zip
Fix streaming JavaAPISuite that depended on order
Diffstat (limited to 'streaming/src/test/java')
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java27
1 files changed, 16 insertions, 11 deletions
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 daeb99f5b7..bf23469936 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -20,6 +20,7 @@ package org.apache.spark.streaming;
import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
import com.google.common.io.Files;
import kafka.serializer.StringDecoder;
@@ -473,13 +474,13 @@ public class JavaAPISuite implements Serializable {
new Tuple2<String, String>("new york", "islanders")));
- List<List<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
- Arrays.asList(
- new Tuple2<String, Tuple2<String, String>>("california",
- new Tuple2<String, String>("dodgers", "giants")),
- new Tuple2<String, Tuple2<String, String>>("new york",
- new Tuple2<String, String>("yankees", "mets"))),
- Arrays.asList(
+ List<HashSet<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
+ Sets.newHashSet(
+ new Tuple2<String, Tuple2<String, String>>("california",
+ new Tuple2<String, String>("dodgers", "giants")),
+ new Tuple2<String, Tuple2<String, String>>("new york",
+ new Tuple2<String, String>("yankees", "mets"))),
+ Sets.newHashSet(
new Tuple2<String, Tuple2<String, String>>("california",
new Tuple2<String, String>("sharks", "ducks")),
new Tuple2<String, Tuple2<String, String>>("new york",
@@ -514,8 +515,12 @@ public class JavaAPISuite implements Serializable {
JavaTestUtils.attachTestOutputStream(joined);
List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+ List<HashSet<Tuple2<String, Tuple2<String, String>>>> unorderedResult = Lists.newArrayList();
+ for (List<Tuple2<String, Tuple2<String, String>>> res: result) {
+ unorderedResult.add(Sets.newHashSet(res));
+ }
- Assert.assertEquals(expected, result);
+ Assert.assertEquals(expected, unorderedResult);
}
@@ -1230,11 +1235,11 @@ public class JavaAPISuite implements Serializable {
List<List<Tuple2<String, Long>>> expected = Arrays.asList(
Arrays.asList(
- new Tuple2<String, Long>("hello", 1L),
- new Tuple2<String, Long>("world", 1L)),
+ new Tuple2<String, Long>("world", 1L),
+ new Tuple2<String, Long>("hello", 1L)),
Arrays.asList(
- new Tuple2<String, Long>("hello", 2L),
new Tuple2<String, Long>("world", 1L),
+ new Tuple2<String, Long>("hello", 2L),
new Tuple2<String, Long>("moon", 1L)),
Arrays.asList(
new Tuple2<String, Long>("hello", 2L),