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



##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/StateConfigUtil.java
##########
@@ -46,10 +41,7 @@ public static StateTtlConfig createTtlConfig(long 
retentionTime) {
         }
     }
 
-    public static boolean isStateImmutableInStateBackend(KeyedStateBackend<?> 
stateBackend) {
-        // TODO: remove the hard code check once FLINK-21027 is supported
-        return (stateBackend instanceof AbstractKeyedStateBackend)
-                && ((AbstractKeyedStateBackend<?>) stateBackend)
-                        
.isStateImmutableInStateBackend(CheckpointType.CHECKPOINT);
+    public static boolean isStateImmutableInStateBackend(KeyedStateBackend<?> 
keyedStateBackend) {
+        return 
keyedStateBackend.isStateImmutableInStateBackend(CheckpointType.CHECKPOINT);

Review comment:
       ```suggestion
           // TODO: remove the hard code check once FLINK-21027 is supported
           return 
keyedStateBackend.isStateImmutableInStateBackend(CheckpointType.CHECKPOINT);
   ```
   We need to keep this TODO comment since invocation of this 
`isStateImmutableInStateBackend` method should be replaced by the to-be-added 
`isStateKeyValueSerialized` method after FLINK-21027 is implemented.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java
##########
@@ -93,17 +93,17 @@ 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,

Review comment:
       I could find many nice cleanups like this but a little bit distracted 
for review. Maybe a separate commit for such cleanups next time (smile)

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.ConfigurableStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.delegate.DelegatingStateBackend;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+
+/**
+ * This state backend holds the working state in the underlying 
delegatedStateBackend, and forwards
+ * state changes to State Changelog.
+ */
+public class ChangelogStateBackend implements DelegatingStateBackend, 
ConfigurableStateBackend {
+
+    private static final long serialVersionUID = 1000L;
+
+    private final StateBackend delegatedStateBackend;
+
+    public ChangelogStateBackend(StateBackend stateBackend) {
+        this.delegatedStateBackend = Preconditions.checkNotNull(stateBackend);
+
+        Preconditions.checkArgument(
+                !(stateBackend instanceof ChangelogStateBackend),

Review comment:
       ```suggestion
                   !(stateBackend instanceof DelegatingStateBackend),
   ```

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackend.java
##########
@@ -182,12 +181,6 @@ public int numKeyValueStateEntries() {
         return count;
     }
 
-    @Override
-    public boolean requiresLegacySynchronousTimerSnapshots(CheckpointType 
checkpointOptions) {
-        return false;
-    }
-
-    @Override

Review comment:
       nit: these changes seem to be irrelative/unnecessary for this PR, 
especially this line (removal of the `@Override` annotation)




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