gyfora commented on code in PR #948:
URL:
https://github.com/apache/flink-kubernetes-operator/pull/948#discussion_r1968220428
##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java:
##########
@@ -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"))
Review Comment:
Is message always not null here?
##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java:
##########
@@ -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);
Review Comment:
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
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]