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/10/13 09:26:13 UTC

[GitHub] [flink] fredia opened a new pull request, #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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

   <!--
   *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
   
   Improve logging in SharedStateRegistry
   and make the conditional logic more clear.
   This PR is copied from https://github.com/apache/flink/pull/20722 and address comment.
   
   ## Verifying this change
   
   This change is a trivial rework without any (new) test coverage.
   
   ## 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): no
     - 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?  no
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
   


-- 
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] klion26 commented on a diff in pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   We only use placeholder state handle if the checkpoint contains sst has been confirmed



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   Aha, I missed this point, thanks for your clarification. 
   
   Since every upload will give a [new fileName](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java#L442) for SST,  is this(delete the old incompleted checkpoint's statehandle) a bug?
   
   And I accidentally deleted the branch of this PR, I will create a new one to changelog the log level.
    



-- 
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 closed pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

Posted by GitBox <gi...@apache.org>.
fredia closed pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry
URL: https://github.com/apache/flink/pull/21050


-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   > did not confirm the logic for changelog state backend now, could you please check this logic and update it?
   
   If Changelog state backend delegate Rocksdb state backend, this wrong logic still exist.
   
   
   I will try to add an ITCase to mock a whole checkpoint and restore scenario, If it is really like the analysis, I will create a ticket.



-- 
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] klion26 commented on a diff in pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   Ah... seems there is a bug in [SharedStateRegistryImpl](https://github.com/apache/flink/blob/5d4a2b6674a2593de2a7e7efa29f55e7e36b4d0b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java#L130), we can't delete the registered state handle here -- we need to reuse the old statehandle and discard the new statehandle for incremental checkpoint, because the metadata contains the old statehandle has already been writen out(can't change it), discard the old state handle will leads to `FileNotFound` exception when restoring. (did not confirm the logic for changelog state backend now, could you please check this logic and update it?)
   
   Could you please create a ticket to track this problem please?
   
   > The previous checkpoint's state handles are discarded, although all of the checkpoints may succeed, the succeed checkpoints may be broken. So I think this is a bug.
   
   the lock guarded the checkpoint completion, so there would be no concurrent checkpoint completion. when registering the second checkpoint, checkpoint 1 is either successful or failed.
   In this situation, we need to mock a whole checkpoint and store the checkpoint metadata. (we need metadata when restoring  checkpoint) Discarding the statehandle is not a problem if we replaced the discarded state handle with the remaining state handle when storing the checkpoint metadata.
   
   the procedure of checkpoint completation for incremental checkpoint below is correct:
   1. checkpoint 1 and checkpoint 2 both are ongoing
   2. task-a in checkpoint 1 and checkpoint 2 both include the 3.sst
   3. checkpoint 1 complete and register successes with 3.sst(maybe `checkpoint-1-3.sst-remote-filename`  in hdfs)
   4. checkpoint 2 complete and register successes with 3.sst(maybe `checkpoint-2-3.sst-remote-filename` in hdfs)
   4.1 discarded 3.sst(`checkpoint-2-3.sst-remote-filename`) in checkpoint 2 in SharedStateRegistry (currently we discarded the `checkpoint-1-3.sst-remote-filename` statehandle)
   4.2 replace the state handle in checkpoint 2 with the handle in checkpoint 1 (`checkpoint-1-3.sst-remote-filename`) [here](https://github.com/apache/flink/blob/8e16cc8e424e352c5b45b46f1520ecf0edec70be/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java#L345)
   5. store the metadata in checkpoint 2 with `checkpoint-1-3.sst-remote-filename`)



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df50f2a707fe85d3f96a9b7cb0324092fcb2917e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "df50f2a707fe85d3f96a9b7cb0324092fcb2917e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * df50f2a707fe85d3f96a9b7cb0324092fcb2917e 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] klion26 commented on a diff in pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {

Review Comment:
   seems this block can cover the logic L110 - L115



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   IIUC, this can also happen when the number of concurrent checkpoint is bigger than 1, does this message needs to be info level(or it can be debug level)



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   > does this message needs to be info level(or it can be debug level)
   
   I think the probability of reaching this branch is low, so using info level doesn't bring much overhead.
   
   > If the checkpoint did not complete successfully, we can't reuse the sst file in it, and the status of the file in the remote file system is unknown (the generated file name is for the file in the remote file system)
   
   When the number of concurrent checkpoint is bigger than 1, is only the last triggered checkpoint completed?
   Suppose`chk-x` and `chk-(x+1)` are pending at the same time, if the state handles of `chk-x` are deleted by the registration of `chk-(x+1)`, `chk-x` is broken.  So, I think this is maybe a bug.
   
   And I open [PR21136](https://github.com/apache/flink/pull/21136) to change the log level, please help to review. :)
   
   



-- 
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] klion26 commented on a diff in pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   > Since every upload will give a [new fileName](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java#L442) for SST, is this(delete the old incompleted checkpoint's statehandle) a bug?
   
   I don't think so, If the checkpoint did not complete successfully, we can't reuse the sst file in it, and the status of the file in the remote file system is unknown (the generated file name is for the file in the remote file system).
   
   Another open question here is whether we need to change the registry's key in incremental checkpoint from the local file name([1][2]) to the remote file name.
   
   [1] https://github.com/apache/flink/blob/c84c9d71dfc577588d788241de230b9619c5b0ef/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java#L327
   [2] https://github.com/apache/flink/blob/c84c9d71dfc577588d788241de230b9619c5b0ef/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java#L397



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {

Review Comment:
   Partial agree.
   I think the original purpose was to distinguish between `entry.stateHandle == newHandle` and `entry.stateHandle.equals( newHandle)`, the log msg is different here, `same object` vs `new 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] klion26 commented on a diff in pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   > When the number of concurrent checkpoint is bigger than 1, is only the last triggered checkpoint completed?
   
   No, all of the checkpoints may succeed.
   
   > Suppose chk-x and chk-(x+1) are pending at the same time, if the state handles of chk-x are deleted by the registration of chk-(x+1), chk-x is broken. So, I think this is maybe a bug.
   
   IIUC, this is not a bug, but the logic here is a bit confusing. the later complete checkpoint will reuse the previous state handle, the logic is [here](https://github.com/apache/flink/blob/8e16cc8e424e352c5b45b46f1520ecf0edec70be/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java#L345), and you can try to add a ut to verify this logic locally.
   
   will review the new pr soon.



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {

Review Comment:
   Partially agree.
   I think the original purpose was to distinguish between `entry.stateHandle == newHandle` and `entry.stateHandle.equals( newHandle)`, the log msg is different here, `same object` vs `new 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] klion26 commented on a diff in pull request #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   IIUC, we only use placeholder state handle if the checkpoint contains sst that *has been confirmed*



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   Sorry for the late reply, I add a UT to verify the concurrent checkpoints logic. https://github.com/fredia/flink/tree/test_concurrent_chk
   > the later complete checkpoint will reuse the previous state handle, the logic is [here](https://github.com/apache/flink/blob/8e16cc8e424e352c5b45b46f1520ecf0edec70be/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java#L345)
   
   I found that the previous state handle is discarded when registering a new state handle with same key ([line 260](https://github.com/apache/flink/compare/master...fredia:flink:test_concurrent_chk#diff-80eee7531d8a093be7d2f4cb69dce47453e22e898662c930fb7557908efbebc4R260)), so the later complete checkpoint would use its **own** state handles not previous handles.  
   The previous checkpoint's state handles are discarded, although all of the checkpoints may succeed, the succeed checkpoints may be broken. So I think this is a bug. 



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   I added an ITCase at https://github.com/fredia/flink/tree/test_concurrent_chk to reproduce this scenario, and created https://issues.apache.org/jira/browse/FLINK-29913 to track this problem.



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   I don't think so, correct me if I am wrong.
   Under normal circumstances(not failover), the concurrent checkpoints may share one SST, But except one handle is SST, the others are placeholder (covered by L122), what do you think?



-- 
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 #21050: [FLINK-29095][state] Improve logging in SharedStateRegistry

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -95,60 +95,78 @@ public StreamStateHandle registerReference(
             entry = registeredStates.get(registrationKey);
 
             if (entry == null) {
-                // Additional check that should never fail, because only state handles that are not
-                // placeholders should
-                // ever be inserted to the registry.
                 checkState(
-                        !isPlaceholder(state),
+                        !isPlaceholder(newHandle),
                         "Attempt to reference unknown state: " + registrationKey);
 
-                entry = new SharedStateEntry(state, checkpointID);
+                LOG.trace(
+                        "Registered new shared state {} under key {}.", newHandle, registrationKey);
+                entry = new SharedStateEntry(newHandle, checkpointID);
                 registeredStates.put(registrationKey, entry);
-                LOG.trace("Registered new shared state {} under key {}.", entry, registrationKey);
 
-            } else {
-                // Delete if this is a real duplicate.
-                // Note that task (backend) is not required to re-upload state
-                // if the confirmation notification was missing.
-                // However, it's also not required to use exactly the same handle or placeholder
-                if (!Objects.equals(state, entry.stateHandle)) {
-                    if (entry.confirmed || isPlaceholder(state)) {
-                        scheduledStateDeletion = state;
-                    } else {
-                        // Old entry is not in a confirmed checkpoint yet, and the new one differs.
-                        // This might result from (omitted KG range here for simplicity):
-                        // 1. Flink recovers from a failure using a checkpoint 1
-                        // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
-                        // 3. JM triggers checkpoint 2
-                        // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
-                        // 5. TM crashes; everything is repeated from (2)
-                        // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
-                        // 7. JM triggers checkpoint 3
-                        // 8. TM sends NEW state "xyz-002.sst"
-                        // 9. JM discards it as duplicate
-                        // 10. checkpoint completes, but a wrong SST file is used
-                        // So we use a new entry and discard the old one:
-                        scheduledStateDeletion = entry.stateHandle;
-                        entry.stateHandle = state;
-                    }
-                    LOG.trace(
-                            "Identified duplicate state registration under key {}. New state {} was determined to "
-                                    + "be an unnecessary copy of existing state {} and will be dropped.",
-                            registrationKey,
-                            state,
-                            entry.stateHandle);
-                }
+                // no further handling
+                return entry.stateHandle;
+
+            } else if (entry.stateHandle == newHandle) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the same object: {}",
+                        registrationKey,
+                        newHandle);
+            } else if (Objects.equals(entry.stateHandle, newHandle)) {
+                // might be a bug but state backend is not required to use a place-holder
+                LOG.debug(
+                        "Duplicated registration under key {} with the new object: {}.",
+                        registrationKey,
+                        newHandle);
+            } else if (isPlaceholder(newHandle)) {
                 LOG.trace(
-                        "Updating last checkpoint for {} from {} to {}",
+                        "Duplicated registration under key {} with a placeholder (normal case)",
+                        registrationKey);
+                scheduledStateDeletion = newHandle;
+            } else if (entry.confirmed) {
+                LOG.info(
+                        "Duplicated registration under key {} of a new state: {}. "
+                                + "This might happen if checkpoint confirmation was delayed and state backend re-uploaded the state. "
+                                + "Discarding the new state and keeping the old one which is included into a completed checkpoint",
                         registrationKey,
-                        entry.lastUsedCheckpointID,
-                        checkpointID);
-                entry.advanceLastUsingCheckpointID(checkpointID);
-                if (preventDiscardingCreatedCheckpoint) {
-                    entry.preventDiscardingCreatedCheckpoint();
-                }
+                        newHandle);
+                scheduledStateDeletion = newHandle;
+            } else {
+                // Old entry is not in a confirmed checkpoint yet, and the new one differs.
+                // This might result from (omitted KG range here for simplicity):
+                // 1. Flink recovers from a failure using a checkpoint 1
+                // 2. State Backend is initialized to UID xyz and a set of SST: { 01.sst }
+                // 3. JM triggers checkpoint 2
+                // 4. TM sends handle: "xyz-002.sst"; JM registers it under "xyz-002.sst"
+                // 5. TM crashes; everything is repeated from (2)
+                // 6. TM recovers from CP 1 again: backend UID "xyz", SST { 01.sst }
+                // 7. JM triggers checkpoint 3
+                // 8. TM sends NEW state "xyz-002.sst"
+                // 9. JM discards it as duplicate
+                // 10. checkpoint completes, but a wrong SST file is used
+                // So we use a new entry and discard the old one:
+                LOG.info(

Review Comment:
   Aha, I missed this point, thanks for your clarification. 
   
   Since every upload will give a [new fileName](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java#L442) for SST,  is this(delete the old incompleted checkpoint's statehandle) a bug?
   
   And I accidentally deleted the branch of this PR, I will create a new one to change the log level.
    



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