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


##########
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();
+    }
+
+    @Override
+    public void put(K key, V value) {
+      contents.put(key, value);
+    }
+
+    @Override
+    public void remove(K key) {
+      contents.removeAll(key);
+    }
+
+    @Override
+    public @UnknownKeyFor @NonNull @Initialized ReadableState<Iterable<V>> 
get(K key) {
+      return contents.containsKey(key)

Review Comment:
   return CollectionViewState.of(contents.get(key)) directly since by default 
it returns empty list if key does not exists?



##########
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();
+    }
+
+    @Override
+    public void put(K key, V value) {
+      contents.put(key, value);
+    }
+
+    @Override
+    public void remove(K key) {
+      contents.removeAll(key);
+    }
+
+    @Override
+    public @UnknownKeyFor @NonNull @Initialized ReadableState<Iterable<V>> 
get(K key) {
+      return contents.containsKey(key)
+          ? CollectionViewState.of(contents.get(key))
+          : CollectionViewState.of(Collections.emptyList());
+    }
+
+    @Override
+    public ReadableState<Iterable<K>> keys() {
+      return CollectionViewState.of(contents.keySet());
+    }
+
+    @Override
+    public ReadableState<Iterable<Map.Entry<K, Iterable<V>>>> entries() {
+      return new ReadableState<Iterable<Map.Entry<K, Iterable<V>>>>() {
+        @Override
+        public Iterable<Map.Entry<K, Iterable<V>>> read() {
+          List<Map.Entry<K, Iterable<V>>> result = new ArrayList<>();
+          for (Map.Entry<K, Collection<V>> entry : 
contents.asMap().entrySet()) {
+            result.add(
+                new AbstractMap.SimpleEntry(
+                    entry.getKey(), ImmutableList.copyOf(entry.getValue())));
+          }
+          return ImmutableList.copyOf(result);
+        }
+
+        @Override
+        public ReadableState<Iterable<Map.Entry<K, Iterable<V>>>> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public @UnknownKeyFor @NonNull @Initialized ReadableState<
+            @UnknownKeyFor @NonNull @Initialized Boolean>
+        containsKey(K key) {
+      return new ReadableState<Boolean>() {
+        @Override
+        public @org.checkerframework.checker.nullness.qual.Nullable Boolean 
read() {
+          return contents.containsKey(key);
+        }
+
+        @Override
+        public @UnknownKeyFor @NonNull @Initialized ReadableState<Boolean> 
readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public @UnknownKeyFor @NonNull @Initialized ReadableState<
+            @UnknownKeyFor @NonNull @Initialized Boolean>
+        isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public @org.checkerframework.checker.nullness.qual.Nullable Boolean 
read() {

Review Comment:
   unnecessary Nullable annotation?



##########
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();
+    }
+
+    @Override
+    public void put(K key, V value) {
+      contents.put(key, value);
+    }
+
+    @Override
+    public void remove(K key) {
+      contents.removeAll(key);
+    }
+
+    @Override
+    public @UnknownKeyFor @NonNull @Initialized ReadableState<Iterable<V>> 
get(K key) {
+      return contents.containsKey(key)
+          ? CollectionViewState.of(contents.get(key))
+          : CollectionViewState.of(Collections.emptyList());
+    }
+
+    @Override
+    public ReadableState<Iterable<K>> keys() {
+      return CollectionViewState.of(contents.keySet());
+    }
+
+    @Override
+    public ReadableState<Iterable<Map.Entry<K, Iterable<V>>>> entries() {
+      return new ReadableState<Iterable<Map.Entry<K, Iterable<V>>>>() {
+        @Override
+        public Iterable<Map.Entry<K, Iterable<V>>> read() {
+          List<Map.Entry<K, Iterable<V>>> result = new ArrayList<>();
+          for (Map.Entry<K, Collection<V>> entry : 
contents.asMap().entrySet()) {
+            result.add(
+                new AbstractMap.SimpleEntry(
+                    entry.getKey(), ImmutableList.copyOf(entry.getValue())));
+          }
+          return ImmutableList.copyOf(result);
+        }
+
+        @Override
+        public ReadableState<Iterable<Map.Entry<K, Iterable<V>>>> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public @UnknownKeyFor @NonNull @Initialized ReadableState<
+            @UnknownKeyFor @NonNull @Initialized Boolean>
+        containsKey(K key) {
+      return new ReadableState<Boolean>() {
+        @Override
+        public @org.checkerframework.checker.nullness.qual.Nullable Boolean 
read() {

Review Comment:
   unnecessary Nullable annotation?



##########
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:
   why not just clear the instantiated contents?



##########
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:
   Does requiresTimeSortedInput leverage MultimapState for its support?



##########
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:
   I feel these CoderInferenceFailure tests should be in a separate 
StateSpecsTest since they don't really validate any runners and the DoFn never 
gets executed.



##########
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java:
##########
@@ -576,6 +577,16 @@ public ReadableState<Boolean> readLater() {
             });
   }
 
+  @Override
+  public <KeyT, ValueT> MultimapState<KeyT, ValueT> bindMultimap(
+      String id,
+      StateSpec<MultimapState<KeyT, ValueT>> spec,
+      Coder<KeyT> keyCoder,
+      Coder<ValueT> valueCoder) {
+    throw new UnsupportedOperationException(
+        "TODO: Add support for a multimap state to the Fn API.");

Review Comment:
   nit:add a github issue link for tracking



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