acrites commented on code in PR #36218:
URL: https://github.com/apache/beam/pull/36218#discussion_r2396170226


##########
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);
+      return PrefetchableIterables.concat(
+          Iterables.concat(
+              Iterables.transform(
+                  pendingAddsNow.entrySet(),
+                  entry ->
+                      Iterables.transform(
+                          entry.getValue().getValue(),
+                          value -> 
Maps.immutableEntry(entry.getValue().getKey(), value)))));
+    }
+
+    Set<Object> pendingRemovesNow = new HashSet<>(pendingRemoves.keySet());
+    // Make a deep copy of pendingAdds so this iterator represents a snapshot 
of state at the time
+    // it was created.
+    Map<K, List<V>> pendingAddsNow = new HashMap<>();
+    for (Map.Entry<Object, KV<K, List<V>>> entry : pendingAdds.entrySet()) {
+      pendingAddsNow.put(
+          entry.getValue().getKey(), new ArrayList<>()); // 
entry.getValue().getValue());
+      for (V value : entry.getValue().getValue()) {
+        List<V> values = pendingAddsNow.get(entry.getValue().getKey());
+        values.add(value);
+      }
+    }
+    return new PrefetchableIterables.Default<Map.Entry<K, V>>() {
+      @Override
+      public PrefetchableIterator<Map.Entry<K, V>> createIterator() {
+        return new PrefetchableIterator<Map.Entry<K, V>>() {
+          PrefetchableIterator<Map.Entry<K, V>> persistedEntriesIterator =
+              PrefetchableIterables.concat(
+                      Iterables.concat(
+                          Iterables.transform(
+                              persistedEntries,
+                              entry ->
+                                  Iterables.transform(
+                                      entry.getValue(),
+                                      value -> 
Maps.immutableEntry(entry.getKey(), value)))))
+                  .iterator();
+          Iterator<Map.Entry<K, V>> pendingAddsNowIterator;
+          boolean hasNext;
+          Map.Entry<K, V> nextEntry;
+
+          @Override
+          public boolean isReady() {
+            return persistedEntriesIterator.isReady();
+          }
+
+          @Override
+          public void prefetch() {
+            if (!isReady()) {
+              persistedEntriesIterator.prefetch();
+            }
+          }
+
+          @Override
+          public boolean hasNext() {
+            if (hasNext) {
+              return true;
+            }
+
+            while (persistedEntriesIterator.hasNext()) {
+              Map.Entry<K, V> nextPersistedEntry = 
persistedEntriesIterator.next();
+              Object nextKeyStructuralValue =
+                  mapKeyCoder.structuralValue(nextPersistedEntry.getKey());
+              if (!pendingRemovesNow.contains(nextKeyStructuralValue)) {
+                nextEntry =
+                    Maps.immutableEntry(nextPersistedEntry.getKey(), 
nextPersistedEntry.getValue());
+                hasNext = true;
+                return true;
+              }
+            }
+
+            if (pendingAddsNowIterator == null) {

Review Comment:
   I changed this up so it merges in the local adds with backend values. Still 
needed two concats for some reason though. Otherwise, it complains that it's 
getting `PrefetchableIterator<Iterable<Entry<K, V>>>` instead of a 
`PrefetchableIterator<Entry<K,V>>`.
   
   I'm not really sure if the prefetching propagates through all these 
`Iterables.transform()` calls though.



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