-
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
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -37,7 +37,9 @@ | |
| import org.apache.flink.kubernetes.operator.utils.EventRecorder; | ||
| import org.apache.flink.kubernetes.operator.utils.FlinkStateSnapshotUtils; | ||
| import org.apache.flink.kubernetes.operator.utils.SnapshotUtils; | ||
| import org.apache.flink.runtime.rest.util.RestClientException; | ||
| import org.apache.flink.util.CollectionUtil; | ||
| import org.apache.flink.util.ExceptionUtils; | ||
|
|
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
@@ -441,21 +443,33 @@ private long getMaxCountForSnapshotType( | |
| } | ||
|
|
||
| private void observeLatestCheckpoint(FlinkResourceContext<CR> ctx, String jobId) { | ||
|
|
||
| var status = ctx.getResource().getStatus(); | ||
| var jobStatus = status.getJobStatus(); | ||
|
|
||
| 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); | ||
| } | ||
| }); | ||
| try { | ||
| 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); | ||
|
||
| return; | ||
| } | ||
| throw 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.
Is message always not null here?
Uh oh!
There was an error while loading. Please reload this page.
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.