-
Notifications
You must be signed in to change notification settings - Fork 498
[FLINK-37370] [Observer] Finished batch jobs throw ReconciliationException and never reach FINISHED in the CR #948
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
Conversation
…ption and never reach FINISHED in the CR
...s-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java
Outdated
Show resolved
Hide resolved
| }); | ||
| } catch (Exception e) { | ||
| if (ExceptionUtils.findThrowable(e, RestClientException.class) | ||
| .map(ex -> ex.getMessage().contains("Checkpointing has not been enabled")) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is message always not null here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point. I'm not sure. I'll add logic to check for null.
| ctx.getFlinkService() | ||
| .getLastCheckpoint(JobID.fromHexString(jobId), ctx.getObserveConfig()) | ||
| .ifPresentOrElse( | ||
| snapshot -> jobStatus.setUpgradeSavepointPath(snapshot.getLocation()), | ||
| () -> { | ||
| if (ReconciliationUtils.isJobCancelled(status)) { | ||
| // For cancelled jobs the observed savepoint is always definite, | ||
| // so if empty we know the job doesn't have any | ||
| // checkpoints/savepoints | ||
| jobStatus.setUpgradeSavepointPath(null); | ||
| } | ||
| }); | ||
| } catch (Exception e) { | ||
| if (ExceptionUtils.findThrowable(e, RestClientException.class) | ||
| .map(ex -> ex.getMessage().contains("Checkpointing has not been enabled")) | ||
| .orElse(false)) { | ||
| LOG.warn( | ||
| "Checkpointing not enabled for job {}, skipping checkpoint observation", | ||
| jobId, | ||
| e); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder if the try/catch logic should be part of getLastCheckpoint of the flink service. That would mean that anywhere else we call this in the future we get a consistently good behaviour for batch jobs
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I thought about that. I'm open to either. We would just have getLastCheckpoint catch the exception and return optional.empty()
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think that would make sense 👍
What is the purpose of the change
This pull request fixes a bug where in the observation phase, the operator tries to observe savepoint information for batch jobs and fails since checkpointing is not enabled for batch jobs.
More information in: https://issues.apache.org/jira/browse/FLINK-37370
Brief change log
Some questions
Verifying this change
If you think my approach is good, as a next step, happy to add some tests for the new logic I've introduced. Where would you like to see tests?
Does this pull request potentially affect one of the following parts:
CustomResourceDescriptors: noDocumentation