acrites commented on code in PR #36218:
URL: https://github.com/apache/beam/pull/36218#discussion_r2383445504
##########
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapUserState.java:
##########
@@ -235,6 +254,122 @@ public K next() {
};
}
+ @SuppressWarnings({
+ "nullness" // TODO(https://github.com/apache/beam/issues/21068)
+ })
+ /*
+ * Returns an Iterable containing all <K, V> entries in this multimap.
+ */
+ public PrefetchableIterable<Map.Entry<K, V>> entries() {
+ checkState(
+ !isClosed,
+ "Multimap user state is no longer usable because it is closed for %s",
+ keysStateRequest.getStateKey());
+ if (isCleared) {
+ Map<Object, KV<K, List<V>>> pendingAddsNow = new HashMap<>(pendingAdds);
Review Comment:
Done.
##########
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapUserState.java:
##########
@@ -106,6 +108,23 @@ public MultimapUserState(
.setWindow(stateKey.getMultimapKeysUserState().getWindow())
.setKey(stateKey.getMultimapKeysUserState().getKey());
this.userStateRequest = userStateRequestBuilder.build();
+
+ StateRequest.Builder entriesStateRequestBuilder =
StateRequest.newBuilder();
Review Comment:
Good idea. I added a note (we could do the same with the keys StateRequest
as well.
##########
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
##########
@@ -2917,6 +2917,72 @@ public void processElement(
pipeline.run();
}
+ @Test
+ @Category({ValidatesRunner.class, UsesStatefulParDo.class,
UsesMultimapState.class})
+ public void testMultimapStateEntries() {
+ final String stateId = "foo:";
+ final String countStateId = "count";
+ DoFn<KV<String, KV<String, Integer>>, KV<String, Integer>> fn =
+ new DoFn<KV<String, KV<String, Integer>>, KV<String, Integer>>() {
+
+ @StateId(stateId)
+ private final StateSpec<MultimapState<String, Integer>>
multimapState =
+ StateSpecs.multimap(StringUtf8Coder.of(), VarIntCoder.of());
+
+ @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, Integer> state,
+ @StateId(countStateId) CombiningState<Integer, int[], Integer>
count,
+ OutputReceiver<KV<String, Integer>> r) {
+ // Empty before we process any elements.
+ if (count.read() == 0) {
+ assertThat(state.entries().read(), emptyIterable());
+ }
+ assertEquals(count.read().intValue(),
Iterables.size(state.entries().read()));
+
+ KV<String, Integer> value = element.getValue();
+ state.put(value.getKey(), value.getValue());
+ count.add(1);
+
+ if (count.read() >= 4) {
+ // Those should be evaluated only when ReadableState.read is
called.
+ ReadableState<Iterable<Entry<String, Integer>>> entriesView =
state.entries();
Review Comment:
Done.
--
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]