rkhachatryan commented on a change in pull request #14799:
URL: https://github.com/apache/flink/pull/14799#discussion_r570555194



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/proxy/ProxyKeyedStateBackend.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.flink.runtime.state.proxy;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.Keyed;
+import org.apache.flink.runtime.state.KeyedStateFunction;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.PriorityComparable;
+import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.runtime.state.StateSnapshotTransformer;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.ttl.TtlStateFactory;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.RunnableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+public class ProxyKeyedStateBackend<K>
+        implements CheckpointableKeyedStateBackend<K>, CheckpointListener {
+    AbstractKeyedStateBackend<K> keyedStateBackend;
+
+    private static final Map<Class<? extends StateDescriptor>, StateFactory> 
STATE_FACTORIES =
+            Stream.of(
+                            Tuple2.of(
+                                    ValueStateDescriptor.class,
+                                    (StateFactory) ProxyValueState::create),
+                            Tuple2.of(
+                                    ListStateDescriptor.class,
+                                    (StateFactory) ProxyListState::create),
+                            Tuple2.of(
+                                    ReducingStateDescriptor.class,
+                                    (StateFactory) ProxyReducingState::create),
+                            Tuple2.of(
+                                    AggregatingStateDescriptor.class,
+                                    (StateFactory) 
ProxyAggregatingState::create),
+                            Tuple2.of(
+                                    MapStateDescriptor.class, (StateFactory) 
ProxyMapState::create))
+                    .collect(Collectors.toMap(t -> t.f0, t -> t.f1));
+
+    // ==============================================================
+    //  cache maintained by the proxyKeyedStateBackend itself
+    //  not the same as the underlying wrapped keyedStateBackend
+    //  InternalKvState is a ProxyXXState, XX stands for Value, List ...
+    /** So that we can give out state when the user uses the same key. */
+    protected final HashMap<String, InternalKvState<K, ?, ?>> 
keyValueStatesByName;
+
+    @SuppressWarnings("rawtypes")
+    protected InternalKvState lastState;
+
+    /** For caching the last accessed partitioned state. */
+    protected String lastName;
+
+    // ==============================================================
+    // ==== the same as the wrapped keyedStateBackend
+
+    public final ExecutionConfig executionConfig;
+
+    public final TtlTimeProvider ttlTimeProvider;
+
+    public ProxyKeyedStateBackend(
+            AbstractKeyedStateBackend<K> keyedStateBackend,
+            ExecutionConfig executionConfig,
+            TtlTimeProvider ttlTimeProvider) {
+        this.keyedStateBackend = keyedStateBackend;
+        this.executionConfig = executionConfig;
+        this.ttlTimeProvider = ttlTimeProvider;
+
+        this.keyValueStatesByName = new HashMap<>();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return keyedStateBackend.getKeyGroupRange();
+    }
+
+    @Override
+    public void close() throws IOException {
+        keyedStateBackend.close();
+    }
+
+    @Override
+    public void setCurrentKey(K newKey) {
+        keyedStateBackend.setCurrentKey(newKey);
+    }
+
+    @Override
+    public K getCurrentKey() {
+        return keyedStateBackend.getCurrentKey();
+    }
+
+    @Override
+    public TypeSerializer<K> getKeySerializer() {
+        return keyedStateBackend.getKeySerializer();
+    }
+
+    @Override
+    public <N, S extends State, T> void applyToAllKeys(
+            N namespace,
+            TypeSerializer<N> namespaceSerializer,
+            StateDescriptor<S, T> stateDescriptor,
+            KeyedStateFunction<K, S> function)
+            throws Exception {
+        try (Stream<K> keyStream = getKeys(stateDescriptor.getName(), 
namespace)) {
+
+            final S state = getPartitionedState(namespace, 
namespaceSerializer, stateDescriptor);
+
+            if (keyedStateBackend.supportConcurrentModification()) {

Review comment:
       Shouldn't the name be `notSupportConcurrentModification` (reversed)?
   If true then we iterate without copying to list.
   
   Just out of curiosity: did you actually get an exception here with 
HeapStateBackend? Which test?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
##########
@@ -204,6 +216,11 @@ public static StateBackend 
fromApplicationOrConfigOrDefault(
 
         // (1) the application defined state backend has precedence
         if (fromApplication != null) {
+
+            checkArgument(
+                    !(fromApplication instanceof ProxyStateBackend),

Review comment:
       This will not detect `new StubBackend(new ProxyBackend))` :)
   I suggested above to add `unwrap` method which could also help here.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
##########
@@ -242,6 +259,34 @@ public static StateBackend 
fromApplicationOrConfigOrDefault(
         return backend;
     }
 
+    public static StateBackend loadStateBackend(
+            @Nullable StateBackend fromApplication,
+            Configuration config,
+            ClassLoader classLoader,
+            @Nullable Logger logger)
+            throws IllegalConfigurationException, DynamicCodeLoadingException, 
IOException {
+
+        final StateBackend backend =
+                fromApplicationOrConfigOrDefault(fromApplication, config, 
classLoader, logger);
+
+        if (needToProxyStateBackend(backend, config)) {
+            LOG.info(
+                    "Proxy State Backend used, and the root State Backend is 
{}",
+                    backend.getClass().getSimpleName());
+            return new ProxyStateBackend(backend);

Review comment:
       I think this class is the right place to decide and create 
`ProxyStateBackend` :+1: 

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java
##########
@@ -99,23 +101,30 @@ private InternalTimeServiceManagerImpl(
      * <p><b>IMPORTANT:</b> Keep in sync with {@link 
InternalTimeServiceManager.Provider}.
      */
     public static <K> InternalTimeServiceManagerImpl<K> create(
-            CheckpointableKeyedStateBackend<K> keyedStatedBackend,
+            CheckpointableKeyedStateBackend<K> keyedStateBackend,
             ClassLoader userClassloader,
             KeyContext keyContext,
             ProcessingTimeService processingTimeService,
             Iterable<KeyGroupStatePartitionStreamProvider> rawKeyedStates)
             throws Exception {
-        final KeyGroupRange keyGroupRange = 
keyedStatedBackend.getKeyGroupRange();
+        final KeyGroupRange keyGroupRange = 
keyedStateBackend.getKeyGroupRange();
+
+        KeyedStateBackend<?> rootKeyedStateBackend =
+                keyedStateBackend instanceof ProxyKeyedStateBackend
+                        ? ((ProxyKeyedStateBackend<?>) keyedStateBackend)
+                                .getProxiedKeyedStateBackend()
+                        : keyedStateBackend;
+
         final boolean requiresSnapshotLegacyTimers =
-                keyedStatedBackend instanceof AbstractKeyedStateBackend
-                        && ((AbstractKeyedStateBackend<K>) keyedStatedBackend)
+                rootKeyedStateBackend instanceof AbstractKeyedStateBackend
+                        && ((AbstractKeyedStateBackend<K>) 
rootKeyedStateBackend)

Review comment:
       WDYT about pulling `requiresLegacySynchronousTimerSnapshots` to the 
interface?
   Then, with `unwrap` added we could just call
   `rootKeyedStateBackend.unwrap().requiresLegacySynchronousTimerSnapshots()`.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorageLoader.java
##########
@@ -162,14 +163,19 @@ public static CheckpointStorage load(
         Preconditions.checkNotNull(classLoader, "classLoader");
         Preconditions.checkNotNull(configuredStateBackend, "statebackend");
 
-        if (configuredStateBackend instanceof CheckpointStorage) {
+        StateBackend rootStateBackend =
+                (configuredStateBackend instanceof ProxyStateBackend)
+                        ? ((ProxyStateBackend) 
configuredStateBackend).getProxiedStateBackend()
+                        : configuredStateBackend;

Review comment:
       I see this fragment several times in your PR. If later we add one more 
layer (say logging or new TTL implementation) this can break (we already have 
`StubStateBackend`).
   
   WDYT about adding `default StateBackend unwrap()` method to the interface?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
##########
@@ -47,6 +52,13 @@
     //  Configuration shortcut names
     // ------------------------------------------------------------------------
 
+    private static final Set<String> STATE_BACKEND_CAN_BE_PROXIED =
+            new HashSet<>(
+                    Arrays.asList(
+                            
"org.apache.flink.contrib.streaming.state.RocksDBStateBackend",

Review comment:
       Why do we want to limit proxying to these classes?
   Besides that, hardcoding class names like this is a bit fragile 
(`RocksDBStateBackend.class`?)
   
   At least one upcoming PR already adds new classes that should be proxied.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapValueState.java
##########
@@ -31,7 +31,7 @@
  * @param <N> The type of the namespace.
  * @param <V> The type of the value.
  */
-class HeapValueState<K, N, V> extends AbstractHeapState<K, N, V>
+public class HeapValueState<K, N, V> extends AbstractHeapState<K, N, V>

Review comment:
       Why is this change needed?




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

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to