lukecwik commented on code in PR #23491:
URL: https://github.com/apache/beam/pull/23491#discussion_r1041608126


##########
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
##########
@@ -2939,22 +3003,108 @@ public void processElement(
                 @StateId(countStateId) CombiningState<Integer, int[], Integer> 
count,
                 OutputReceiver<KV<String, Integer>> r) {
               KV<String, Integer> value = element.getValue();
-              ReadableState<Iterable<Entry<String, Iterable<Integer>>>> 
entriesView =
-                  state.entries();
               state.put(value.getKey(), value.getValue());
               count.add(1);
+
               if (count.read() >= 4) {
-                Iterable<Entry<String, Iterable<Integer>>> iterate = 
state.entries().read();
-                state.put("BadKey", -1);
-                state.remove("b");
-                assertEquals(4, countNestedIterables(iterate));
-                assertEquals(4, countNestedIterables(entriesView.read()));
-                assertEquals(4, countNestedIterables(state.entries().read()));
-
-                for (Entry<String, Iterable<Integer>> entry : 
entriesView.read()) {
-                  for (Integer v : entry.getValue()) {
-                    r.output(KV.of(entry.getKey(), v));
-                  }
+                // those should be evaluated only when ReadableState.read is 
called.
+                ReadableState<Iterable<Entry<String, Integer>>> entriesView = 
state.entries();
+                ReadableState<Iterable<String>> keysView = state.keys();
+                ReadableState<Boolean> containsBView = state.containsKey("b");
+                ReadableState<Iterable<Integer>> getBView = state.get("b");
+                ReadableState<Boolean> isEmptyView = state.isEmpty();
+
+                // those are evaluated immediately.
+                Iterable<Entry<String, Integer>> entries = 
state.entries().read();
+                Iterable<String> keys = state.keys().read();
+                boolean containsB = state.containsKey("b").read();
+                Iterable<Integer> getB = state.get("b").read();
+                boolean isEmpty = state.isEmpty().read();
+
+                state.clear();
+
+                // entries before and after clear:
+                assertEquals(4, Iterables.size(entries));
+                assertThat(entriesView.read(), emptyIterable());
+                assertThat(state.entries().read(), emptyIterable());
+
+                // keys before and after clear:
+                assertThat(keys, containsInAnyOrder("a", "b"));
+                assertThat(keysView.read(), emptyIterable());
+                assertThat(state.keys().read(), emptyIterable());
+
+                // containsKey before and after clear:
+                assertTrue(containsB);
+                assertFalse(containsBView.read());
+                assertFalse(state.containsKey("b").read());
+
+                // get before and after clear:
+                assertThat(getB, containsInAnyOrder(33));
+                assertThat(getBView.read(), emptyIterable());
+                assertThat(state.get("b").read(), emptyIterable());
+
+                // isEmpty before and after clear:
+                assertFalse(isEmpty);
+                assertTrue(isEmptyView.read());
+                assertTrue(state.isEmpty().read());
+
+                for (Entry<String, Integer> entry : entriesView.read()) {
+                  r.output(KV.of(entry.getKey(), entry.getValue()));
+                }
+              }
+            }
+          };
+      PCollection<KV<String, Integer>> 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));
+      PAssert.that(output).empty();
+      pipeline.run();
+    }
+
+    // Use byte[] as the key type of Multimap to test that Multimap should 
treat different keys with
+    // equivalent structural values as the same key.
+    @Test
+    @Category({ValidatesRunner.class, UsesStatefulParDo.class, 
UsesMultimapState.class})
+    public void testMultimapStateStructuralValue() {
+      final String stateId = "foo:";
+      final String countStateId = "count";
+      final byte[] bagKey = "BadKey".getBytes(StandardCharsets.UTF_8);
+      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<byte[], Integer>> 
multimapState =
+                StateSpecs.multimap(ByteArrayCoder.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<byte[], Integer> state,
+                @StateId(countStateId) CombiningState<Integer, int[], Integer> 
count,
+                OutputReceiver<KV<String, Integer>> r) {
+              KV<String, Integer> value = element.getValue();
+              ReadableState<Iterable<Entry<byte[], Integer>>> entriesView = 
state.entries();
+              state.put(value.getKey().getBytes(StandardCharsets.UTF_8), 
value.getValue());
+              count.add(1);
+              if (count.read() >= 4) {
+                Iterable<Entry<byte[], Integer>> entries = 
state.entries().read();
+                state.put(bagKey, -1);
+                assertEquals(4, Iterables.size(entries));
+                assertEquals(5, Iterables.size(entriesView.read()));
+                assertEquals(5, Iterables.size(state.entries().read()));
+

Review Comment:
   please add coverage for containsKey/get since these accessors are important 
and need to do the structural key conversion.



##########
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:
   The +1 was for your response that we need to keep stuff consistent for past 
reads that were returned.



-- 
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]

Reply via email to