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



##########
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:
       If it supports concurrent modification, you do not need to get the full 
list of keys in front (iterator can not be modified).
   
   Yes, there is test-case for this: 
`StateBackendTestBase#testConcurrentModificationWithApplyToAllKeys`
   And in the Jira ticket why this test is added there is more details on what 
exactly "ConcurrentModification" refers to. It refers to concurrent 
modification on key-value entries of the state; StateTable is a HashTable 
(something similar, do not remember details), which does not support concurrent 
modification.




----------------------------------------------------------------
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:
[email protected]


Reply via email to