[ 
https://issues.apache.org/jira/browse/BEAM-12588?focusedWorklogId=648975&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-648975
 ]

ASF GitHub Bot logged work on BEAM-12588:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 10/Sep/21 02:01
            Start Date: 10/Sep/21 02:01
    Worklog Time Spent: 10m 
      Work Description: kileys commented on a change in pull request #15238:
URL: https://github.com/apache/beam/pull/15238#discussion_r705833950



##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java
##########
@@ -334,7 +423,138 @@ public void clear() {
       StateSpec<MapState<KeyT, ValueT>> spec,
       Coder<KeyT> mapKeyCoder,
       Coder<ValueT> mapValueCoder) {
-    throw new UnsupportedOperationException("TODO: Add support for a map state 
to the Fn API.");
+    return (MapState<KeyT, ValueT>)
+        stateKeyObjectCache.computeIfAbsent(
+            createMultimapUserStateKey(id),
+            new Function<StateKey, Object>() {
+              @Override
+              public Object apply(StateKey key) {
+                return new MapState<KeyT, ValueT>() {
+                  private final MultimapUserState<KeyT, ValueT> impl =
+                      createMultimapUserState(id, mapKeyCoder, mapValueCoder);
+
+                  @Override
+                  public void clear() {
+                    impl.clear();
+                  }
+
+                  @Override
+                  public void put(KeyT key, ValueT value) {
+                    impl.remove(key);
+                    impl.put(key, value);
+                  }
+
+                  @Override
+                  public ReadableState<ValueT> computeIfAbsent(
+                      KeyT key, Function<? super KeyT, ? extends ValueT> 
mappingFunction) {
+                    Iterable<ValueT> values = impl.get(key);
+                    if (Iterables.isEmpty(values)) {

Review comment:
       same here

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java
##########
@@ -325,7 +327,94 @@ public void clear() {
 
   @Override
   public <T> SetState<T> bindSet(String id, StateSpec<SetState<T>> spec, 
Coder<T> elemCoder) {
-    throw new UnsupportedOperationException("TODO: Add support for a map state 
to the Fn API.");
+    return (SetState<T>)
+        stateKeyObjectCache.computeIfAbsent(
+            createMultimapUserStateKey(id),
+            new Function<StateKey, Object>() {
+              @Override
+              public Object apply(StateKey key) {
+                return new SetState<T>() {
+                  private final MultimapUserState<T, Void> impl =
+                      createMultimapUserState(id, elemCoder, VoidCoder.of());
+
+                  @Override
+                  public void clear() {
+                    impl.clear();
+                  }
+
+                  @Override
+                  public ReadableState<Boolean> contains(T t) {
+                    return new ReadableState<Boolean>() {
+                      @Override
+                      public @Nullable Boolean read() {
+                        return !Iterables.isEmpty(impl.get(t));
+                      }
+
+                      @Override
+                      public ReadableState<Boolean> readLater() {
+                        return this;
+                      }
+                    };
+                  }
+
+                  @Override
+                  public ReadableState<Boolean> addIfAbsent(T t) {
+                    Iterable<Void> values = impl.get(t);
+                    if (Iterables.isEmpty(values)) {
+                      impl.put(t, null);
+                    }
+                    return new ReadableState<Boolean>() {
+                      @Override
+                      public @Nullable Boolean read() {
+                        return Iterables.isEmpty(values);
+                      }
+
+                      @Override
+                      public ReadableState<Boolean> readLater() {
+                        return this;
+                      }
+                    };
+                  }
+
+                  @Override
+                  public void remove(T t) {
+                    impl.remove(t);
+                  }
+
+                  @Override
+                  public void add(T value) {
+                    impl.remove(value);
+                    impl.put(value, null);
+                  }
+
+                  @Override
+                  public ReadableState<Boolean> isEmpty() {
+                    return new ReadableState<Boolean>() {
+                      @Override
+                      public @Nullable Boolean read() {
+                        return Iterables.isEmpty(impl.keys());

Review comment:
       same here

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java
##########
@@ -325,7 +327,94 @@ public void clear() {
 
   @Override
   public <T> SetState<T> bindSet(String id, StateSpec<SetState<T>> spec, 
Coder<T> elemCoder) {
-    throw new UnsupportedOperationException("TODO: Add support for a map state 
to the Fn API.");
+    return (SetState<T>)
+        stateKeyObjectCache.computeIfAbsent(
+            createMultimapUserStateKey(id),
+            new Function<StateKey, Object>() {
+              @Override
+              public Object apply(StateKey key) {
+                return new SetState<T>() {
+                  private final MultimapUserState<T, Void> impl =
+                      createMultimapUserState(id, elemCoder, VoidCoder.of());
+
+                  @Override
+                  public void clear() {
+                    impl.clear();
+                  }
+
+                  @Override
+                  public ReadableState<Boolean> contains(T t) {
+                    return new ReadableState<Boolean>() {
+                      @Override
+                      public @Nullable Boolean read() {
+                        return !Iterables.isEmpty(impl.get(t));

Review comment:
       It would be impl.get(t).iterator().hasNext(), this seems simpler

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapUserState.java
##########
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.fn.harness.state;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateAppendRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateClearRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ArrayListMultimap;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * An implementation of a multimap user state that utilizes the Beam Fn State 
API to fetch, clear
+ * and persist values.
+ *
+ * <p>Calling {@link #asyncClose()} schedules any required persistence 
changes. This object should
+ * no longer be used after it is closed.
+ *
+ * <p>TODO: Move to an async persist model where persistence is signalled 
based upon cache memory
+ * pressure and its need to flush.
+ *
+ * <p>TODO: Support block level caching and prefetch.
+ */
+public class MultimapUserState<K, V> {
+
+  private final BeamFnStateClient beamFnStateClient;
+  private final Coder<K> mapKeyCoder;
+  private final Coder<V> valueCoder;
+  private final String stateId;
+  private final StateRequest keysStateRequest;
+  private final StateRequest userStateRequest;
+
+  private boolean isClosed;
+  private boolean isCleared;
+  // Pending updates to persistent storage
+  private Set<K> pendingRemoves = Sets.newHashSet();
+  private Multimap<K, V> pendingAdds = ArrayListMultimap.create();
+  // Map keys with no values in persistent storage
+  private Set<K> negativeCache = Sets.newHashSet();
+  // Values retrieved from persistent storage
+  private Multimap<K, V> persistedValues = ArrayListMultimap.create();
+  private @Nullable Iterable<K> persistedKeys = null;
+
+  public MultimapUserState(
+      BeamFnStateClient beamFnStateClient,
+      String instructionId,
+      String pTransformId,
+      String stateId,
+      ByteString encodedWindow,
+      ByteString encodedKey,
+      Coder<K> mapKeyCoder,
+      Coder<V> valueCoder) {
+    this.beamFnStateClient = beamFnStateClient;
+    this.mapKeyCoder = mapKeyCoder;
+    this.valueCoder = valueCoder;
+    this.stateId = stateId;
+
+    StateRequest.Builder keysStateRequestBuilder = StateRequest.newBuilder();
+    keysStateRequestBuilder
+        .setInstructionId(instructionId)
+        .getStateKeyBuilder()
+        .getMultimapKeysUserStateBuilder()
+        .setTransformId(pTransformId)
+        .setUserStateId(stateId)
+        .setKey(encodedKey)
+        .setWindow(encodedWindow);
+    keysStateRequest = keysStateRequestBuilder.build();
+
+    StateRequest.Builder userStateRequestBuilder = StateRequest.newBuilder();
+    userStateRequestBuilder
+        .setInstructionId(instructionId)
+        .getStateKeyBuilder()
+        .getMultimapUserStateBuilder()
+        .setTransformId(pTransformId)
+        .setUserStateId(stateId)
+        .setWindow(encodedWindow)
+        .setKey(encodedKey);
+    userStateRequest = userStateRequestBuilder.build();
+  }
+
+  public void clear() {
+    checkState(
+        !isClosed,
+        "Multimap user state is no longer usable because it is closed for %s",
+        keysStateRequest.getStateKey());
+
+    isCleared = true;
+    persistedValues = ArrayListMultimap.create();
+    persistedKeys = null;
+    pendingRemoves = Sets.newHashSet();
+    pendingAdds = ArrayListMultimap.create();
+  }
+
+  /*
+   * Returns an iterable of the values associated with key in this multimap, 
if any.
+   * If there are no values, this returns an empty collection, not null.
+   */
+  public Iterable<V> get(@NonNull K key) {
+    checkState(
+        !isClosed,
+        "Multimap user state is no longer usable because it is closed for %s",
+        keysStateRequest.getStateKey());
+
+    Collection<V> pendingValues =
+        
Collections.unmodifiableCollection(Lists.newArrayList(pendingAdds.get(key)));
+    if (isCleared || pendingRemoves.contains(key)) {
+      return pendingValues;
+    }
+
+    Iterable<V> persistedValues = getPersistedValues(key);
+    return Iterables.concat(persistedValues, pendingValues);
+  }
+
+  @SuppressWarnings({
+    "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-12687)
+  })
+  /*
+   * Returns an iterables containing all distinct keys in this multimap.
+   */
+  public Iterable<K> keys() {
+    checkState(
+        !isClosed,
+        "Multimap user state is no longer usable because it is closed for %s",
+        keysStateRequest.getStateKey());
+    if (isCleared) {
+      return 
Collections.unmodifiableCollection(Sets.newHashSet(pendingAdds.keySet()));
+    }
+
+    Set<K> keys = Sets.newHashSet(getPersistedKeys());

Review comment:
       It's above




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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 648975)
    Time Spent: 5h 50m  (was: 5h 40m)

> Support multimapstate over fnapi
> --------------------------------
>
>                 Key: BEAM-12588
>                 URL: https://issues.apache.org/jira/browse/BEAM-12588
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-harness
>            Reporter: Kiley Sok
>            Priority: P2
>          Time Spent: 5h 50m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to