-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-3660][STREAMING] Initial RDD for updateStateByKey transformation #2665
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
fdd7db3
dde4271
8f40ca0
4efa58b
31399a4
d0ce2cd
d4fdd18
2f78f7e
3da51a2
9781135
304f636
ee8980b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -492,6 +492,25 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( | |
| dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), partitioner) | ||
| } | ||
|
|
||
| /** | ||
| * Return a new "state" DStream where the state for each key is updated by applying | ||
| * the given function on the previous state of the key and the new values of the key. | ||
| * org.apache.spark.Partitioner is used to control the partitioning of each RDD. | ||
| * @param updateFunc State update function. If `this` function returns None, then | ||
| * corresponding state key-value pair will be eliminated. | ||
| * @param partitioner Partitioner for controlling the partitioning of each RDD in the new | ||
| * DStream. | ||
| * @param initialRDD initial state value of each key. | ||
| * @tparam S State type | ||
| */ | ||
| def updateStateByKey[S]( | ||
| updateFunc: JFunction2[JList[V], Optional[S], Optional[S]], | ||
| partitioner: Partitioner, | ||
| initialRDD: JavaPairRDD[K, S] | ||
| ): JavaPairDStream[K, S] = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. two space indent here. |
||
| implicit val cm: ClassTag[S] = fakeClassTag | ||
| dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), partitioner, initialRDD) | ||
| } | ||
|
|
||
| /** | ||
| * Return a new DStream by applying a map function to the value of each key-value pairs in | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -806,15 +806,17 @@ public void testUnion() { | |
| * 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 <T extends Comparable<T>> void assertOrderInvariantEquals( | ||
| public static <T> void assertOrderInvariantEquals( | ||
| List<List<T>> expected, List<List<T>> actual) { | ||
| List<Set<T>> expectedSets = new ArrayList<Set<T>>(); | ||
| for (List<T> list: expected) { | ||
| Collections.sort(list); | ||
| expectedSets.add(Collections.unmodifiableSet(new HashSet<T>(list))); | ||
| } | ||
| List<Set<T>> actualSets = new ArrayList<Set<T>>(); | ||
| for (List<T> list: actual) { | ||
| Collections.sort(list); | ||
| actualSets.add(Collections.unmodifiableSet(new HashSet<T>(list))); | ||
| } | ||
| Assert.assertEquals(expected, actual); | ||
| Assert.assertEquals(expectedSets, actualSets); | ||
| } | ||
|
|
||
|
|
||
|
|
@@ -1239,6 +1241,49 @@ public Optional<Integer> call(List<Integer> values, Optional<Integer> state) { | |
| Assert.assertEquals(expected, result); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Test | ||
| public void testUpdateStateByKeyWithInitial() { | ||
| List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream; | ||
|
|
||
| List<Tuple2<String, Integer>> initial = Arrays.asList ( | ||
| new Tuple2<String, Integer> ("california", 1), | ||
| new Tuple2<String, Integer> ("new york", 2)); | ||
|
|
||
| JavaRDD<Tuple2<String, Integer>> tmpRDD = ssc.sparkContext().parallelize(initial); | ||
| JavaPairRDD<String, Integer> initialRDD = JavaPairRDD.fromJavaRDD (tmpRDD); | ||
|
|
||
| List<List<Tuple2<String, Integer>>> expected = Arrays.asList( | ||
| Arrays.asList(new Tuple2<String, Integer>("california", 5), | ||
| new Tuple2<String, Integer>("new york", 7)), | ||
| Arrays.asList(new Tuple2<String, Integer>("california", 15), | ||
| new Tuple2<String, Integer>("new york", 11)), | ||
| Arrays.asList(new Tuple2<String, Integer>("california", 15), | ||
| new Tuple2<String, Integer>("new york", 11))); | ||
|
|
||
| JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); | ||
| JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); | ||
|
|
||
| JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey( | ||
| new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() { | ||
| @Override | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit (do this if you have to make more changes): incorrect indentation. should be indented by more than the line with
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't have any other change to do. Actually I copied the previous method. |
||
| public Optional<Integer> call(List<Integer> values, Optional<Integer> 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<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3); | ||
|
|
||
| assertOrderInvariantEquals(expected, result); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Test | ||
| public void testReduceByKeyAndWindowWithInverse() { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess this makes it different from the Scala api, and the reason is that the Java API does not expose the version of the API that uses iterators. And as far as I remember, that was not done because its tricky to write a function Java dealing with scala iterators and all.
Lets try to make the Java and Scala API consistent (minus the iterator based ones). So could you please add a Scala API equivalent to this, that is, (simple non-iterator function, partitioner, initial RDD)? Also unit test for that please :)