From 36ddeb7bf83ac5a1af9d3db07ad4c380777e4d1a Mon Sep 17 00:00:00 2001 From: Soumitra Kumar Date: Wed, 12 Nov 2014 12:25:31 -0800 Subject: [SPARK-3660][STREAMING] Initial RDD for updateStateByKey transformation SPARK-3660 : Initial RDD for updateStateByKey transformation I have added a sample StatefulNetworkWordCountWithInitial inspired by StatefulNetworkWordCount. Please let me know if any changes are required. Author: Soumitra Kumar Closes #2665 from soumitrak/master and squashes the following commits: ee8980b [Soumitra Kumar] Fixed copy/paste issue. 304f636 [Soumitra Kumar] Added simpler version of updateStateByKey API with initialRDD and test. 9781135 [Soumitra Kumar] Fixed test, and renamed variable. 3da51a2 [Soumitra Kumar] Adding updateStateByKey with initialRDD API to JavaPairDStream. 2f78f7e [Soumitra Kumar] Merge remote-tracking branch 'upstream/master' d4fdd18 [Soumitra Kumar] Renamed variable and moved method. d0ce2cd [Soumitra Kumar] Merge remote-tracking branch 'upstream/master' 31399a4 [Soumitra Kumar] Merge remote-tracking branch 'upstream/master' 4efa58b [Soumitra Kumar] [SPARK-3660][STREAMING] Initial RDD for updateStateByKey transformation 8f40ca0 [Soumitra Kumar] Merge remote-tracking branch 'upstream/master' dde4271 [Soumitra Kumar] Merge remote-tracking branch 'upstream/master' fdd7db3 [Soumitra Kumar] Adding support of initial value for state update. SPARK-3660 : Initial RDD for updateStateByKey transformation --- .../org/apache/spark/streaming/JavaAPISuite.java | 53 ++++++++++++++++++++-- 1 file changed, 49 insertions(+), 4 deletions(-) (limited to 'streaming/src/test/java') 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 4efeb8dfbe..ce645fccba 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -806,15 +806,17 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa * Performs an order-invariant comparison of lists representing two RDD streams. This allows * us to account for ordering variation within individual RDD's which occurs during windowing. */ - public static > void assertOrderInvariantEquals( + public static void assertOrderInvariantEquals( List> expected, List> actual) { + List> expectedSets = new ArrayList>(); for (List list: expected) { - Collections.sort(list); + expectedSets.add(Collections.unmodifiableSet(new HashSet(list))); } + List> actualSets = new ArrayList>(); for (List list: actual) { - Collections.sort(list); + actualSets.add(Collections.unmodifiableSet(new HashSet(list))); } - Assert.assertEquals(expected, actual); + Assert.assertEquals(expectedSets, actualSets); } @@ -1239,6 +1241,49 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") + @Test + public void testUpdateStateByKeyWithInitial() { + List>> inputData = stringIntKVStream; + + List> initial = Arrays.asList ( + new Tuple2 ("california", 1), + new Tuple2 ("new york", 2)); + + JavaRDD> tmpRDD = ssc.sparkContext().parallelize(initial); + JavaPairRDD initialRDD = JavaPairRDD.fromJavaRDD (tmpRDD); + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2("california", 5), + new Tuple2("new york", 7)), + Arrays.asList(new Tuple2("california", 15), + new Tuple2("new york", 11)), + Arrays.asList(new Tuple2("california", 15), + new Tuple2("new york", 11))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream updated = pairStream.updateStateByKey( + new Function2, Optional, Optional>() { + @Override + public Optional call(List values, Optional state) { + int out = 0; + if (state.isPresent()) { + out = out + state.get(); + } + for (Integer v: values) { + out = out + v; + } + return Optional.of(out); + } + }, new HashPartitioner(1), initialRDD); + JavaTestUtils.attachTestOutputStream(updated); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + @SuppressWarnings("unchecked") @Test public void testReduceByKeyAndWindowWithInverse() { -- cgit v1.2.3