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 2020/10/10 20:01:28 UTC

[GitHub] [flink] kl0u opened a new pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

kl0u opened a new pull request #13583:
URL: https://github.com/apache/flink/pull/13583


   ## What is the purpose of the change
   
   Currently in Application Mode when we shutdown the cluster we always clean up the HA data of a job. This also includes the cases when the job fails due to a transient error. This behaviour leads to the job not being able to restart even if HA is activated. This PR fixes the issue by only cleaning up the HA data when the job has explicitly reported reaching a terminal state (FAILED, SUCCEEDED, or CANCELLED), and not as a reaction to framework failures or other issues.
   
   ## Brief change log
   
   The main change is in the `ApplicationDispatcherBootstrap.runApplicationAndShutdownClusterAsync()`. This is where the shutting down of the cluster is happening.
   
   ## Verifying this change
   
   This change is already covered by existing tests, such as the tests in the `ApplicationDispatcherBootstrapTest` that were modified.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
   


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

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



[GitHub] [flink] kl0u commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java
##########
@@ -92,6 +91,7 @@ public ApplicationDispatcherGatewayServiceFactory(
 			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
 		}
 
+		bootstrap.setErrorHandler(exception -> dispatcher.getShutDownFuture().completeExceptionally(exception));

Review comment:
       Terminating the dispatcher process will eventually bring down the cluster, right @tillrohrmann ?




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

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



[GitHub] [flink] flinkbot edited a comment on pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386",
       "triggerID" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7450",
       "triggerID" : "ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7450) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386",
       "triggerID" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fe1d5b4deb0c6b94ce8a7c564e98caec5217339 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java
##########
@@ -141,25 +141,29 @@ public void stop() {
 					final ApplicationStatus applicationStatus;
 					if (t != null) {
 
-						final Optional<JobCancellationException> cancellationException =
-								ExceptionUtils.findThrowable(t, JobCancellationException.class);
-
-						if (cancellationException.isPresent()) {
+						if (ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) {
 							// this means the Flink Job was cancelled
 							applicationStatus = ApplicationStatus.CANCELED;
-						} else if (t instanceof CancellationException) {
-							// this means that the future was cancelled
-							applicationStatus = ApplicationStatus.UNKNOWN;
-						} else {
+							LOG.warn("Application {}: ", applicationStatus, t);

Review comment:
       Warn log level seems to high for a normal state transition.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java
##########
@@ -82,4 +82,8 @@
 	default CompletableFuture<Acknowledge> shutDownCluster(ApplicationStatus applicationStatus) {
 		return shutDownCluster();
 	}
+
+	default CompletableFuture<Acknowledge> shutDownClusterExceptionally(final Throwable throwable) {
+		throw new UnsupportedOperationException();
+	}

Review comment:
       I think we should not introduce this method. Instead the `ApplicationDispatcherBootstrap` should get a `FatalErrorHandler` which it simply calls if it sees an unknown exception.

##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java
##########
@@ -141,25 +141,29 @@ public void stop() {
 					final ApplicationStatus applicationStatus;
 					if (t != null) {
 
-						final Optional<JobCancellationException> cancellationException =
-								ExceptionUtils.findThrowable(t, JobCancellationException.class);
-
-						if (cancellationException.isPresent()) {
+						if (ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) {
 							// this means the Flink Job was cancelled
 							applicationStatus = ApplicationStatus.CANCELED;
-						} else if (t instanceof CancellationException) {
-							// this means that the future was cancelled
-							applicationStatus = ApplicationStatus.UNKNOWN;
-						} else {
+							LOG.warn("Application {}: ", applicationStatus, t);
+
+							return dispatcher.shutDownCluster(applicationStatus);
+						}
+
+						if (ExceptionUtils.findThrowable(t, JobExecutionException.class).isPresent()) {

Review comment:
       Are you sure that a `JobExecutionException` means that the Flink job has reached a globally terminal state? Looking at `JobResult.toJobExecutionResult` it seems that we also throw this exception if the state is `ApplicationStatus.UNKNOWN`.

##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java
##########
@@ -141,25 +141,29 @@ public void stop() {
 					final ApplicationStatus applicationStatus;
 					if (t != null) {
 
-						final Optional<JobCancellationException> cancellationException =
-								ExceptionUtils.findThrowable(t, JobCancellationException.class);
-
-						if (cancellationException.isPresent()) {
+						if (ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) {
 							// this means the Flink Job was cancelled
 							applicationStatus = ApplicationStatus.CANCELED;
-						} else if (t instanceof CancellationException) {
-							// this means that the future was cancelled
-							applicationStatus = ApplicationStatus.UNKNOWN;
-						} else {
+							LOG.warn("Application {}: ", applicationStatus, t);
+
+							return dispatcher.shutDownCluster(applicationStatus);
+						}
+
+						if (ExceptionUtils.findThrowable(t, JobExecutionException.class).isPresent()) {
 							applicationStatus = ApplicationStatus.FAILED;
+							LOG.warn("Application {}: ", applicationStatus, t);

Review comment:
       Same here.




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

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java
##########
@@ -92,6 +91,7 @@ public ApplicationDispatcherGatewayServiceFactory(
 			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
 		}
 
+		bootstrap.setErrorHandler(exception -> dispatcher.getShutDownFuture().completeExceptionally(exception));

Review comment:
       I don't understand the chicken-egg problem here. Why don't we pass in a bootstrap factory which creates the `ApplicationDispatcherBootstrap` with the appropriate fatal error handler when `Dispatcher.onStart` is being called?




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

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



[GitHub] [flink] kl0u commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java
##########
@@ -92,6 +91,7 @@ public ApplicationDispatcherGatewayServiceFactory(
 			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
 		}
 
+		bootstrap.setErrorHandler(exception -> dispatcher.getShutDownFuture().completeExceptionally(exception));

Review comment:
       Yes @aljoscha. The solution I gave was to set the error handler using a setter, but it is not elegant as a construct. It is fragile, as there must be a default handler that does nothing, and the user has to keep in mind to set it when needed. 
   
   @tillrohrmann Errors that lead to job execution termination are fatal and continuing to work does not really make sense. The semantics are that the cluster is created for the application only, so if any job in the application fails irrecoverably (even with status UNKNOWN), it should kill also the cluster (potentially without deleting HA data).




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

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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java
##########
@@ -92,6 +91,7 @@ public ApplicationDispatcherGatewayServiceFactory(
 			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
 		}
 
+		bootstrap.setErrorHandler(exception -> dispatcher.getShutDownFuture().completeExceptionally(exception));

Review comment:
       We could also instantiate a separate error handler, right? Is the assumption that any error which `bootstrap` receives is a fatal exception and continuing the work does not make sense? If yes, then we could simply use a `FatalErrorHandler` which terminates the process.

##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java
##########
@@ -144,19 +151,21 @@ public void stop() {
 						if (ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) {
 							// this means the Flink Job was cancelled
 							applicationStatus = ApplicationStatus.CANCELED;
-							LOG.warn("Application {}: ", applicationStatus, t);
+							LOG.info("Application {}: ", applicationStatus, t);
 
 							return dispatcher.shutDownCluster(applicationStatus);
 						}
 
 						if (ExceptionUtils.findThrowable(t, JobExecutionException.class).isPresent()) {
 							applicationStatus = ApplicationStatus.FAILED;
-							LOG.warn("Application {}: ", applicationStatus, t);
+							LOG.info("Application {}: ", applicationStatus, t);
 
 							return dispatcher.shutDownCluster(applicationStatus);
 						}
 
-						return dispatcher.shutDownClusterExceptionally(t);
+						LOG.warn("Exiting with Application Status UNKNOWN: ", t);
+						errorHandler.onFatalError(t);
+						return CompletableFuture.completedFuture(Acknowledge.get());

Review comment:
       Why do we finish the handler with `Acknowledge` here?




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

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



[GitHub] [flink] aljoscha commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java
##########
@@ -92,6 +91,7 @@ public ApplicationDispatcherGatewayServiceFactory(
 			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
 		}
 
+		bootstrap.setErrorHandler(exception -> dispatcher.getShutDownFuture().completeExceptionally(exception));

Review comment:
       Yes, I think a factory is the right approach. 👌




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

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



[GitHub] [flink] flinkbot edited a comment on pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386",
       "triggerID" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7450",
       "triggerID" : "ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fe1d5b4deb0c6b94ce8a7c564e98caec5217339 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386) 
   * ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7450) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot commented on pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fe1d5b4deb0c6b94ce8a7c564e98caec5217339 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] kl0u commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java
##########
@@ -92,6 +91,7 @@ public ApplicationDispatcherGatewayServiceFactory(
 			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
 		}
 
+		bootstrap.setErrorHandler(exception -> dispatcher.getShutDownFuture().completeExceptionally(exception));

Review comment:
       I agree with @aljoscha. The solution I gave was to set the error handler using a setter, but it is not elegant as a construct because there must be a default handler that does nothing, and the user has to keep in mind to set it when needed. Errors that lead to job execution termination are fatal and continuing to work does not really make sense. The semantics are that the cluster is created for the application only, so if any job in the application fails irrecoverably, it should kill also the cluster (potentially without deleting HA data).




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

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



[GitHub] [flink] kl0u closed pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

Posted by GitBox <gi...@apache.org>.
kl0u closed pull request #13583:
URL: https://github.com/apache/flink/pull/13583


   


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

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



[GitHub] [flink] flinkbot commented on pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 5fe1d5b4deb0c6b94ce8a7c564e98caec5217339 (Sat Oct 10 20:04:18 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386",
       "triggerID" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fe1d5b4deb0c6b94ce8a7c564e98caec5217339 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] flinkbot edited a comment on pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386",
       "triggerID" : "5fe1d5b4deb0c6b94ce8a7c564e98caec5217339",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5fe1d5b4deb0c6b94ce8a7c564e98caec5217339 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7386) 
   * ce67786d1c3816e6ef0c0aa2dd5e758e239c5a2d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.

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



[GitHub] [flink] aljoscha commented on a change in pull request #13583: [FLINK-19154] ApplicationDispatcherBootstrap clean up HA data only when necessary.

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



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java
##########
@@ -92,6 +91,7 @@ public ApplicationDispatcherGatewayServiceFactory(
 			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
 		}
 
+		bootstrap.setErrorHandler(exception -> dispatcher.getShutDownFuture().completeExceptionally(exception));

Review comment:
       You can't pass it in the constructor because of the chicken-and-egg problem, right? You need the bootstrap to create the dispatcher but you need the dispatcher to create the `FatalErrorHandler`.




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

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