zhengbuqian commented on code in PR #23491:
URL: https://github.com/apache/beam/pull/23491#discussion_r993911294
##########
runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java:
##########
@@ -463,6 +476,116 @@ public InMemoryBag<T> copy() {
}
}
+ /** An {@link InMemoryState} implementation of {@link MultimapState}. */
+ public static final class InMemoryMultimap<K, V>
+ implements MultimapState<K, V>, InMemoryState<InMemoryMultimap<K, V>> {
+ private final Coder<K> keyCoder;
+ private final Coder<V> valueCoder;
+ private Multimap<K, V> contents = ArrayListMultimap.create();
+
+ public InMemoryMultimap(Coder<K> keyCoder, Coder<V> valueCoder) {
+ this.keyCoder = keyCoder;
+ this.valueCoder = valueCoder;
+ }
+
+ @Override
+ public void clear() {
+ contents = ArrayListMultimap.create();
Review Comment:
See
https://github.com/apache/beam/blob/af6a04ca87d63107ab2b1fac7981921bcecbe017/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java#L411
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java:
##########
@@ -2511,6 +2512,10 @@ public static boolean usesBagState(DoFn<?, ?> doFn) {
return usesGivenStateClass(doFn, BagState.class) ||
requiresTimeSortedInput(doFn);
}
+ public static boolean usesMultimapState(DoFn<?, ?> doFn) {
+ return usesGivenStateClass(doFn, MultimapState.class) ||
requiresTimeSortedInput(doFn);
Review Comment:
I think I misunderstood, removed it.
##########
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
##########
@@ -3739,6 +3864,122 @@ public void processElement(
pipeline.run();
}
+ @Test
+ @Category({ValidatesRunner.class, UsesStatefulParDo.class,
UsesMultimapState.class})
+ public void testMultimapStateCoderInference() {
+ final String stateId = "foo:";
+ final String countStateId = "count";
+ Coder<MyInteger> myIntegerCoder = MyIntegerCoder.of();
+ pipeline.getCoderRegistry().registerCoderForClass(MyInteger.class,
myIntegerCoder);
+
+ DoFn<KV<String, KV<String, Integer>>, KV<String, MyInteger>> fn =
+ new DoFn<KV<String, KV<String, Integer>>, KV<String, MyInteger>>() {
+
+ @StateId(stateId)
+ private final StateSpec<MultimapState<String, MyInteger>>
multimapState =
+ StateSpecs.multimap();
+
+ @StateId(countStateId)
+ private final StateSpec<CombiningState<Integer, int[], Integer>>
countState =
+ StateSpecs.combiningFromInputInternal(VarIntCoder.of(),
Sum.ofIntegers());
+
+ @ProcessElement
+ public void processElement(
+ ProcessContext c,
+ @Element KV<String, KV<String, Integer>> element,
+ @StateId(stateId) MultimapState<String, MyInteger> state,
+ @StateId(countStateId) CombiningState<Integer, int[], Integer>
count,
+ OutputReceiver<KV<String, MyInteger>> r) {
+ KV<String, Integer> value = element.getValue();
+ ReadableState<Iterable<Entry<String, Iterable<MyInteger>>>>
entriesView =
+ state.entries();
+ state.put(value.getKey(), new MyInteger(value.getValue()));
+ count.add(1);
+ if (count.read() >= 4) {
+ Iterable<Entry<String, Iterable<MyInteger>>> iterate =
state.entries().read();
+ state.put("BadKey", new MyInteger(-1));
+ assertEquals(4, countNestedIterables(iterate));
+ assertEquals(5, countNestedIterables(entriesView.read()));
+ assertEquals(5, countNestedIterables(state.entries().read()));
+
+ for (Entry<String, Iterable<MyInteger>> entry : iterate) {
+ for (MyInteger v : entry.getValue()) {
+ r.output(KV.of(entry.getKey(), v));
+ }
+ }
+ }
+ }
+ };
+ PCollection<KV<String, MyInteger>> output =
+ pipeline
+ .apply(
+ Create.of(
+ KV.of("hello", KV.of("a", 97)), KV.of("hello",
KV.of("a", 97)),
+ KV.of("hello", KV.of("a", 98)), KV.of("hello",
KV.of("b", 33))))
+ .apply(ParDo.of(fn))
+ .setCoder(KvCoder.of(StringUtf8Coder.of(), myIntegerCoder));
+ PAssert.that(output)
+ .containsInAnyOrder(
+ KV.of("a", new MyInteger(97)), KV.of("a", new MyInteger(97)),
+ KV.of("a", new MyInteger(98)), KV.of("b", new MyInteger(33)));
+ pipeline.run();
+ }
+
+ @Test
+ @Category({ValidatesRunner.class, UsesStatefulParDo.class,
UsesMultimapState.class})
+ public void testMultimapStateCoderInferenceFailure() throws Exception {
Review Comment:
Currently such tests of other states also lives in this file, I prefer to
not move them in this PR.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]