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 2022/07/08 07:10:54 UTC

[GitHub] [flink] fredia opened a new pull request, #20217: [FLINK-27693][state] Support local recovery for non-materialized part

fredia opened a new pull request, #20217:
URL: https://github.com/apache/flink/pull/20217

   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   Support local recovery for the non-materialized part of changelog.
   
   
   ## Brief change log
   
   - [8cc0d2](https://github.com/apache/flink/commit/8cc0d210c109768c705f1e4d774cf577e0d56f12) add localConfig param
   - [8f7f7dd](https://github.com/apache/flink/commit/8f7f7dd21d5b652f3de5bca0858510ca0b0d8d66) write to double stream
   - [ed4600](https://github.com/apache/flink/commit/ed46006ecf037d807be54dbf5725b5dc55f0c23c) add local state registry to discard local changelog handle
   - [8088f0](https://github.com/apache/flink/commit/8088f02c705971404a038b783e6a8c2093cf43dd) return local handle after snapshot
   
   
   ## Verifying this change
   
   Please make sure both new and modified tests in this PR follows the conventions defined in our code quality guide: https://flink.apache.org/contributing/code-style-and-quality-common.html#testing
   
   This change added tests and can be verified as follows:
   - `ChangelogLocalRecoveryITCase`
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: ( no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (JavaDocs)
   


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r940126766


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java:
##########
@@ -346,12 +376,42 @@ private static ChangelogStateHandleStreamImpl buildHandle(
             tuples.add(Tuple2.of(uploadResult.getStreamStateHandle(), uploadResult.getOffset()));
             size += uploadResult.getSize();
         }
-        return new ChangelogStateHandleStreamImpl(
-                tuples,
-                keyGroupRange,
-                size,
-                incrementalSize,
-                FsStateChangelogStorageFactory.IDENTIFIER);
+        ChangelogStateHandleStreamImpl jmChangelogStateHandle =
+                new ChangelogStateHandleStreamImpl(
+                        tuples,
+                        keyGroupRange,
+                        size,
+                        incrementalSize,
+                        FsStateChangelogStorageFactory.IDENTIFIER);
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            size = 0;
+            List<Tuple2<StreamStateHandle, Long>> localTuples = new ArrayList<>();
+            for (UploadResult uploadResult : results.values()) {
+                if (uploadResult.getLocalStreamHandleStateHandle() != null) {
+                    localTuples.add(
+                            Tuple2.of(
+                                    uploadResult.getLocalStreamHandleStateHandle(),
+                                    uploadResult.getOffset()));

Review Comment:
   The local offset is now passed and saved in `uploadResult`, but not actually used here.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939852338


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java:
##########
@@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) {
         }
     }
 
+    public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) {
+        File outDir = provider.selectAllocationBaseDirectory(0);

Review Comment:
   Yes, it should be rotated.
   I updated it with reference to :
   
   ```
   public File allocationBaseDirectory(long checkpointId) {
           return selectAllocationBaseDirectory(
                   (((int) checkpointId) & Integer.MAX_VALUE) % allocationBaseDirs.length);
       }
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on PR #20217:
URL: https://github.com/apache/flink/pull/20217#issuecomment-1203410468

   Hi @rkhachatryan, I updated this PR to handle "the last checkpoint(s) aborted and then job terminated" case, and added `LocalStateRegistryTest`. Could you please take a look again?


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r930908557


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   > I was assuming that deleting the whole directory still does not guarantee deleting the changelog (because it can reside outside); that's why there is an explicit call taskStateSnapshot.discardState(); in "normal" local state store.
   But either way, recursive deletion or the next checkpoint should discard it.
   
   Yes, deleting the whole directory still does not guarantee deleting the changelog (because it can reside outside). `ChangelogStateHandleStreamImpl` are placed in `/dstl`. 
   For **Local** `ChangelogStateHandleStreamImpl`, they are placed in <LOCAL_DIR>/taskowned, and they would be discarded by  `TaskChangelogRegistry` and  `LocalStateRegistry`. 
   So,  *recursive deletion `changelog_chk_` directory* + *`LocalStateRegistry`* + *`TaskChangelogRegistry`* are used to guarantee non-materialized part's deletion.
   
   > Could you add this comments ("total discard logic") somewhere in the code?
   
   Sure, I will add this comment in `ChangelogTaskLocalStateStore`.
   
   > Probably this case is not handled: the last checkpoint(s) aborted and then job terminated?
   
   You are right, "the last checkpoint(s) aborted and then job terminated"  can't be handled in this code. 
   Regardless of the deletion of local dstl, do you know how **remote dstl** handles this case?  if one `StateChangeSet` is uploaded to `/dstl`,  will it be removed on abort?
    
   



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r926143502


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   > we can't wrap the OutputStream with StreamCompressionDecorator in it.
   
   IIUC, we'd need to return `FSDataOutputStream` from `wrap()` instead of `BufferedOutputStream`.
   The only difficulty is `FSDataOutputStream.getPos()` method, but it can be implemented by counting written bytes?
   
   Or am I missing something?



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r926152251


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   > Yes, TaskLocalStateStore call no-op method, but TaskLocalStateStore would delete the whole chk folder(changelog_chk_ and chk_) when confirm checkpoint.
   
   I was assuming that deleting the whole directory still does not guarantee deleting the changelog (because it can reside outside); that's why there is an explicit call `taskStateSnapshot.discardState();` in "normal" local state store.
   But either way, recursive deletion or the next checkpoint should discard it.
   
   > The total discard logic of local dstl files is
   
   Thanks for the explanation. 
   Could you add this comments ("total discard logic") somewhere int the code?
   
   And I still don't see how this case is handled:
   > the last checkpoint(s) aborted and then job terminated?
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939796611


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java:
##########
@@ -181,6 +226,16 @@ public void pruneCheckpoints(LongPredicate pruningChecker, boolean breakOnceChec
     @Override
     public CompletableFuture<Void> dispose() {
         deleteMaterialization(id -> true);
+        // delete all ChangelogStateHandle in taskowned directory.
+        discardExecutor.execute(
+                () ->
+                        syncDiscardFileForCollection(

Review Comment:
   Yes, this will delete other job's states.
   I moved `aid_xx/taskowned` folder to `aid_xxx/jid_xxx/taskowned` to avoid this.
   
   > Why don't we put this logic in LocalStateRegistry - that would allow to avoid clashing and guessing the locations
   
   Because the `taskowned` folder needs to be deleted when `dispose()`, and `LocalStateRegistry` can only delete the files in the folder, can't delete the folder itself.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r920054435


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   Because compression is needed in changelog, but `CheckpointStateOutputStream` only offers the constructor with `FileBasedStateOutputStream(FileSystem fileSystem, Path path)` instead of `OutputStream`, we can't wrap the `OutputStream` with `StreamCompressionDecorator` in it. 



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r937909250


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }
+        }
+        return handle;
+    }
+
+    public void unRegister(long upTo) {
+        List<StreamStateHandle> handles = new ArrayList<>();
+        synchronized (registeredHandles) {
+            Iterator<Entry<StreamStateHandle, Long>> iterator =
+                    registeredHandles.entrySet().iterator();
+            while (iterator.hasNext()) {
+                Entry<StreamStateHandle, Long> entry = iterator.next();
+                if (entry.getValue() < upTo) {
+                    handles.add(entry.getKey());
+                    iterator.remove();
+                }
+            }
+        }
+        for (StreamStateHandle handle : handles) {
+            scheduleAsyncDelete(handle);
+        }
+    }
+
+    private void scheduleAsyncDelete(StreamStateHandle streamStateHandle) {
+        if (streamStateHandle != null) {
+            LOG.trace("Scheduled delete of state handle {}.", streamStateHandle);
+            Runnable discardRunner =
+                    () -> {
+                        try {
+                            streamStateHandle.discardState();
+                        } catch (Exception exception) {
+                            LOG.warn(
+                                    "A problem occurred during asynchronous disposal of a stream handle {}.",
+                                    streamStateHandle);
+                        }
+                    };
+            try {
+                asyncDisposalExecutor.execute(discardRunner);
+            } catch (RejectedExecutionException ex) {
+                discardRunner.run();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        synchronized (registeredHandles) {
+            open = false;
+        }
+    }

Review Comment:
   As mentioned [above](https://github.com/apache/flink/pull/20217#discussion_r937884066), this method should probably discard the remaining state.
   
   If not, and if this class will stay per task (likely not); I'd remove `open` because it requires adding `syncrhonized` sections, which IMO doesn't worth enforcing the lifecycle without any cleanup.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r923182158


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFsUploader.java:
##########
@@ -19,50 +19,30 @@
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.changelog.fs.StateChangeUploadScheduler.UploadTask;
-import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator;
-import org.apache.flink.runtime.state.StreamCompressionDecorator;
 import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
-import org.apache.flink.util.clock.Clock;
-import org.apache.flink.util.clock.SystemClock;
 
 import org.apache.flink.shaded.guava30.com.google.common.io.Closer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
 import java.util.function.BiFunction;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.core.fs.FileSystem.WriteMode.NO_OVERWRITE;
 
 /**
  * A synchronous {@link StateChangeUploadScheduler} implementation that uploads the changes using
  * {@link FileSystem}.
  */
-public class StateChangeFsUploader implements StateChangeUploader {
+public class StateChangeFsUploader extends AbstractStateChangeFsUploader {

Review Comment:
   Nice suggestion, I extracted the logic of creating stream into `prepareStream()`.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r923179460


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {
+    private static final Logger LOG = LoggerFactory.getLogger(DuplicatingOutputStreamWithPos.class);
+
+    private final OutputStream secondaryStream;
+
+    public DuplicatingOutputStreamWithPos(OutputStream primaryStream, OutputStream secondaryStream)
+            throws IOException {
+        super(primaryStream);
+        this.secondaryStream = Preconditions.checkNotNull(secondaryStream);
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        outputStream.write(b);
+        try {
+            secondaryStream.write(b);
+        } catch (Exception ex) {
+            LOG.warn("Exception encountered during write to secondary stream");
+        }

Review Comment:
   I modified it with reference to the method in `DuplicatingCheckpointOutputStream`.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939805560


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }
+        }
+        return handle;
+    }
+
+    public void unRegister(long upTo) {
+        List<StreamStateHandle> handles = new ArrayList<>();
+        synchronized (registeredHandles) {
+            Iterator<Entry<StreamStateHandle, Long>> iterator =
+                    registeredHandles.entrySet().iterator();
+            while (iterator.hasNext()) {
+                Entry<StreamStateHandle, Long> entry = iterator.next();
+                if (entry.getValue() < upTo) {
+                    handles.add(entry.getKey());
+                    iterator.remove();
+                }
+            }
+        }
+        for (StreamStateHandle handle : handles) {
+            scheduleAsyncDelete(handle);
+        }
+    }
+
+    private void scheduleAsyncDelete(StreamStateHandle streamStateHandle) {
+        if (streamStateHandle != null) {
+            LOG.trace("Scheduled delete of state handle {}.", streamStateHandle);
+            Runnable discardRunner =
+                    () -> {
+                        try {
+                            streamStateHandle.discardState();
+                        } catch (Exception exception) {
+                            LOG.warn(
+                                    "A problem occurred during asynchronous disposal of a stream handle {}.",
+                                    streamStateHandle);
+                        }
+                    };
+            try {
+                asyncDisposalExecutor.execute(discardRunner);
+            } catch (RejectedExecutionException ex) {
+                discardRunner.run();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        synchronized (registeredHandles) {
+            open = false;
+        }
+    }

Review Comment:
   Yes, it doesn't worth enforcing the lifecycle without any cleanup, removed it.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r940126766


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java:
##########
@@ -346,12 +376,42 @@ private static ChangelogStateHandleStreamImpl buildHandle(
             tuples.add(Tuple2.of(uploadResult.getStreamStateHandle(), uploadResult.getOffset()));
             size += uploadResult.getSize();
         }
-        return new ChangelogStateHandleStreamImpl(
-                tuples,
-                keyGroupRange,
-                size,
-                incrementalSize,
-                FsStateChangelogStorageFactory.IDENTIFIER);
+        ChangelogStateHandleStreamImpl jmChangelogStateHandle =
+                new ChangelogStateHandleStreamImpl(
+                        tuples,
+                        keyGroupRange,
+                        size,
+                        incrementalSize,
+                        FsStateChangelogStorageFactory.IDENTIFIER);
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            size = 0;
+            List<Tuple2<StreamStateHandle, Long>> localTuples = new ArrayList<>();
+            for (UploadResult uploadResult : results.values()) {
+                if (uploadResult.getLocalStreamHandleStateHandle() != null) {
+                    localTuples.add(
+                            Tuple2.of(
+                                    uploadResult.getLocalStreamHandleStateHandle(),
+                                    uploadResult.getOffset()));

Review Comment:
   The local offset is now passed and saved in `uploadResult`, but I think you forgot to actually used here:)



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r940264768


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java:
##########
@@ -63,6 +66,9 @@ public class FsStateChangelogStorage extends FsStateChangelogStorageForRecovery
     private final AtomicInteger logIdGenerator = new AtomicInteger(0);
 
     private final TaskChangelogRegistry changelogRegistry;
+
+    @Nullable private LocalChangelogRegistry localChangelogRegistry = null;
+
     /** The configuration for local recovery. */
     @Nonnull private final LocalRecoveryConfig localRecoveryConfig;

Review Comment:
   nit: I'd replace this pair `Nullable registry` and `Nonnull config` with some no-op `LocalChangelogRegistry`.
   In `FsStateChangelogWriter.confirm()`, we could just check `if (result.localStreamHandle != null) { ... } `



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java:
##########
@@ -104,17 +126,24 @@ public FsStateChangelogStorage(
                                 metricGroup,
                                 changelogRegistry)),
                 PREEMPTIVE_PERSIST_THRESHOLD.defaultValue().getBytes(),
-                changelogRegistry);
+                changelogRegistry,
+                localRecoveryConfig);
     }
 
     @VisibleForTesting
     public FsStateChangelogStorage(
             StateChangeUploadScheduler uploader,
             long preEmptivePersistThresholdInBytes,
-            TaskChangelogRegistry changelogRegistry) {
+            TaskChangelogRegistry changelogRegistry,
+            LocalRecoveryConfig localRecoveryConfig) {
         this.preEmptivePersistThresholdInBytes = preEmptivePersistThresholdInBytes;
         this.changelogRegistry = changelogRegistry;
         this.uploader = uploader;
+        this.localRecoveryConfig = localRecoveryConfig;
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            this.localChangelogRegistry =
+                    new LocalChangelogRegistry(Executors.newSingleThreadExecutor());

Review Comment:
   `TaskManagerServices.ioExecutor` is used for other jobs as well; re-using it might break resource isolation.
   It would make sense to share the threads between `TaskChangelogRegistry` and `LocalChangelogRegistry`, but creating two threads per job on TM seems fine too.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939839267


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {

Review Comment:
   > I guess it's discarding of local changelog; which can't be discarded by ChangelogTaskLocalStateStore because they are not bound to a specific checkpoint.
   1. Yes, I updated the javadoc of `LocalStateRegistry`;
   2. Renamed `LocalStateRegistry` to `LocalChangelogRegistry`.
   3. It has to be used by `flink-dstl` module, I annotated it with `@Internal`



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r940213842


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java:
##########
@@ -104,17 +126,24 @@ public FsStateChangelogStorage(
                                 metricGroup,
                                 changelogRegistry)),
                 PREEMPTIVE_PERSIST_THRESHOLD.defaultValue().getBytes(),
-                changelogRegistry);
+                changelogRegistry,
+                localRecoveryConfig);
     }
 
     @VisibleForTesting
     public FsStateChangelogStorage(
             StateChangeUploadScheduler uploader,
             long preEmptivePersistThresholdInBytes,
-            TaskChangelogRegistry changelogRegistry) {
+            TaskChangelogRegistry changelogRegistry,
+            LocalRecoveryConfig localRecoveryConfig) {
         this.preEmptivePersistThresholdInBytes = preEmptivePersistThresholdInBytes;
         this.changelogRegistry = changelogRegistry;
         this.uploader = uploader;
+        this.localRecoveryConfig = localRecoveryConfig;
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            this.localChangelogRegistry =
+                    new LocalChangelogRegistry(Executors.newSingleThreadExecutor());

Review Comment:
   This and `defaultChangelogRegistry` would create two executors for each job, how about passing the `TaskManagerServices.ioExecutor` for this?



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on PR #20217:
URL: https://github.com/apache/flink/pull/20217#issuecomment-1209205537

   Merged manually as 1f9632a07199854c0225bd7f416c038fbf59abe0..52eb7e76b5d66ff5c4d9d4af8a213b5b8f9f8322
   (build [succeeded](https://dev.azure.com/khachatryanroman/flink/_build/results?buildId=1600&view=results) after rebasing).


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r919544947


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   Probably this case is not handled: the last checkpoint(s) aborted and then job terminated?



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r923190093


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java:
##########
@@ -329,14 +343,28 @@ public void confirm(SequenceNumber from, SequenceNumber to) {
         uploaded.subMap(from, to).values().stream()
                 .map(UploadResult::getStreamStateHandle)
                 .forEach(changelogRegistry::stopTracking);
+
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            LocalStateRegistry localStateRegistry =
+                    localRecoveryConfig
+                            .getLocalStateRegistry()
+                            .orElseThrow(LocalRecoveryConfig.localRecoveryNotEnabled());
+
+            // transfer the control of localHandle to localStateRegistry.
+            for (UploadResult result : uploaded.subMap(from, to).values()) {
+                changelogRegistry.stopTracking(result.localStreamHandle);
+                localStateRegistry.register(result.localStreamHandle, checkpointId);
+            }
+            localStateRegistry.unRegister(checkpointId);

Review Comment:
   👍Right, we only retain local state for the last one checkpoint.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939805155


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }
+        }
+        return handle;
+    }
+
+    public void unRegister(long upTo) {
+        List<StreamStateHandle> handles = new ArrayList<>();
+        synchronized (registeredHandles) {
+            Iterator<Entry<StreamStateHandle, Long>> iterator =
+                    registeredHandles.entrySet().iterator();
+            while (iterator.hasNext()) {
+                Entry<StreamStateHandle, Long> entry = iterator.next();
+                if (entry.getValue() < upTo) {
+                    handles.add(entry.getKey());
+                    iterator.remove();
+                }

Review Comment:
   Yes,  the actual underlying file might be used by multiple state backends / tasks,  and I think the file has been **implicitly** tracked by every backend:
   
   `LocalStateRegistry#register()` is called when `ChangelogKeyedStateBackend#notifyCheckpointComplete` ->`FsStateChangelogWriter#confirm()`.  When a file is still referenced by a state backend,  the state backend would update its `LastUsedCheckpointID`. Suppose there are three state backend data in file 1,  
   ```
   File 1:
       backend A : c1, c2
       backend B : c1
       backend C : c2, c3
   ```
   
   - checkpoint 1 confirm, LastUsedCheckpointID = 1,  upTo=1 , won't delete;
   - checkpoint 2 confirm, LastUsedCheckpointID = 2, upTo=2, won't delete;
   - checkpoint 3 confirm, LastUsedCheckpointID = 3, upTo=3, won't delete;
   - checkpoint 4 confirm,  LastUsedCheckpointID = 3, upTo=4, **delete**;
   
   
   I think the role of `LocalStateRegistry` is similar to `SharedStateRegisrty` on JM. when passing the control of remote handles to JM, the remote handles are not explicitly tracked by multi-state backends, so I don't think `LocalStateRegistry` needs to be either.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r936181068


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   Because of [FLINK-28602](https://issues.apache.org/jira/browse/FLINK-28602), we also should override `close()` method.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r930714382


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   We have three options to wrap a output stream:
   
    1. Path ->`FSDataOutputStream` -> (compress) -> `OutputStream` -> (buffer) -> `BufferedOutputStream` -> (add pos) `OutputStreamWithPos or DuplicatingOutputStreamWithPos`.  
    2. Path -> `CheckpointStateOutputStream or DuplicatingCheckpointOutputStream ` -> (compress) ->  `OutputStream`  -> (counting written bytes?).
    3. Path -> `FSDataOutputStream` -> (compress) -> `OutputStream` -> (reuse `CheckpointStateOutputStream` if it offers a constructor with `OutputStream` parameter) -> `CheckpointStateOutputStream or DuplicatingCheckpointOutputStream`.
    
   Option 1 is the current implementation.
   
   Is option 2 what you want to express?  I think the `getPos()` before and after compression are not the same, maybe we still need a wrapper class to count written bytes.
   
   Option 3 needs to modify the constructor method of `CheckpointStateOutputStream`.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   We have three options to wrap an output stream:
   
    1. Path ->`FSDataOutputStream` -> (compress) -> `OutputStream` -> (buffer) -> `BufferedOutputStream` -> (add pos) `OutputStreamWithPos or DuplicatingOutputStreamWithPos`.  
    2. Path -> `CheckpointStateOutputStream or DuplicatingCheckpointOutputStream ` -> (compress) ->  `OutputStream`  -> (counting written bytes?).
    3. Path -> `FSDataOutputStream` -> (compress) -> `OutputStream` -> (reuse `CheckpointStateOutputStream` if it offers a constructor with `OutputStream` parameter) -> `CheckpointStateOutputStream or DuplicatingCheckpointOutputStream`.
    
   Option 1 is the current implementation.
   
   Is option 2 what you want to express?  I think the `getPos()` before and after compression are not the same, maybe we still need a wrapper class to count written bytes.
   
   Option 3 needs to modify the constructor method of `CheckpointStateOutputStream`.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r937962534


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   Yes, I meant something like (2):
   - `FileBasedStateOutputStream` 
   - decorated by `StreamCompressionDecorator`
   - wrapped by `BufferedOutputStream` 
   - wrapped into `CheckpointStateOutputStream` 
   
   Adding buffering and `getPos` might be problematic though. 
   
   I just noticed that the current local recovery stream (for Heap and RocksDB metadata) is also NOT buffered. 
   So I'd add a ticket to add buffering there, and then reconsider this issue (leaving the code as is until then).
   
   WDYT?
   



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r919521374


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   Is this class necessary? Can't the existing `DuplicatingCheckpointOutputStream` be used instead?



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {
+    private static final Logger LOG = LoggerFactory.getLogger(DuplicatingOutputStreamWithPos.class);
+
+    private final OutputStream secondaryStream;
+
+    public DuplicatingOutputStreamWithPos(OutputStream primaryStream, OutputStream secondaryStream)
+            throws IOException {
+        super(primaryStream);
+        this.secondaryStream = Preconditions.checkNotNull(secondaryStream);
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        outputStream.write(b);
+        try {
+            secondaryStream.write(b);
+        } catch (Exception ex) {
+            LOG.warn("Exception encountered during write to secondary stream");
+        }

Review Comment:
   if this class remains:  think such error handling might result in data loss on recovery.
   
   ditt: other methods



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFsUploader.java:
##########
@@ -19,50 +19,30 @@
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.changelog.fs.StateChangeUploadScheduler.UploadTask;
-import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator;
-import org.apache.flink.runtime.state.StreamCompressionDecorator;
 import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
-import org.apache.flink.util.clock.Clock;
-import org.apache.flink.util.clock.SystemClock;
 
 import org.apache.flink.shaded.guava30.com.google.common.io.Closer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
 import java.util.function.BiFunction;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.core.fs.FileSystem.WriteMode.NO_OVERWRITE;
 
 /**
  * A synchronous {@link StateChangeUploadScheduler} implementation that uploads the changes using
  * {@link FileSystem}.
  */
-public class StateChangeFsUploader implements StateChangeUploader {
+public class StateChangeFsUploader extends AbstractStateChangeFsUploader {

Review Comment:
   `AbstractStateChangeFsUploader` was extracted and is extended by `DuplicatingStateChangeFsUploader` and `StateChangeFsUploader`.
   It has some `protected` methods that are used by descendants; and no `abstract` methods. And the calling methods in descendants (`upload`) have some duplication.
   
   That means there is something wrong with the design.
   How about applying template method pattern and having `upload()` in the base class and some `abstract` methods to customize it (i.e. create stream and handle errors)?
   
   (alternatively, the two classes can be just implementers; code can be shared via composition or just static methods)
   
   WDYT?



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingStateChangeFsUploader.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.changelog.fs.StateChangeUploadScheduler.UploadTask;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.LocalRecoveryDirectoryProvider;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+
+import org.apache.flink.shaded.guava30.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.CHECKPOINT_TASK_OWNED_STATE_DIR;
+
+/** A StateChangeFsUploader implementation that writes the changes to remote and local. */
+public class DuplicatingStateChangeFsUploader extends AbstractStateChangeFsUploader {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DuplicatingStateChangeFsUploader.class);
+
+    private final Path basePath;
+    private final FileSystem fileSystem;
+    private LocalRecoveryDirectoryProvider localRecoveryDirectoryProvider;

Review Comment:
   Can be final.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java:
##########
@@ -329,14 +343,28 @@ public void confirm(SequenceNumber from, SequenceNumber to) {
         uploaded.subMap(from, to).values().stream()
                 .map(UploadResult::getStreamStateHandle)
                 .forEach(changelogRegistry::stopTracking);
+
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            LocalStateRegistry localStateRegistry =
+                    localRecoveryConfig
+                            .getLocalStateRegistry()
+                            .orElseThrow(LocalRecoveryConfig.localRecoveryNotEnabled());
+
+            // transfer the control of localHandle to localStateRegistry.
+            for (UploadResult result : uploaded.subMap(from, to).values()) {
+                changelogRegistry.stopTracking(result.localStreamHandle);
+                localStateRegistry.register(result.localStreamHandle, checkpointId);

Review Comment:
   Null check for `result.localStreamHandle`?
   



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {
+    private static final Logger LOG = LoggerFactory.getLogger(DuplicatingOutputStreamWithPos.class);
+
+    private final OutputStream secondaryStream;
+
+    public DuplicatingOutputStreamWithPos(OutputStream primaryStream, OutputStream secondaryStream)
+            throws IOException {
+        super(primaryStream);
+        this.secondaryStream = Preconditions.checkNotNull(secondaryStream);
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        outputStream.write(b);
+        try {
+            secondaryStream.write(b);
+        } catch (Exception ex) {
+            LOG.warn("Exception encountered during write to secondary stream");
+        }
+        pos++;
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException {
+        outputStream.write(b);
+        try {
+            secondaryStream.write(b);
+        } catch (Exception ex) {
+            LOG.warn("Exception encountered during write to secondary stream");
+        }
+        pos += b.length;
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+        outputStream.write(b, off, len);
+        try {
+            secondaryStream.write(b, off, len);
+        } catch (Exception ex) {
+            LOG.warn("Exception encountered during writing to secondary stream");
+        }
+        pos += len;
+    }
+
+    @Override
+    public void flush() throws IOException {
+        outputStream.flush();
+        try {
+            secondaryStream.flush();
+        } catch (Exception ex) {
+            LOG.warn("Exception encountered during flushing secondary stream");
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        outputStream.close();
+        secondaryStream.close();

Review Comment:
   if this class remains:  shouldn't both `close` be wrapped with `try`, or `Closer`?



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/UploadResult.java:
##########
@@ -21,12 +21,15 @@
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.changelog.SequenceNumber;
 
+import javax.annotation.Nullable;
+
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /** Result of uploading state changes. */
 @Internal
 public final class UploadResult {
     public final StreamStateHandle streamStateHandle;
+    public final StreamStateHandle localStreamHandle;

Review Comment:
   Mark `@Nullable`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   :+1: IIUC, the discard logic is correct:
   1. `(Changelog)TaskLocalStateStore` calls this no-op method
   1. `LocalStateRegistry` calls `discardState()` on "low-level" handle (`FileStateHandle`) - on checkpoint confirmation
   1. `TaskChangelogRegistry` - calls `discardState()` on "low-level" handle (`FileStateHandle`) - when upload is no more needed
   1. As opposed to "normal" local state, local changelog state is *not* discarded on recovery (because of 1); but it will be discarded on the 1st checkpoint confirmation
   
   Can you confirm that?
   
   Although I found it tricky to understand, so it makes sense to document it somewhere, WDYT?
   
   (and also to cover with tests).



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java:
##########
@@ -329,14 +343,28 @@ public void confirm(SequenceNumber from, SequenceNumber to) {
         uploaded.subMap(from, to).values().stream()
                 .map(UploadResult::getStreamStateHandle)
                 .forEach(changelogRegistry::stopTracking);
+
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            LocalStateRegistry localStateRegistry =
+                    localRecoveryConfig
+                            .getLocalStateRegistry()
+                            .orElseThrow(LocalRecoveryConfig.localRecoveryNotEnabled());
+
+            // transfer the control of localHandle to localStateRegistry.
+            for (UploadResult result : uploaded.subMap(from, to).values()) {
+                changelogRegistry.stopTracking(result.localStreamHandle);
+                localStateRegistry.register(result.localStreamHandle, checkpointId);
+            }
+            localStateRegistry.unRegister(checkpointId);

Review Comment:
   Just to clarify: we only retain local state for the last checkpoint (num retained checkpoint is ignored). Same as the current local recovery, right?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry {

Review Comment:
   I'm thinking about the case when local changelog state explodes and causes `no space left on device`.
   The host might become unresponsive and it will be difficult to debug such an issue.
   
   To ease the debug, we should probably monitor the space taken by the changelog; and stop writing locally or throw an exception when it takes too much (above some configured limit).
   This class seems the best place for such monitoring.
   
   This can be a separate ticket though, at least if local recovery is not the default.
   
   WDYT?
   



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r937909250


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }
+        }
+        return handle;
+    }
+
+    public void unRegister(long upTo) {
+        List<StreamStateHandle> handles = new ArrayList<>();
+        synchronized (registeredHandles) {
+            Iterator<Entry<StreamStateHandle, Long>> iterator =
+                    registeredHandles.entrySet().iterator();
+            while (iterator.hasNext()) {
+                Entry<StreamStateHandle, Long> entry = iterator.next();
+                if (entry.getValue() < upTo) {
+                    handles.add(entry.getKey());
+                    iterator.remove();
+                }
+            }
+        }
+        for (StreamStateHandle handle : handles) {
+            scheduleAsyncDelete(handle);
+        }
+    }
+
+    private void scheduleAsyncDelete(StreamStateHandle streamStateHandle) {
+        if (streamStateHandle != null) {
+            LOG.trace("Scheduled delete of state handle {}.", streamStateHandle);
+            Runnable discardRunner =
+                    () -> {
+                        try {
+                            streamStateHandle.discardState();
+                        } catch (Exception exception) {
+                            LOG.warn(
+                                    "A problem occurred during asynchronous disposal of a stream handle {}.",
+                                    streamStateHandle);
+                        }
+                    };
+            try {
+                asyncDisposalExecutor.execute(discardRunner);
+            } catch (RejectedExecutionException ex) {
+                discardRunner.run();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        synchronized (registeredHandles) {
+            open = false;
+        }
+    }

Review Comment:
   As mentioned above, this method should probably discard the remaining state.
   
   If not, and if this class will stay per task (likely not); I'd remove `open` because it requires adding `syncrhonized` sections, which IMO doesn't worth enforcing the lifecycle without any cleanup.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {

Review Comment:
   1. Could you please state the purpose of this class in the javadoc; given that there are `ChangelogTaskLocalStateStore` and `TaskChangelogRegistry` already? I guess it's discarding of local changelog; which can't be discarded by `ChangelogTaskLocalStateStore` because they are not bound to a specific checkpoint.
   1. The name `LocalStateRegistry` is quite generic. How about `LocalTaskChangelogRegistry` or `LocalChangelogRegistry`?
   1. Does it have to be `public`? If so, please annotate it with `@Internal`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }
+        }
+        return handle;
+    }
+
+    public void unRegister(long upTo) {

Review Comment:
   I'd add some javadoc explaining when this method is called.
   
   I'd also rename it to reflect what it actually does. WDYT about `discardUpToCheckpoint(long checkpointID)`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java:
##########
@@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) {
         }
     }
 
+    public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) {
+        File outDir = provider.selectAllocationBaseDirectory(0);
+        if (!outDir.exists() && !outDir.mkdirs()) {
+            LOG.error(
+                    "Local state base directory does not exist and could not be created: "
+                            + outDir);
+        }
+        return new Path(outDir.toURI().toString(), CHECKPOINT_TASK_OWNED_STATE_DIR);
+    }
+
+    @Override
+    public void abortCheckpoint(long abortedCheckpointId) {
+
+        LOG.debug(
+                "Received abort information for checkpoint {} in subtask ({} - {} - {}). Starting to prune history.",
+                abortedCheckpointId,
+                jobID,
+                jobVertexID,
+                subtaskIndex);
+
+        pruneCheckpoints(
+                snapshotCheckpointId -> snapshotCheckpointId == abortedCheckpointId, false);
+
+        // Local store only keeps one checkpoint, discard all changelog handle in taskowned
+        // directory.
+        // Scenarios:
+        //   cp1: m1
+        //   confirm cp1, do nothing
+        //   cp2: m1, c1
+        //   abort cp2, delete m1, c1
+        //   cp3: m1, c1, c2
+        //   confirm cp3, do nothing
+        //   -> if failover, restore from local cp3 will fail, because m1 does not exist, c1 may not
+        // exist either(depend on BatchingStateChangeUploadScheduler).
+        File[] fileInTaskOwned =
+                new File(getLocalTaskOwnedDirectory(getLocalRecoveryDirectoryProvider()).toUri())
+                        .listFiles();
+        syncDiscardFileForCollection(
+                fileInTaskOwned == null ? Collections.emptyList() : Arrays.asList(fileInTaskOwned));

Review Comment:
   Do we really need this if there's a similar deletion in `dispose()`?
   I'm concerned that this will delete ALL local state, even the used one.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }

Review Comment:
   How about simplifying it with:
   ```
   registeredHandles.put(
       handle,
       Math.max(checkpointID, registeredHandles.getOrDefault(handle, -1L)));
   ```
   ?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }
+        }
+        return handle;

Review Comment:
   The returned value is never used.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java:
##########
@@ -346,12 +376,42 @@ private static ChangelogStateHandleStreamImpl buildHandle(
             tuples.add(Tuple2.of(uploadResult.getStreamStateHandle(), uploadResult.getOffset()));
             size += uploadResult.getSize();
         }
-        return new ChangelogStateHandleStreamImpl(
-                tuples,
-                keyGroupRange,
-                size,
-                incrementalSize,
-                FsStateChangelogStorageFactory.IDENTIFIER);
+        ChangelogStateHandleStreamImpl jmChangelogStateHandle =
+                new ChangelogStateHandleStreamImpl(
+                        tuples,
+                        keyGroupRange,
+                        size,
+                        incrementalSize,
+                        FsStateChangelogStorageFactory.IDENTIFIER);
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            size = 0;
+            List<Tuple2<StreamStateHandle, Long>> localTuples = new ArrayList<>();
+            for (UploadResult uploadResult : results.values()) {
+                if (uploadResult.getLocalStreamHandleStateHandle() != null) {
+                    localTuples.add(
+                            Tuple2.of(
+                                    uploadResult.getLocalStreamHandleStateHandle(),
+                                    uploadResult.getOffset()));

Review Comment:
   This makes `FsStateChangelogWriter` be aware that local and remote handles use the same offset.
   
   But the decision to reuse the offset is made in `AbstractStateChangeFsUploader.uploadInternal`.
   So I'd introduce a separate field for local offset in `UploadTasksResult` and `UploadResult` and pass the same offset there.
   
   That would make it more clear and easier to change.
   
   WDYT?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java:
##########
@@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) {
         }
     }
 
+    public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) {
+        File outDir = provider.selectAllocationBaseDirectory(0);
+        if (!outDir.exists() && !outDir.mkdirs()) {
+            LOG.error(
+                    "Local state base directory does not exist and could not be created: "
+                            + outDir);
+        }
+        return new Path(outDir.toURI().toString(), CHECKPOINT_TASK_OWNED_STATE_DIR);
+    }
+
+    @Override
+    public void abortCheckpoint(long abortedCheckpointId) {
+
+        LOG.debug(
+                "Received abort information for checkpoint {} in subtask ({} - {} - {}). Starting to prune history.",
+                abortedCheckpointId,
+                jobID,
+                jobVertexID,
+                subtaskIndex);
+
+        pruneCheckpoints(
+                snapshotCheckpointId -> snapshotCheckpointId == abortedCheckpointId, false);
+
+        // Local store only keeps one checkpoint, discard all changelog handle in taskowned
+        // directory.
+        // Scenarios:
+        //   cp1: m1
+        //   confirm cp1, do nothing
+        //   cp2: m1, c1
+        //   abort cp2, delete m1, c1
+        //   cp3: m1, c1, c2
+        //   confirm cp3, do nothing
+        //   -> if failover, restore from local cp3 will fail, because m1 does not exist, c1 may not
+        // exist either(depend on BatchingStateChangeUploadScheduler).
+        File[] fileInTaskOwned =
+                new File(getLocalTaskOwnedDirectory(getLocalRecoveryDirectoryProvider()).toUri())
+                        .listFiles();
+        syncDiscardFileForCollection(
+                fileInTaskOwned == null ? Collections.emptyList() : Arrays.asList(fileInTaskOwned));

Review Comment:
   Do we really need this if there's a similar deletion in `dispose()`?
   I'm concerned that this will delete ALL local state, even the used one.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;

Review Comment:
   1. Why don't we use `PhysicalStateHandleID` as a key, so that we don't rely on handle equality/hashCode?
   2. I'd add "checkpoint" to the field name to make the contents clear, something like `handleToLastUsedCheckpointID`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java:
##########
@@ -101,6 +103,7 @@ public TaskExecutorLocalStateStoresManager(
         this.discardExecutor = discardExecutor;
         this.lock = new Object();
         this.closed = false;
+        this.localStateRegistry = new LocalStateRegistry(discardExecutor);

Review Comment:
   I think the lifecycle of this object (`LocalStateRegistry`) should be per job.
   The code here assumes it is per TM.
   The code in `LocalStateRegistry` probably assumes it is per backend.
   
   If per job, a good candidate to manage it would be `TaskExecutorLocalStateStoresManager`.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingStateChangeFsUploader.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.LocalRecoveryDirectoryProvider;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.runtime.state.ChangelogTaskLocalStateStore.getLocalTaskOwnedDirectory;
+
+/**
+ * A StateChangeFsUploader implementation that writes the changes to remote and local.
+ *
+ * <p>The total discard logic of local dstl files is:
+ *

Review Comment:
   I think `ChangelogTaskLocalStateStore` should also be mentioned here
   IIUC, it is invoked twice:
   - after 1 (`storeLocalState` from `AsyncCheckpointRunnable`)
   - after 2 (`confirmCheckpoint`)
   
   Could you please also clarify the responsibilities:
   - `LocalStateRegistry` and `ChangelogTaskLocalStateStore` - local files (managed purely by TM)
   - `TaskChangelogRegistry` - "remote" files (managed also by JM);



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {
+        synchronized (registeredHandles) {
+            checkState(open, "Attempt to register state to closed LocalStateRegistry.");
+            if (registeredHandles.containsKey(handle)) {
+                long pre = registeredHandles.get(handle);
+                if (checkpointID > pre) {
+                    registeredHandles.put(handle, checkpointID);
+                }
+            } else {
+                registeredHandles.put(handle, checkpointID);
+            }
+        }
+        return handle;
+    }
+
+    public void unRegister(long upTo) {
+        List<StreamStateHandle> handles = new ArrayList<>();
+        synchronized (registeredHandles) {
+            Iterator<Entry<StreamStateHandle, Long>> iterator =
+                    registeredHandles.entrySet().iterator();
+            while (iterator.hasNext()) {
+                Entry<StreamStateHandle, Long> entry = iterator.next();
+                if (entry.getValue() < upTo) {
+                    handles.add(entry.getKey());
+                    iterator.remove();
+                }

Review Comment:
   I'm afraid this is not enough because the actual underlying file might be used by multiple state backends / tasks (`AbstractStateChangeFsUploader` is shared by multiple of them).
   
   So we either need to 
   - track usage by backend (similar to `TaskChangelogRegistry`)
   - or create a separate duplicating stream per state backend - and manage separate offsets



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java:
##########
@@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) {
         }
     }
 
+    public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) {
+        File outDir = provider.selectAllocationBaseDirectory(0);

Review Comment:
   Why the index is always `0`, shouldn't it be rotated?
   Something like `random.nextInt(numDirs)`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;
+
+    /** This flag indicates whether the registry is open or if close() was called. */
+    private boolean open;
+
+    /** Executor for async state deletion */
+    private final Executor asyncDisposalExecutor;
+
+    public LocalStateRegistry(Executor ioExecutor) {
+        this.registeredHandles = new HashMap<>();
+        this.asyncDisposalExecutor = ioExecutor;
+        this.open = true;
+    }
+
+    public StreamStateHandle register(StreamStateHandle handle, long checkpointID) {

Review Comment:
   I'd add some javadoc explaining when this method is called.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java:
##########
@@ -181,6 +226,16 @@ public void pruneCheckpoints(LongPredicate pruningChecker, boolean breakOnceChec
     @Override
     public CompletableFuture<Void> dispose() {
         deleteMaterialization(id -> true);
+        // delete all ChangelogStateHandle in taskowned directory.
+        discardExecutor.execute(
+                () ->
+                        syncDiscardFileForCollection(

Review Comment:
   Won't this clash with other jobs' state?
   
   Why don't we put this logic in `LocalStateRegistry` - that would allow to avoid clashing and guessing the locations.
   
   WDYT?



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r926143658


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java:
##########
@@ -329,14 +343,28 @@ public void confirm(SequenceNumber from, SequenceNumber to) {
         uploaded.subMap(from, to).values().stream()
                 .map(UploadResult::getStreamStateHandle)
                 .forEach(changelogRegistry::stopTracking);
+
+        if (localRecoveryConfig.isLocalRecoveryEnabled()) {
+            LocalStateRegistry localStateRegistry =
+                    localRecoveryConfig
+                            .getLocalStateRegistry()
+                            .orElseThrow(LocalRecoveryConfig.localRecoveryNotEnabled());
+
+            // transfer the control of localHandle to localStateRegistry.
+            for (UploadResult result : uploaded.subMap(from, to).values()) {
+                changelogRegistry.stopTracking(result.localStreamHandle);
+                localStateRegistry.register(result.localStreamHandle, checkpointId);
+            }
+            localStateRegistry.unRegister(checkpointId);

Review Comment:
   Thanks!



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r919537669


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   :+1: IIUC, discard logic is correct:
   1. `(Changelog)TaskLocalStateStore` calls this no-op method
   1. `LocalStateRegistry` calls `discardState()` on "low-level" handle (`FileStateHandle`) - on checkpoint confirmation
   1. `TaskChangelogRegistry` - calls `discardState()` on "low-level" handle (`FileStateHandle`) - when upload is no more needed
   1. As opposed to "normal" local state, local changelog state is *not* discarded on recovery (because of 1); but it will be discarded on the 1st checkpoint confirmation
   
   Can you confirm that?
   
   Although I found it tricky to understand, so it makes sense to document it somewhere, WDYT?
   
   (and also to cover with tests).



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r919521941


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {
+    private static final Logger LOG = LoggerFactory.getLogger(DuplicatingOutputStreamWithPos.class);
+
+    private final OutputStream secondaryStream;
+
+    public DuplicatingOutputStreamWithPos(OutputStream primaryStream, OutputStream secondaryStream)
+            throws IOException {
+        super(primaryStream);
+        this.secondaryStream = Preconditions.checkNotNull(secondaryStream);
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        outputStream.write(b);
+        try {
+            secondaryStream.write(b);
+        } catch (Exception ex) {
+            LOG.warn("Exception encountered during write to secondary stream");
+        }

Review Comment:
   if this class remains: I think such error handling might result in data loss on recovery.
   
   ditt: other methods



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r924060396


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry {

Review Comment:
   Nice suggestion, another tick for monitoring the space taken by the changelog could be created later, but we may not have enough time to finish it in v1.16(Changelog V2).



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r930908557


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   > I was assuming that deleting the whole directory still does not guarantee deleting the changelog (because it can reside outside); that's why there is an explicit call taskStateSnapshot.discardState(); in "normal" local state store.
   But either way, recursive deletion or the next checkpoint should discard it.
   
   Yes, deleting the whole directory still does not guarantee deleting the changelog (because it can reside outside). `ChangelogStateHandleStreamImpl` are placed in `/dstl`. 
   For **Local** `ChangelogStateHandleStreamImpl`, they are placed in <LOCAL_DIR>/taskowned, and they would be discarded by  `TaskChangelogRegistry` and  `LocalStateRegistry`. 
   So,  *recursive deletion `changelog_chk_` directory* + *`LocalStateRegistry`* + *`TaskChangelogRegistry`* are used to guarantee non-materialized part's deletion.
   
   > Could you add this comments ("total discard logic") somewhere in the code?
   
   Sure, I added this comment in `DuplicatingStateChangeFsUploader`.
   
   > Probably this case is not handled: the last checkpoint(s) aborted and then job terminated?
   
   You are right, "the last checkpoint(s) aborted and then job terminated"  can't be handled before.  I updated [`ChangelogTaskLocalStateStore#abortCheckpoint`] (https://github.com/apache/flink/pull/20217/commits/7b6e986e75913a93bbbd6a9d477205e742876995#diff-e30bc79ffaff8b5737b1df46462536a6b3544688139b15a7baa6b3569d8b5d6cR139-R144) to handle this case.
   
   Regardless of the deletion of local dstl, do you know how **remote dstl** handles this case?  if one `StateChangeSet` is uploaded to `/dstl`,  will it be removed on abort?



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r940120698


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java:
##########
@@ -101,6 +103,7 @@ public TaskExecutorLocalStateStoresManager(
         this.discardExecutor = discardExecutor;
         this.lock = new Object();
         this.closed = false;
+        this.localStateRegistry = new LocalStateRegistry(discardExecutor);

Review Comment:
   Yes, I meant per job per TM, so that state backends of the same job running on the same TM should share this object.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java:
##########
@@ -101,6 +103,7 @@ public TaskExecutorLocalStateStoresManager(
         this.discardExecutor = discardExecutor;
         this.lock = new Object();
         this.closed = false;
+        this.localStateRegistry = new LocalStateRegistry(discardExecutor);

Review Comment:
   Yes, I meant per job per TM, so that state backends of the same job running on the same TM should share this object.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939797960


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java:
##########
@@ -101,6 +103,7 @@ public TaskExecutorLocalStateStoresManager(
         this.discardExecutor = discardExecutor;
         this.lock = new Object();
         this.closed = false;
+        this.localStateRegistry = new LocalStateRegistry(discardExecutor);

Review Comment:
   The lifecycle of `LocalStateRegistry` can only be per TM, because `LocalStateRegistry` is running on TM, it can't access other tasks on other TMs. 
   



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan closed pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan closed pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part
URL: https://github.com/apache/flink/pull/20217


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20217:
URL: https://github.com/apache/flink/pull/20217#issuecomment-1178645915

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8088f02c705971404a038b783e6a8c2093cf43dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8088f02c705971404a038b783e6a8c2093cf43dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8088f02c705971404a038b783e6a8c2093cf43dd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r924056241


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   > 1. (Changelog)TaskLocalStateStore calls this no-op method
   
   Yes, `TaskLocalStateStore` call no-op method, but `TaskLocalStateStore` would delete the whole chk folder(`changelog_chk_` and `chk_`) when confirm checkpoint. 
   
   > 4. As opposed to "normal" local state, local changelog state is not discarded on recovery (because of 1); but it will be discarded on the 1st checkpoint confirmation
   
   Like "normal" local state, local changelog state would **be discarded** on recovery, it would be delete when `deleteDirectory()`, because of `recursive=true`.
   
   ```java
   protected void deleteDirectory(File directory) throws IOException {
           Path path = new Path(directory.toURI());
           FileSystem fileSystem = path.getFileSystem();
           if (fileSystem.exists(path)) {
               fileSystem.delete(path, true);
           }
       }
   ```
   
   The total discard logic of local dstl files is:
   1. Register files to `TaskChangelogRegistry#startTracking()` on `StateChangeUploader#upload()`.
   2.  Pass control of the file to `LocalStateRegistry#register()` **when** `confirm()` , files of the previous checkpoint will be unregistered/deleted by `LocalStateRegistry#unRegister()`  at the same time ; for remote file, we pass control of the file to JM.
   3. When `materialized()` or `checkpointSubsumed()` is called, `TaskChangelogRegistry#notUse()` is responsible for deleting local files. BTW, it is possible that the file has already been truncated by `LocalStateRegistry`.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r938674520


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/DuplicatingOutputStreamWithPos.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.runtime.state.DuplicatingCheckpointOutputStream;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A DuplicatingOutputStreamWithPos is similar to {@link DuplicatingCheckpointOutputStream} which
+ * wraps a primary and a secondary OutputStream and duplicates all writes into both streams. The
+ * difference is that this stream does not delete the file when {@link #close()}.
+ */
+class DuplicatingOutputStreamWithPos extends OutputStreamWithPos {

Review Comment:
   Sounds good, thanks.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r940125933


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;

Review Comment:
   1. How about `Map<PhysicalStateHandleID, Tuple<StreamStateHandle, Long>>`?



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939812417


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateRegistry.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This registry manages handles which is written for local recovery. */
+public class LocalStateRegistry implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalStateRegistry.class);
+    /**
+     * All registered handles. (handle,checkpointID) represents a handle and the latest checkpoint
+     * that refer to this handle.
+     */
+    private final Map<StreamStateHandle, Long> registeredHandles;

Review Comment:
   1. Because `StreamStateHandle.discardState()` needs to be called when `unRegister`, `PhysicalStateHandleID` doesn't have access to  `discardState()`;
   2. 👍Renamed `registeredHandles` to `handleToLastUsedCheckpointID`.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r939841395


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/ChangelogTaskLocalStateStore.java:
##########
@@ -97,6 +101,47 @@ private void updateReference(long checkpointId, TaskStateSnapshot localState) {
         }
     }
 
+    public static Path getLocalTaskOwnedDirectory(LocalRecoveryDirectoryProvider provider) {
+        File outDir = provider.selectAllocationBaseDirectory(0);
+        if (!outDir.exists() && !outDir.mkdirs()) {
+            LOG.error(
+                    "Local state base directory does not exist and could not be created: "
+                            + outDir);
+        }
+        return new Path(outDir.toURI().toString(), CHECKPOINT_TASK_OWNED_STATE_DIR);
+    }
+
+    @Override
+    public void abortCheckpoint(long abortedCheckpointId) {
+
+        LOG.debug(
+                "Received abort information for checkpoint {} in subtask ({} - {} - {}). Starting to prune history.",
+                abortedCheckpointId,
+                jobID,
+                jobVertexID,
+                subtaskIndex);
+
+        pruneCheckpoints(
+                snapshotCheckpointId -> snapshotCheckpointId == abortedCheckpointId, false);
+
+        // Local store only keeps one checkpoint, discard all changelog handle in taskowned
+        // directory.
+        // Scenarios:
+        //   cp1: m1
+        //   confirm cp1, do nothing
+        //   cp2: m1, c1
+        //   abort cp2, delete m1, c1
+        //   cp3: m1, c1, c2
+        //   confirm cp3, do nothing
+        //   -> if failover, restore from local cp3 will fail, because m1 does not exist, c1 may not
+        // exist either(depend on BatchingStateChangeUploadScheduler).
+        File[] fileInTaskOwned =
+                new File(getLocalTaskOwnedDirectory(getLocalRecoveryDirectoryProvider()).toUri())
+                        .listFiles();
+        syncDiscardFileForCollection(
+                fileInTaskOwned == null ? Collections.emptyList() : Arrays.asList(fileInTaskOwned));

Review Comment:
   You're right, I moved the deletion to `LocalStateRegistry#prune` to handle "the last checkpoint(s) aborted and then job terminated".



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20217: [FLINK-27693][state] Support local recovery for non-materialized part

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20217:
URL: https://github.com/apache/flink/pull/20217#discussion_r937925172


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateHandleID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class);
+    private final List<KeyedStateHandle> localMaterialized;
+    private final List<ChangelogStateHandle> localNonMaterialized;
+    private final ChangelogStateBackendHandleImpl remoteHandle;
+
+    public ChangelogStateBackendLocalHandle(
+            List<KeyedStateHandle> localMaterialized,
+            List<ChangelogStateHandle> localNonMaterialized,
+            ChangelogStateBackendHandleImpl remoteHandle) {
+        this.localMaterialized = localMaterialized;
+        this.localNonMaterialized = localNonMaterialized;
+        this.remoteHandle = remoteHandle;
+    }
+
+    @Override
+    public List<KeyedStateHandle> getMaterializedStateHandles() {
+        return localMaterialized;
+    }
+
+    @Override
+    public List<ChangelogStateHandle> getNonMaterializedStateHandles() {
+        return localNonMaterialized;
+    }
+
+    @Override
+    public long getMaterializationID() {
+        return remoteHandle.getMaterializationID();
+    }
+
+    @Override
+    public ChangelogStateBackendHandle rebound(long checkpointId) {
+        return remoteHandle.rebound(checkpointId);
+    }
+
+    public List<KeyedStateHandle> getRemoteMaterializedStateHandles() {
+        return remoteHandle.getMaterializedStateHandles();
+    }
+
+    public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() {
+        return remoteHandle.getNonMaterializedStateHandles();
+    }
+
+    @Override
+    public long getCheckpointId() {
+        return remoteHandle.getCheckpointId();
+    }
+
+    @Override
+    public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+        remoteHandle.registerSharedStates(stateRegistry, checkpointID);
+    }
+
+    @Override
+    public long getCheckpointedSize() {
+        return remoteHandle.getCheckpointedSize();
+    }
+
+    @Override
+    public KeyGroupRange getKeyGroupRange() {
+        return remoteHandle.getKeyGroupRange();
+    }
+
+    @Nullable
+    @Override
+    public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+        throw new UnsupportedOperationException(
+                "This is a local state handle for the TM side only.");
+    }
+
+    @Override
+    public StateHandleID getStateHandleId() {
+        return remoteHandle.getStateHandleId();
+    }
+
+    @Override
+    public void discardState() throws Exception {}

Review Comment:
   Thanks! 
   I've commented on the updated code.
   
   > Regardless of the deletion of local dstl, do you know how remote dstl handles this case? if one StateChangeSet is uploaded to /dstl, will it be removed on abort?
   
   Good question. For the remote state, TM can not make any assumptions whether it's in use or not, so this is managed by JM. The class responsible for that is checkpoint store, e.g. [`DefaultCompletedCheckpointStore.shutdown()`](https://github.com/apache/flink/blob/5405239dec0884dff746129c73955c90f455c465/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStore.java#L208).



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org