You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/29 18:39:28 UTC

[GitHub] [flink] rkhachatryan commented on a change in pull request #14799: [WIP] Wrapped StateBackend to forward state changes to StateChangleLog

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



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

Review comment:
       I would prefer a code like this:
   ```
   InternalKvState nested =
           keyedStateBackend.createInternalState(
                   namespaceSerializer, stateDesc, snapshotTransformFactory);
   if (stateDesc instanceof ValueStateDescriptor) {
       return new ProxyValueState((InternalValueState) nested);
   } else if (stateDesc instanceof ListStateDescriptor) {
       return ...
   }
   ```
   It would be simpler and shorter.
   
   I think that `Map` can be useful when the mappings can be updated dynamically or programmatically. Neither is the case.
   And then factory interface can be dropped as it's only needed to work with map IIUC.

##########
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:
       In runtime, we end up with 4 backend:
   ```
   Proxy extends Abstract (has it's own fields AND refers to passed fields)
       Proxied extends Abstract (has it's own fields AND refers to passed fields)
   ```
   Right?
   
   I see the following drawbacks:
   1. Double caching and lookup in `keyValueStatesByName`
   2. Error-prone: 4 backend classes refer to the same `kvStateRegistry`, `keySerializer` etc. making it easy to either call some method twice or not call it for a wrapped backend (e.g. `setCurrentKey` isn't called for the wrapped backend)
   4. To me this hierarchy is very confusing
   
   Why not refactor this in some way or another? Can `AbstractKeyedStateBackend` be also a proxy (i.e. replace inheritance with delegation)?

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
##########
@@ -44,6 +44,15 @@
                     .noDefaultValue()
                     .withDescription("The state backend to be used to checkpoint state.");
 
+    /** Whether to enable state change log. */
+    // @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS)
+    public static final ConfigOption<Boolean> ENABLE_STATE_CHANGE_LOG =
+            ConfigOptions.key("state.backend.enable-statechangelog")

Review comment:
       I think this option should be hidden.




----------------------------------------------------------------
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:
users@infra.apache.org