curcur commented on a change in pull request #14799: URL: https://github.com/apache/flink/pull/14799#discussion_r567556094
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/proxy/ProxyKeyedStateBackend.java ########## @@ -0,0 +1,306 @@ +/* + * 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.state.AggregatingStateDescriptor; +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.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.HeapKeyedStateBackend; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.state.internal.InternalKvState; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.IOUtils; + +import javax.annotation.Nonnull; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.RunnableFuture; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** */ +public class ProxyKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> { + // wrapped keyed state backend, either HeapKeyedStateBackend or RocksDBKeyedStateBackend + 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)); + + public ProxyKeyedStateBackend(AbstractKeyedStateBackend<K> keyedStateBackend) { + super( + keyedStateBackend.kvStateRegistry, + keyedStateBackend.keySerializer, + keyedStateBackend.userCodeClassLoader, + keyedStateBackend.executionConfig, + keyedStateBackend.ttlTimeProvider, + keyedStateBackend.cancelStreamRegistry, + keyedStateBackend.keyGroupCompressionDecorator, + keyedStateBackend.keyContext); + this.keyedStateBackend = keyedStateBackend; Review comment: All these questions are great! For all the reasons mentioned or not mentioned above, I feel this hierarchy is confusing as well. So I've spent some time during the weekend simplifying it. Originally, the reason I extend `AbstractKeyedStateBackend` is I thought all fields have to be wrapped and delegated. So if I implement `CheckpointableKeyedStateBackend`, I basically rewrite everything in `AbstractKeyedStateBackend`. But it is not true, only the InternalKvState related (that's the state representation to internal ops) have to be wrapped, everything else (keys, namespace e.t.c) is kept untouched. So, I reimplement this to `implements CheckpointableKeyedStateBackend`. With a bit more duplicated code, I think this is much more simple and clean. **However**, the problem `keyValueStatesByName` is duplicated but one instance that isn't actually used is still there. I am still not happy with this part, but much better. ---------------------------------------------------------------- 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