lukecwik commented on a change in pull request #15170:
URL: https://github.com/apache/beam/pull/15170#discussion_r672685496



##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/CachingBeamFnStateClient.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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 com.google.auto.value.AutoValue;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import 
org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleRequest.CacheToken;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateGetResponse;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey;
+import 
org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey.IterableSideInput;
+import 
org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey.MultimapKeysSideInput;
+import 
org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey.MultimapSideInput;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse;
+import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LoadingCache;
+
+/**
+ * Wraps a delegate BeamFnStateClient and stores the result of state requests 
in cross bundle cache
+ * according to the available cache tokens. If there are no cache tokens for 
the state key requested
+ * the request is forwarded to the client and executed normally.
+ */
+public class CachingBeamFnStateClient implements BeamFnStateClient {
+
+  private final BeamFnStateClient beamFnStateClient;
+  private final LoadingCache<StateKey, Map<StateCacheKey, StateGetResponse>> 
stateCache;
+  private final Map<CacheToken.SideInput, ByteString> sideInputCacheTokens;
+  private final ByteString userStateToken;
+
+  /**
+   * Creates a CachingBeamFnStateClient that wraps a BeamFnStateClient with a 
LoadingCache. Cache
+   * tokens are sent by the runner to indicate which state is able to be 
cached.
+   */
+  public CachingBeamFnStateClient(
+      BeamFnStateClient beamFnStateClient,
+      LoadingCache<StateKey, Map<StateCacheKey, StateGetResponse>> stateCache,
+      List<CacheToken> cacheTokenList) {
+    this.beamFnStateClient = beamFnStateClient;
+    this.stateCache = stateCache;
+    this.sideInputCacheTokens = new HashMap<>();
+
+    // Set up cache tokens.
+    ByteString tempUserStateToken = ByteString.EMPTY;
+    for (BeamFnApi.ProcessBundleRequest.CacheToken token : cacheTokenList) {
+      if (token.hasUserState()) {
+        tempUserStateToken = token.getToken();
+      } else if (token.hasSideInput()) {
+        sideInputCacheTokens.put(token.getSideInput(), token.getToken());
+      }
+    }
+    this.userStateToken = tempUserStateToken;
+  }
+
+  /**
+   * Completes the response with a cached value if possible, if not forwards 
the response to the
+   * BeamFnStateClient and tries caching the result. All Append and Clear 
requests are forwarded.
+   */
+  @Override
+  public void handle(
+      StateRequest.Builder requestBuilder, CompletableFuture<StateResponse> 
response) {
+
+    StateKey stateKey = requestBuilder.getStateKey();
+    ByteString cacheToken = getCacheToken(stateKey);
+
+    // If state is not cacheable proceed as normal.
+    if (ByteString.EMPTY.equals(cacheToken)) {
+      beamFnStateClient.handle(requestBuilder, response);
+      return;
+    }
+
+    switch (requestBuilder.getRequestCase()) {
+      case GET:
+        // Check if data is in the cache.
+        StateCacheKey cacheKey =
+            StateCacheKey.create(cacheToken, 
requestBuilder.getGet().getContinuationToken());
+        Map<StateCacheKey, StateGetResponse> stateKeyMap = 
stateCache.getUnchecked(stateKey);
+        StateGetResponse cachedPage = stateKeyMap.get(cacheKey);
+
+        // If data is not cached, add callback to add response to cache on 
completion.
+        // Otherwise, complete the response with the cached data.
+        if (cachedPage == null) {
+          beamFnStateClient.handle(requestBuilder, response);
+          CompletableFuture<Void> callback =
+              response.thenAccept(
+                  stateResponse -> {
+                    stateCache.getUnchecked(stateKey).put(cacheKey, 
stateResponse.getGet());
+                  });
+          callback.getNow(null);
+        } else {
+          response.complete(
+              
StateResponse.newBuilder().setId(requestBuilder.getId()).setGet(cachedPage).build());
+        }
+
+        return;
+
+      case APPEND:
+        // TODO(BEAM-12637): Support APPEND in CachingBeamFnStateClient.
+        beamFnStateClient.handle(requestBuilder, response);

Review comment:
       We need to know that the cache is dirty otherwise we will return what we 
thought was the last page as the last page without a continuation token and or 
the appended values missing.
   
   Try adding a test where you read the entire bag, and then append a value and 
then try reading the entire bag again.




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