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/03/15 15:37:41 UTC

[GitHub] [flink] XComp opened a new pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

XComp opened a new pull request #19102:
URL: https://github.com/apache/flink/pull/19102


   ## What is the purpose of the change
   
   Flink 1.14- would just print warning in case of failures. The initial 1.15 retrable cleanup functionality failed the cluster fatally if the cleanup failed. This can happen if the user decides to limit the number of attempts (by default, Flink will try to cleanup infinitely). This would cause issues in session mode: Other jobs stop as well with the whole Flink cluster failing fatally. That's not what we want.
   
   In contrast, for job and application mode: Failing fatally would be an option to notify the user about the failure. Flink is left in an inconsistent state. In HA mode, this would lead to a restart of the worker and the cleanup is picked up again. That would result in the cleanup logic being triggered again which might not what the user intended when limiting the cleanup retries.
   
   Therefore, logging a warning in case of the job and application mode is the more reasonable thing to do.
   
   ## Brief change log
   
   * Added log message to Dispatcher after cleanup
   * Updated default value for `fixed-delay` to also try infinitely to force the user to explicitly change the configuration parameter if he/she really wants to leave jobs in an inconsistent state
   * Updated the documentation of the configuration parameters and the retryable cleanup accordingly
   
   ## Verifying this change
   
   * Adapted `DispatcherCleanupITCase` - the test would now fail if the cluster fails fatally due to the `TestingFatalErrorHandlerResource` before verified after the test succeeded.
   
   ## 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: yes
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? docs
   


-- 
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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068436525


   I addressed your comments, updated the docs, fixed the test flakiness, squashed the commits and rebased the branch. PTAL


-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 40620b64562e40f46db423c3e496b85fda4e4a9f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147) 
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7167336940703396283eec813bab3e0696095672 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121) 
   * 485b8a4ff2d1369b028fbab37281f40a49869bf0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125) 
   
   <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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827939888



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java
##########
@@ -128,19 +134,23 @@ public static String extractAlphaNumericCharacters(String paramName) {
     public static final ConfigOption<Integer> CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS =
             ConfigOptions.key(createFixedDelayParameterPrefix("attempts"))
                     .intType()
-                    .defaultValue(1)
+                    .noDefaultValue()

Review comment:
       uh, thanks for the pointer. I didn't know about it. I will use that and close FLINK-26350 👍 




-- 
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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827877458



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java
##########
@@ -128,19 +134,23 @@ public static String extractAlphaNumericCharacters(String paramName) {
     public static final ConfigOption<Integer> CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS =
             ConfigOptions.key(createFixedDelayParameterPrefix("attempts"))
                     .intType()
-                    .defaultValue(1)
+                    .noDefaultValue()

Review comment:
       Why not set this to Integer.MAX_VALUE?




-- 
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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827129983



##########
File path: docs/content.zh/docs/deployment/overview.md
##########
@@ -158,6 +158,11 @@ Once a job has reached a globally terminal state of either finished, failed or c
 external component resources associated with the job are then cleaned up. In the event of a
 failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can
 [configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used.
+Reaching the maximum number of retries without succeeding will leave the job in a dirty state.
+Its artifacts (and its [JobResultStore]({{< ref "docs/deployment/ha/overview#jobresultstore" >}})
+entry like in Application Mode) would need to be cleaned up manually. Restarting the very same

Review comment:
       ```suggestion
   entry) would need to be cleaned up manually. Restarting the very same
   ```
   Reads a bit weird, and I'm not sure why application mode should be explicitly mentioned.

##########
File path: docs/content/docs/deployment/overview.md
##########
@@ -158,7 +158,12 @@ When deploying Flink, there are often multiple options available for each buildi
 Once a job has reached a globally terminal state of either finished, failed or cancelled, the
 external component resources associated with the job are then cleaned up. In the event of a
 failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can
-[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used.
+[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. 
+Reaching the maximum number of retries without succeeding will leave the job in a dirty state. 

Review comment:
       Do we have any documentation that explains the consequences of being in a dirty state?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -615,7 +615,16 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy
 
         final CompletableFuture<Void> jobTerminationFuture =
                 cleanupJobStateFuture.thenCompose(
-                        cleanupJobState -> removeJob(jobId, cleanupJobState));
+                        cleanupJobState ->
+                                removeJob(jobId, cleanupJobState)
+                                        .exceptionally(
+                                                throwable -> {
+                                                    log.warn(
+                                                            "The cleanup of job {} failed. The job's artifacts and its JobResultStore entry needs to be cleaned manually.",
+                                                            jobId,

Review comment:
       would it be possible to list the paths that need to be cleaned up?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -288,7 +304,7 @@ public void testCleanupAfterLeadershipChange() throws Exception {
         dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
         waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId);
-        jobGraphRemovalErrorReceived.await();
+        firstCleanupFailsLatch.trigger();

Review comment:
       I don't see where we guarantee that the cleanup isn't re-run immediately and finishes before the leadership has changed.

##########
File path: docs/content/docs/deployment/overview.md
##########
@@ -158,7 +158,12 @@ When deploying Flink, there are often multiple options available for each buildi
 Once a job has reached a globally terminal state of either finished, failed or cancelled, the
 external component resources associated with the job are then cleaned up. In the event of a
 failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can
-[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used.
+[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. 
+Reaching the maximum number of retries without succeeding will leave the job in a dirty state. 
+Its artifacts (and its [JobResultStore]({{< ref "docs/deployment/ha/overview#jobresultstore" >}}) 
+entry like in Application Mode) would need to be cleaned up manually. Restarting the very same 
+job (i.e. using the same job ID) would result in the retryable cleanup being picked up again 

Review comment:
       ```suggestion
   job (i.e. using the same job ID) will result in the cleanup being restarted 
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -249,11 +259,30 @@ public void testCleanupAfterLeadershipChange() throws Exception {
 
         // Construct job graph store.
         final AtomicInteger actualGlobalCleanupCallCount = new AtomicInteger();
+        final OneShotLatch firstCleanupFailsLatch = new OneShotLatch();
         final OneShotLatch successfulCleanupLatch = new OneShotLatch();
-        final RuntimeException temporaryError = new RuntimeException("Unable to remove job graph.");
         final JobGraphStore jobGraphStore =
-                createAndStartJobGraphStoreWithCleanupFailures(
-                        1, temporaryError, actualGlobalCleanupCallCount, successfulCleanupLatch);
+                TestingJobGraphStore.newBuilder()
+                        .setGlobalCleanupFunction(
+                                (ignoredJobId, ignoredExecutor) -> {
+                                    try {
+                                        firstCleanupFailsLatch.await();
+                                    } catch (InterruptedException e) {
+                                        throw new CompletionException(e);
+                                    }
+
+                                    if (actualGlobalCleanupCallCount.getAndIncrement() < 1) {
+                                        return FutureUtils.completedExceptionally(
+                                                new RuntimeException(
+                                                        "Expected RuntimeException: Unable to remove job graph."));
+                                    }
+
+                                    successfulCleanupLatch.trigger();
+                                    return FutureUtils.completedVoidFuture();
+                                })
+                        .build();
+
+        jobGraphStore.start(null);

Review comment:
       see above

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -133,12 +132,23 @@ public void testCleanupThroughRetries() throws Exception {
         final int numberOfErrors = 5;
         final RuntimeException temporaryError =
                 new RuntimeException("Expected RuntimeException: Unable to remove job graph.");
+        final AtomicInteger failureCount = new AtomicInteger(numberOfErrors);
         final JobGraphStore jobGraphStore =
-                createAndStartJobGraphStoreWithCleanupFailures(
-                        numberOfErrors,
-                        temporaryError,
-                        actualGlobalCleanupCallCount,
-                        successfulCleanupLatch);
+                TestingJobGraphStore.newBuilder()
+                        .setGlobalCleanupFunction(
+                                (ignoredJobId, ignoredExecutor) -> {
+                                    actualGlobalCleanupCallCount.incrementAndGet();
+
+                                    if (failureCount.getAndDecrement() > 0) {
+                                        return FutureUtils.completedExceptionally(temporaryError);
+                                    }
+
+                                    successfulCleanupLatch.trigger();
+                                    return FutureUtils.completedVoidFuture();
+                                })
+                        .build();
+
+        jobGraphStore.start(null);

Review comment:
       ```suggestion
           jobGraphStore.start(NoOpJobGraphListener.INSTANCE);
   ```




-- 
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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068824958


   I did another force-push after realizing that there were some minor doc issues (double period at the end of the sentence). Rebase performed as well


-- 
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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827883559



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -288,7 +304,7 @@ public void testCleanupAfterLeadershipChange() throws Exception {
         dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
         waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId);
-        jobGraphRemovalErrorReceived.await();
+        firstCleanupFailsLatch.trigger();

Review comment:
       > The Dispatcher is configured to not retry
   
   ~~Where is that done?~~




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   * 0667dc7840d513e3abc16a7e095c3987e2304007 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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   * a969297a41d6ebf85b2a71d19051ce9132f5c46c 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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   
   <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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068799867


   Ok, looks like my browser cache wasn't freed properly. Now, the successful run shows up


-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33188",
       "triggerID" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   * a969297a41d6ebf85b2a71d19051ce9132f5c46c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33188) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7167336940703396283eec813bab3e0696095672 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121) 
   * 485b8a4ff2d1369b028fbab37281f40a49869bf0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125) 
   * df9034e8981222f4cfa7c9c106b14ff3cea81a2a 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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1069056505


   I force-pushed the minor docs change. The test failure of the previous ci run is documented in FLINK-26684


-- 
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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827967866



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java
##########
@@ -79,8 +79,7 @@ private static ExponentialBackoffRetryStrategy createExponentialBackoffRetryStra
                 configuration.get(CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF);
         final int maxAttempts =
                 configuration.getInteger(

Review comment:
       could use #get().




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 40620b64562e40f46db423c3e496b85fda4e4a9f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * df9034e8981222f4cfa7c9c106b14ff3cea81a2a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127) 
   
   <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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827965761



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java
##########
@@ -65,7 +65,7 @@ private static RetryStrategy createNoRetryStrategy() {
     private static FixedRetryStrategy createFixedRetryStrategy(Configuration configuration) {
         final Duration delay = configuration.get(CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_DELAY);
         final int maxAttempts =
-                configuration.get(CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS);
+                configuration.getInteger(CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS);

Review comment:
       seems unnecessary




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * df9034e8981222f4cfa7c9c106b14ff3cea81a2a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127) 
   * 40620b64562e40f46db423c3e496b85fda4e4a9f 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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827938035



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java
##########
@@ -128,19 +134,23 @@ public static String extractAlphaNumericCharacters(String paramName) {
     public static final ConfigOption<Integer> CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS =
             ConfigOptions.key(createFixedDelayParameterPrefix("attempts"))
                     .intType()
-                    .defaultValue(1)
+                    .noDefaultValue()

Review comment:
       Or you could just use `org.apache.flink.annotation.docs.Documentation.OverrideDefault` 🤷 




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   * a969297a41d6ebf85b2a71d19051ce9132f5c46c 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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827919525



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java
##########
@@ -128,19 +134,23 @@ public static String extractAlphaNumericCharacters(String paramName) {
     public static final ConfigOption<Integer> CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS =
             ConfigOptions.key(createFixedDelayParameterPrefix("attempts"))
                     .intType()
-                    .defaultValue(1)
+                    .noDefaultValue()

Review comment:
       because it would then print this a weirdly big number and I thought that it's better to have an explicit no value instead.




-- 
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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827345773



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -288,7 +304,7 @@ public void testCleanupAfterLeadershipChange() throws Exception {
         dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
         waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId);
-        jobGraphRemovalErrorReceived.await();
+        firstCleanupFailsLatch.trigger();

Review comment:
       btw. this trigger call was the reason for the flakyness of the test. 🤦 I swapped trigger and await again. We want to wait here for the cleanup and not trigger it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -288,7 +304,7 @@ public void testCleanupAfterLeadershipChange() throws Exception {
         dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
         waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId);
-        jobGraphRemovalErrorReceived.await();
+        firstCleanupFailsLatch.trigger();

Review comment:
       btw. this trigger call was the reason for the flakiness of the test. 🤦 I swapped trigger and await again. We want to wait here for the cleanup and not trigger it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



[GitHub] [flink] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 485b8a4ff2d1369b028fbab37281f40a49869bf0 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125) 
   * df9034e8981222f4cfa7c9c106b14ff3cea81a2a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127) 
   
   <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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1069081537


   I override the values now. That required some changes in the `CleanupRetryStrategyFactory` and `CleanupRetryStrategyFactoryTest`. PTAL


-- 
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] XComp merged pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp merged pull request #19102:
URL: https://github.com/apache/flink/pull/19102


   


-- 
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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827883559



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -288,7 +304,7 @@ public void testCleanupAfterLeadershipChange() throws Exception {
         dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
         waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId);
-        jobGraphRemovalErrorReceived.await();
+        firstCleanupFailsLatch.trigger();

Review comment:
       > The Dispatcher is configured to not retry
   
   Where is that done?




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33188",
       "triggerID" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   * a969297a41d6ebf85b2a71d19051ce9132f5c46c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33188) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 40620b64562e40f46db423c3e496b85fda4e4a9f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147) 
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33188",
       "triggerID" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a969297a41d6ebf85b2a71d19051ce9132f5c46c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33188) 
   
   <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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827345225



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -288,7 +304,7 @@ public void testCleanupAfterLeadershipChange() throws Exception {
         dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
         waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId);
-        jobGraphRemovalErrorReceived.await();
+        firstCleanupFailsLatch.trigger();

Review comment:
       The Dispatcher is configured to not retry. But I also added another assert that checks that cleanup is only called once after the leadership is lost. 👍 




-- 
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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827349921



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -615,7 +615,16 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy
 
         final CompletableFuture<Void> jobTerminationFuture =
                 cleanupJobStateFuture.thenCompose(
-                        cleanupJobState -> removeJob(jobId, cleanupJobState));
+                        cleanupJobState ->
+                                removeJob(jobId, cleanupJobState)
+                                        .exceptionally(
+                                                throwable -> {
+                                                    log.warn(
+                                                            "The cleanup of job {} failed. The job's artifacts and its JobResultStore entry needs to be cleaned manually.",
+                                                            jobId,

Review comment:
       I added the values of the configuration parameters. It feels a bit odd to access these configuration parameters here. But it's worth it to have a better log message, I guess




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * df9034e8981222f4cfa7c9c106b14ff3cea81a2a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127) 
   * 40620b64562e40f46db423c3e496b85fda4e4a9f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 40620b64562e40f46db423c3e496b85fda4e4a9f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147) 
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   * 0667dc7840d513e3abc16a7e095c3987e2304007 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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827335461



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -133,12 +132,23 @@ public void testCleanupThroughRetries() throws Exception {
         final int numberOfErrors = 5;
         final RuntimeException temporaryError =
                 new RuntimeException("Expected RuntimeException: Unable to remove job graph.");
+        final AtomicInteger failureCount = new AtomicInteger(numberOfErrors);
         final JobGraphStore jobGraphStore =
-                createAndStartJobGraphStoreWithCleanupFailures(
-                        numberOfErrors,
-                        temporaryError,
-                        actualGlobalCleanupCallCount,
-                        successfulCleanupLatch);
+                TestingJobGraphStore.newBuilder()
+                        .setGlobalCleanupFunction(
+                                (ignoredJobId, ignoredExecutor) -> {
+                                    actualGlobalCleanupCallCount.incrementAndGet();
+
+                                    if (failureCount.getAndDecrement() > 0) {
+                                        return FutureUtils.completedExceptionally(temporaryError);
+                                    }
+
+                                    successfulCleanupLatch.trigger();
+                                    return FutureUtils.completedVoidFuture();
+                                })
+                        .build();
+
+        jobGraphStore.start(null);

Review comment:
       It's not really necessary because the consumer of the start method is a noop in `TestingJobGraphStore`. But it doesn't hurt, I guess.




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7167336940703396283eec813bab3e0696095672 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7167336940703396283eec813bab3e0696095672 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121) 
   * 485b8a4ff2d1369b028fbab37281f40a49869bf0 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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827873510



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java
##########
@@ -133,12 +132,23 @@ public void testCleanupThroughRetries() throws Exception {
         final int numberOfErrors = 5;
         final RuntimeException temporaryError =
                 new RuntimeException("Expected RuntimeException: Unable to remove job graph.");
+        final AtomicInteger failureCount = new AtomicInteger(numberOfErrors);
         final JobGraphStore jobGraphStore =
-                createAndStartJobGraphStoreWithCleanupFailures(
-                        numberOfErrors,
-                        temporaryError,
-                        actualGlobalCleanupCallCount,
-                        successfulCleanupLatch);
+                TestingJobGraphStore.newBuilder()
+                        .setGlobalCleanupFunction(
+                                (ignoredJobId, ignoredExecutor) -> {
+                                    actualGlobalCleanupCallCount.incrementAndGet();
+
+                                    if (failureCount.getAndDecrement() > 0) {
+                                        return FutureUtils.completedExceptionally(temporaryError);
+                                    }
+
+                                    successfulCleanupLatch.trigger();
+                                    return FutureUtils.completedVoidFuture();
+                                })
+                        .build();
+
+        jobGraphStore.start(null);

Review comment:
       According to the interface the listener must not be null, and the only implementation making use of it relies on that fact.




-- 
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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827922757



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java
##########
@@ -128,19 +134,23 @@ public static String extractAlphaNumericCharacters(String paramName) {
     public static final ConfigOption<Integer> CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS =
             ConfigOptions.key(createFixedDelayParameterPrefix("attempts"))
                     .intType()
-                    .defaultValue(1)
+                    .noDefaultValue()

Review comment:
       I created FLINK-26350 to cover that issue




-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a969297a41d6ebf85b2a71d19051ce9132f5c46c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   * a969297a41d6ebf85b2a71d19051ce9132f5c46c 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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1069136829


   Thanks for your review, @zentol . I addressed your comments and rebased the branch. I'm gonna go ahead and create a 1.15 backport


-- 
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 #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7167336940703396283eec813bab3e0696095672 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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * df9034e8981222f4cfa7c9c106b14ff3cea81a2a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * df9034e8981222f4cfa7c9c106b14ff3cea81a2a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127) 
   
   <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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068796110


   I fixed the docs issue in the force push. Don't know why ci didn't pick it up 🤔 


-- 
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] XComp commented on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068795921


   @flinkbot run azure


-- 
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 edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7167336940703396283eec813bab3e0696095672 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121) 
   
   <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] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0667dc7840d513e3abc16a7e095c3987e2304007 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33181) 
   
   <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] XComp commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827372013



##########
File path: docs/content/docs/deployment/overview.md
##########
@@ -158,7 +158,12 @@ When deploying Flink, there are often multiple options available for each buildi
 Once a job has reached a globally terminal state of either finished, failed or cancelled, the
 external component resources associated with the job are then cleaned up. In the event of a
 failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can
-[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used.
+[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. 
+Reaching the maximum number of retries without succeeding will leave the job in a dirty state. 

Review comment:
       I updated the documentation under `Deployment / High Availability / Overview / JobResultStore` to also cover dirty entries with references to the folders.




-- 
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] zentol commented on a change in pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #19102:
URL: https://github.com/apache/flink/pull/19102#discussion_r827874011



##########
File path: docs/content.zh/docs/deployment/overview.md
##########
@@ -158,6 +158,11 @@ Once a job has reached a globally terminal state of either finished, failed or c
 external component resources associated with the job are then cleaned up. In the event of a
 failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can
 [configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used.
+Reaching the maximum number of retries without succeeding will leave the job in a dirty state.
+Its artifacts would need to be cleaned up manually (see the
+[High Availability Services / JobResultStore]({{< ref "docs/deployment/ha/overview#jobresultstore" >}})
+section for further details). Restarting the very same job (i.e. using the same
+job ID) will result in the cleanup being restarted again without running the job again.

Review comment:
       ```suggestion
   job ID) will result in the cleanup being restarted without running the job again.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



[GitHub] [flink] flinkbot edited a comment on pull request #19102: [FLINK-26652][runtime] Makes Flink cluster not fail in case of the cleanup retries being exhausted

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #19102:
URL: https://github.com/apache/flink/pull/19102#issuecomment-1068161025


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7167336940703396283eec813bab3e0696095672",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33121",
       "triggerID" : "7167336940703396283eec813bab3e0696095672",
       "triggerType" : "PUSH"
     }, {
       "hash" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33125",
       "triggerID" : "485b8a4ff2d1369b028fbab37281f40a49869bf0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df9034e8981222f4cfa7c9c106b14ff3cea81a2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33127",
       "triggerID" : "1068795921",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33147",
       "triggerID" : "40620b64562e40f46db423c3e496b85fda4e4a9f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176",
       "triggerID" : "f379acffb6ce7b1d71c1c1ca21538593043646c5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0667dc7840d513e3abc16a7e095c3987e2304007",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f379acffb6ce7b1d71c1c1ca21538593043646c5 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33176) 
   * 0667dc7840d513e3abc16a7e095c3987e2304007 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