Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-24021][HA]Handle curator framework start error by register UnhandledErrorListener before start zk client #17053

Merged
merged 1 commit into from Sep 3, 2021

Conversation

Aitozi
Copy link
Contributor

@Aitozi Aitozi commented Aug 30, 2021

What is the purpose of the change

As described in issue , this PR is meant to fix the potential job unrecoverable due to network failure . This is done by add an unhandledErrorListener before start curator client

Tests:
ZookeeperUtilsTest#testStartCuratorFrameworkFailed

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)

@flinkbot
Copy link
Collaborator

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 3e938a7 (Mon Aug 30 13:18:03 UTC 2021)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

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 for a full explanation of the review process.


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

@flinkbot
Copy link
Collaborator

flinkbot commented Aug 30, 2021

CI report:

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

@Aitozi Aitozi changed the title [FLINK-36488942][HA]Handle curator framework start error by register fatalErrorHandler befor start zk client [FLINK-36488942][HA]Handle curator framework start error by register fatalErrorHandler before start zk client Aug 30, 2021
@Aitozi Aitozi changed the title [FLINK-36488942][HA]Handle curator framework start error by register fatalErrorHandler before start zk client [FLINK-24021][HA]Handle curator framework start error by register UnhandledErrorHandler before start zk client Aug 30, 2021
@Aitozi Aitozi changed the title [FLINK-24021][HA]Handle curator framework start error by register UnhandledErrorHandler before start zk client [FLINK-24021][HA]Handle curator framework start error by register UnhandledErrorListener before start zk client Aug 31, 2021
Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for creating this PR @Aitozi. I think you are right that we don't handle the case properly where the CuratorZookeeperClient.start fails. In order to do that we have to register an UnhandledErrorListener right away.

I left a couple of comments. Please take a look.

Comment on lines 233 to 258
// This handler is only used to handle the error during start phase, and should be
// removed after start curator success.
UnhandledErrorListener unhandledErrorListener =
(message, throwable) -> {
LOG.error(
"Exiting process for unhandled error in start curator framework, "
+ "error message: {}, exiting code: {}",
message,
ZOOKEEPER_FAILURE_EXIT_CODE,
throwable);
System.exit(ZOOKEEPER_FAILURE_EXIT_CODE);
};
cf.getUnhandledErrorListenable().addListener(unhandledErrorListener);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am actually wondering whether we shouldn't only register this one handler and remove the listeners in the ZooKeeperLeaderElectionDriver and ZooKeeperLeaderRetrievalDriver because these two listeners would be called for every unhandled exception that occurs in Curator independent of the actual source. Moreover, the current UnhandledErrorListener always call the FatalExitExceptionHandler eventually.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't really get your meaning, may be it's should only ?

I think we could safely remove the listener in leaderElectionDriver and leaderRetrievalDriver like you said that,all the three handlers always exit the process now, after one execute other will not be executed. So there is no meaning to register other handlers.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I meant to only register this one failure handler and remove the listener in leaderElectionDriver and leaderRetrievalDriver.

@Aitozi
Copy link
Contributor Author

Aitozi commented Sep 1, 2021

Thanks for your review @tillrohrmann . I have addressed most of your comments. Regarding to whether to remove the listener in ZooKeeperLeaderElectionDriver and ZooKeeperLeaderRetrievalDriver, I think it's somehow out of this issue. Maybe I can open another PR to remove them.

Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for updating this PR @Aitozi. I think it is ok to remove the UnhandledErrorListener from the ZooKeeperLeaderElectionDriver and ZooKeeperLeaderRetrievalDriver as a follow-up step. What we probably should do is to pass in a FatalErrorHandler into the ZooKeeperUtils.startCuratorFramework. That way we can integrate the failure handling with the ClusterEntrypoint, TaskManagerRunner, etc.

Comment on lines 245 to 252
(message, throwable) -> {
LOG.error(
"Exiting process for unhandled error in curator framework, "
+ "error message: {}, exiting code: {}",
message,
ZOOKEEPER_FAILURE_EXIT_CODE,
throwable);
FlinkSecurityManager.forceProcessExit(ZOOKEEPER_FAILURE_EXIT_CODE);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe it is actually better if we can pass in a FatalErrorHandler. That way we can integrate the failure handling with the general logic (e.g. in the ClusterEntrypoint we add additional information if there was a out of memory error).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Then we wouldn't have the problems of how to test for FlinkSecurityManager.forceProcessExit.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Get it, I will add the FatalErrorHandler

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Passed FatalErrorHandler to ZookeeperUtils#startCuratorFramework. Please help review again.

@Aitozi
Copy link
Contributor Author

Aitozi commented Sep 2, 2021

@flinkbot run azure

Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for updating this PR @Aitozi. I think it looks almost mergeable. I left some suggestions for improvements and a comment how to set the FatalErrorHandler in the RestClusterClient. Please take a look.

Comment on lines 187 to 212
this.configuration = checkNotNull(configuration);

this.restClusterClientConfiguration =
RestClusterClientConfiguration.fromConfiguration(configuration);

if (restClient != null) {
this.restClient = restClient;
} else {
this.restClient = new RestClient(configuration, executorService);
}

this.waitStrategy = checkNotNull(waitStrategy);
this.clusterId = checkNotNull(clusterId);

this.clientHAServices =
HighAvailabilityServicesUtils.createClientHAService(
configuration,
exception ->
webMonitorLeaderRetriever.handleError(
new FlinkException(exception)));

this.webMonitorRetrievalService = clientHAServices.getClusterRestEndpointLeaderRetriever();
this.retryExecutorService =
Executors.newSingleThreadScheduledExecutor(
new ExecutorThreadFactory("Flink-RestClusterClient-Retry"));
startLeaderRetrievers();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This does not feel right. We are effectively copying the next constructor. I think there must be another solution. Maybe, if we don't specify a ClientHighAvailabilityServices, then we instantiate it with a handler that terminates the process in case of an unhandled exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But I think it's strange to directly terminate the client process. And the current webMonitorRetrievalService also just complete the future with exception. I have to copy the constructor mainly due to there is a constructor which accept the StandaloneClientHAServices.

Copy link
Contributor Author

@Aitozi Aitozi Sep 2, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe I can use an ClientHighAvailabilityServicesFactory here to decouple the RestClusterClient with concrete ClientHighAvailabilityServices

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this could be a good solution to decouple these two components a bit better.

@@ -149,9 +151,12 @@ public static String generateLeaderLatchPath(String path) {
* Starts a {@link CuratorFramework} instance and connects it to the given ZooKeeper quorum.
*
* @param configuration {@link Configuration} object containing the configuration values
* @param fatalErrorHandler {@link FatalErrorHandler} fatalErrorHandler to handle unExpected
* error of {@link CuratorFramework}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* error of {@link CuratorFramework}
* errors of {@link CuratorFramework}

"Unhandled error in curator framework, error message: {}",
message,
throwable);
// The exception thrown in UnhandledErrorListener will be catch by
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Which exception is thrown here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I comment to draw attention to UnhandledErrorListener will catch all exception when executed. The FatalErrorHandler used here should be aware of this. If a handler just throw an exception may be eat by curator framework.

@Aitozi Aitozi force-pushed the zk-handler branch 2 times, most recently from fd4a9eb to 948cd08 Compare September 2, 2021 16:56
@Aitozi
Copy link
Contributor Author

Aitozi commented Sep 2, 2021

Thanks for your review , I have fixed all of your comments, plz take a look again. @tillrohrmann

@Aitozi
Copy link
Contributor Author

Aitozi commented Sep 2, 2021

@flinkbot run azure

Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for updating this PR @Aitozi. I think we are almost done. I only had a few minor comments left concerning the usage of anonymous classes. Once these comments are addressed and Azure gives green light, I will merge this PR.

There are also some merge conflicts. Maybe you can rebase this PR while resolving the last comments onto the latest master. Moreover, it would be awesome if you could create a backport against the release-1.14 branch so that we also run CI for this version before merging it back.

Comment on lines 176 to 183
new ClientHighAvailabilityServicesFactory() {
@Override
public ClientHighAvailabilityServices create(
Configuration configuration, FatalErrorHandler fatalErrorHandler)
throws Exception {
return HighAvailabilityServicesUtils.createClientHAService(
configuration, fatalErrorHandler);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's create a DefaultClientHighAvailabilityServicesFactory that has a singleton INSTANCE that does exactly this here. Then we don't have to use anonymous classes.

Comment on lines 240 to 230
new FatalErrorHandler() {
@Override
public void onFatalError(Throwable exception) {
webMonitorLeaderRetriever.handleError(
new FlinkException(exception));
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
new FatalErrorHandler() {
@Override
public void onFatalError(Throwable exception) {
webMonitorLeaderRetriever.handleError(
new FlinkException(exception));
}
exception ->
webMonitorLeaderRetriever.handleError(
new FlinkException(exception))

A bit less boilerplate.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dose this means add more error msg?

configuration,
clusterId,
new StandaloneClientHAServices(getWebMonitorAddress(configuration)));
String webMonitorAddress = getWebMonitorAddress(configuration);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would move this into the factory because we don't know whether configuration won't be changed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I made a wrong thought here, done!

@Aitozi
Copy link
Contributor Author

Aitozi commented Sep 2, 2021

Having addressed all your concerns, thanks for your review time @tillrohrmann

@Aitozi
Copy link
Contributor Author

Aitozi commented Sep 2, 2021

Created the backport PR

@Aitozi
Copy link
Contributor Author

Aitozi commented Sep 2, 2021

@flinkbot run azure

Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the update. LGTM. Addressing my last comments while merging this PR.

Comment on lines 119 to 127
new ClientHighAvailabilityServicesFactory() {
@Override
public ClientHighAvailabilityServices create(
Configuration configuration,
FatalErrorHandler fatalErrorHandler)
throws Exception {
return new StandaloneClientHAServices(
getWebMonitorAddress(configuration));
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here probably.

tillrohrmann pushed a commit to Aitozi/flink that referenced this pull request Sep 3, 2021
…andledErrorListener before start zk client

This closes apache#17053.
…andledErrorListener before start zk client

This closes apache#17053.
@tillrohrmann tillrohrmann merged commit 6b4f8a4 into apache:master Sep 3, 2021
niklassemmler pushed a commit to niklassemmler/flink that referenced this pull request Feb 3, 2022
…andledErrorListener before start zk client

This closes apache#17053.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
4 participants