Myasuka commented on a change in pull request #14943:
URL: https://github.com/apache/flink/pull/14943#discussion_r590570878



##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogValueState.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.state.changelog;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.internal.InternalValueState;
+
+import java.io.IOException;
+
+/**
+ * Delegated partitioned {@link ValueState} that forwards changes to {@link 
StateChange} upon {@link
+ * ValueState} is updated.
+ *
+ * @param <K> The type of the key.
+ * @param <N> The type of the namespace.
+ * @param <V> The type of the value.
+ */
+class ChangelogValueState<K, N, V> implements InternalValueState<K, N, V> {

Review comment:
       We could introduce `AbstractChangelogState` to act as `InternalKvState`, 
so that we could only keep only one place of those `getKeySerializer`, 
`getNamespaceSerializer`, `getSerializedValue` and so on.
   
   Maybe you could refer to 
[pull/15119](https://github.com/apache/flink/pull/15119/files#diff-52de1f1a9a4e344e62306261f83533bc6a87326f372d775bc984c3a944d66943)
 for similar implementation.

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogValueState.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.state.changelog;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.internal.InternalValueState;
+
+import java.io.IOException;
+
+/**
+ * Delegated partitioned {@link ValueState} that forwards changes to {@link 
StateChange} upon {@link
+ * ValueState} is updated.
+ *
+ * @param <K> The type of the key.
+ * @param <N> The type of the namespace.
+ * @param <V> The type of the value.
+ */
+class ChangelogValueState<K, N, V> implements InternalValueState<K, N, V> {
+    private final InternalValueState<K, N, V> valueState;
+
+    ChangelogValueState(InternalValueState<K, N, V> valueState) {
+        this.valueState = valueState;
+    }
+
+    @Override
+    public V value() throws IOException {
+        return valueState.value();
+    }
+
+    @Override
+    public void update(V value) throws IOException {
+        valueState.update(value);
+    }
+
+    @Override
+    public TypeSerializer<K> getKeySerializer() {
+        return valueState.getKeySerializer();
+    }
+
+    @Override
+    public TypeSerializer<N> getNamespaceSerializer() {
+        return valueState.getNamespaceSerializer();
+    }
+
+    @Override
+    public TypeSerializer<V> getValueSerializer() {
+        return valueState.getValueSerializer();
+    }
+
+    @Override
+    public void setCurrentNamespace(N namespace) {
+        valueState.setCurrentNamespace(namespace);
+    }
+
+    @Override
+    public byte[] getSerializedValue(
+            byte[] serializedKeyAndNamespace,
+            TypeSerializer<K> safeKeySerializer,
+            TypeSerializer<N> safeNamespaceSerializer,
+            TypeSerializer<V> safeValueSerializer)
+            throws Exception {
+        return valueState.getSerializedValue(
+                serializedKeyAndNamespace,
+                safeKeySerializer,
+                safeNamespaceSerializer,
+                safeValueSerializer);
+    }
+
+    @Override
+    public StateIncrementalVisitor<K, N, V> getStateIncrementalVisitor(
+            int recommendedMaxNumberOfReturnedRecords) {
+        return 
valueState.getStateIncrementalVisitor(recommendedMaxNumberOfReturnedRecords);
+    }
+
+    @Override
+    public void clear() {
+        valueState.clear();
+    }
+
+    public InternalValueState<K, N, V> getDelegatedValueState() {

Review comment:
       Actually, I think all such `getDelegatedValueState` could be simplified 
to `getDelegatedState`, and we always keep just protected `delegatedState` 
within the `AbstractChangelogState`.
   

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
##########
@@ -193,24 +193,24 @@ protected CheckpointStreamFactory createStreamFactory() 
throws Exception {
         return checkpointStreamFactory;
     }
 
-    protected <K> AbstractKeyedStateBackend<K> 
createKeyedBackend(TypeSerializer<K> keySerializer)
-            throws Exception {
+    protected <K> CheckpointableKeyedStateBackend<K> createKeyedBackend(

Review comment:
       Actually, I don't know why we have to change the returned class from 
`AbstractKeyedStateBackend` to `CheckpointableKeyedStateBackend`, is it 
necessary?

##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
##########
@@ -96,6 +96,16 @@
                                             "Recognized shortcut names are 
'jobmanager' and 'filesystem'.")
                                     .build());
 
+    /** Whether to enable state change log. */
+    @Documentation.Section(value = 
Documentation.Sections.COMMON_STATE_BACKENDS)
+    @Documentation.ExcludeFromDocumentation("Hidden for now")
+    public static final ConfigOption<Boolean> ENABLE_STATE_CHANGE_LOG =
+            ConfigOptions.key("state.backend.enable-statechangelog")

Review comment:
       I think this option targets for checkpointing, and maybe 
`state.checkpoints.enable-changelog` looks 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:
[email protected]


Reply via email to