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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.changelog;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateObject;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.unmodifiableList;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * A handle to ChangelogStateBackend state. Consists of the base and delta 
parts. Base part
+ * references materialized state (e.g. SST files), while delta part references 
state changes that
+ * were not not materialized at the time of the snapshot. Both are potentially 
empty lists as there
+ * can be no state or multiple states (e.g. after rescaling).
+ */
+@Internal
+public interface ChangelogStateBackendHandle extends KeyedStateHandle {
+    List<KeyedStateHandle> getMaterializedStateHandles();
+
+    List<StateChangelogHandle> getNonMaterializedStateHandles();
+
+    class ChangelogStateBackendHandleImpl implements 
ChangelogStateBackendHandle {
+        private static final long serialVersionUID = 1L;
+        private final List<KeyedStateHandle> materialized;
+        private final List<StateChangelogHandle> nonMaterialized;
+        private final KeyGroupRange keyGroupRange;
+
+        public ChangelogStateBackendHandleImpl(
+                List<KeyedStateHandle> materialized,
+                List<StateChangelogHandle> nonMaterialized,
+                KeyGroupRange keyGroupRange) {
+            this.materialized = unmodifiableList(materialized);
+            this.nonMaterialized = unmodifiableList(nonMaterialized);
+            this.keyGroupRange = keyGroupRange;
+            checkArgument(keyGroupRange.getNumberOfKeyGroups() > 0);
+        }
+
+        @Override
+        public void registerSharedStates(SharedStateRegistry stateRegistry) {
+            stateRegistry.registerAll(materialized);
+            stateRegistry.registerAll(nonMaterialized);
+        }
+
+        @Override
+        public void discardState() throws Exception {
+            try (Closer closer = Closer.create()) {
+                materialized.forEach(h -> closer.register(asCloseable(h)));
+                nonMaterialized.forEach(h -> closer.register(asCloseable(h)));
+            }
+        }
+
+        @Override
+        public KeyGroupRange getKeyGroupRange() {
+            return keyGroupRange;
+        }
+
+        @Nullable
+        @Override
+        public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+            // todo: revisit/review
+            KeyGroupRange intersection = 
this.keyGroupRange.getIntersection(keyGroupRange);
+            if (intersection.getNumberOfKeyGroups() == 0) {
+                return null;
+            }
+            List<KeyedStateHandle> basePart =
+                    this.materialized.stream()
+                            .map(handle -> 
handle.getIntersection(keyGroupRange))
+                            .filter(Objects::nonNull)
+                            .collect(Collectors.toList());
+            List<StateChangelogHandle> deltaPart =
+                    this.nonMaterialized.stream()
+                            .map(
+                                    handle ->
+                                            (StateChangelogHandle)
+                                                    
handle.getIntersection(keyGroupRange))
+                            .filter(Objects::nonNull)
+                            .collect(Collectors.toList());
+            return new ChangelogStateBackendHandleImpl(basePart, deltaPart, 
intersection);
+        }
+
+        @Override
+        public long getStateSize() {
+            return 
materialized.stream().mapToLong(StateObject::getStateSize).sum()
+                    + 
nonMaterialized.stream().mapToLong(StateObject::getStateSize).sum();
+        }
+
+        @Override
+        public List<KeyedStateHandle> getMaterializedStateHandles() {
+            return materialized;
+        }
+
+        @Override
+        public List<StateChangelogHandle> getNonMaterializedStateHandles() {
+            return nonMaterialized;
+        }
+
+        public ChangelogStateBackendHandleImpl withDelta(StateChangelogHandle 
delta) {

Review comment:
       where is this method used?

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java
##########
@@ -240,8 +275,38 @@ public boolean 
deregisterKeySelectionListener(KeySelectionListener<K> listener)
             @Nonnull CheckpointStreamFactory streamFactory,
             @Nonnull CheckpointOptions checkpointOptions)
             throws Exception {
-        return keyedStateBackend.snapshot(
-                checkpointId, timestamp, streamFactory, checkpointOptions);
+        // The range to upload may overlap with the previous one(s). To reuse 
them, we could store
+        // the previous results either here in the backend or in the writer. 
However,
+        // materialization may truncate only a part of the previous result and 
the backend would
+        // have to split it somehow for the former option, so the latter is 
used.

Review comment:
       "materialization may truncate only a part of the previous result" I 
think I know what you mean, but how does it affect the decision to put the 
previous results to the writer?
   
   I feel here is a most natural place to hold the previous results, given all 
the materialized, non-materialized, materialize to has already been stored 
here. 
   
   WDYT

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java
##########
@@ -245,8 +280,38 @@ public boolean 
deregisterKeySelectionListener(KeySelectionListener<K> listener)
             @Nonnull CheckpointStreamFactory streamFactory,
             @Nonnull CheckpointOptions checkpointOptions)
             throws Exception {
-        return keyedStateBackend.snapshot(
-                checkpointId, timestamp, streamFactory, checkpointOptions);
+        // The range to upload may overlap with the previous one(s). To reuse 
them, we could store
+        // the previous results either here in the backend or in the writer. 
However,
+        // materialization may truncate only a part of the previous result and 
the backend would
+        // have to split it somehow for the former option, so the latter is 
used.
+        lastCheckpointId = checkpointId;
+        lastUploadedFrom = materializedTo;
+        lastUploadedTo = 
stateChangelogWriter.lastAppendedSequenceNumber().next();
+
+        LOG.debug(
+                "snapshot for checkpoint {}, change range: {}..{}",
+                checkpointId,
+                lastUploadedFrom,
+                lastUploadedTo);
+        return toRunnableFuture(
+                stateChangelogWriter
+                        .persist(lastUploadedFrom)

Review comment:
       This happens in the task thread. Does state change forwarding happens in 
the same task thread as well, I guess so. Asking to make sure `lastUploadedTo` 
and `persist` are consistent.




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