Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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"))
Copy link
Contributor

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?

Copy link
Contributor Author

@luca-p-castelli luca-p-castelli Feb 24, 2025

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.

.orElse(false)) {
LOG.warn(
"Checkpointing not enabled for job {}, skipping checkpoint observation",
jobId,
e);
Copy link
Contributor

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

Copy link
Contributor Author

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()

Copy link
Contributor

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 👍

return;
}
throw e;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration;
import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions;
import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext;
import org.apache.flink.kubernetes.operator.exception.ReconciliationException;
import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException;
import org.apache.flink.kubernetes.operator.observer.CheckpointFetchResult;
import org.apache.flink.kubernetes.operator.observer.CheckpointStatsResult;
Expand Down Expand Up @@ -138,6 +139,7 @@ public class TestingFlinkService extends AbstractFlinkService {
@Getter private final Map<String, Boolean> savepointTriggers = new HashMap<>();
@Getter private final Map<String, Boolean> checkpointTriggers = new HashMap<>();
private final Map<Long, String> checkpointStats = new HashMap<>();
@Setter private boolean throwCheckpointingDisabledError = false;

@Getter private int desiredReplicas = 0;
@Getter private int cancelJobCallCount = 0;
Expand Down Expand Up @@ -574,6 +576,13 @@ public void disposeSavepoint(String savepointPath, Configuration conf) throws Ex

@Override
public Optional<Savepoint> getLastCheckpoint(JobID jobId, Configuration conf) {
if (throwCheckpointingDisabledError) {
throw new ReconciliationException(
"Could not observe latest savepoint information",
new RestClientException(
"Checkpointing has not been enabled", HttpResponseStatus.BAD_REQUEST));
}

jobs.stream()
.filter(js -> js.f1.getJobId().equals(jobId))
.findAny()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -898,4 +898,35 @@ public void jobStatusNotOverwrittenWhenTerminal() throws Exception {
org.apache.flink.api.common.JobStatus.FINISHED,
deployment.getStatus().getJobStatus().getState());
}

@Test
public void observeLatestCheckpointShouldSkipWhenCheckpointingDisabled() throws Exception {
Configuration conf =
configManager.getDeployConfig(deployment.getMetadata(), deployment.getSpec());
flinkService.submitApplicationCluster(deployment.getSpec().getJob(), conf, false);
bringToReadyStatus(deployment);

deployment
.getStatus()
.getJobStatus()
.setState(org.apache.flink.api.common.JobStatus.FINISHED);
var jobs = flinkService.listJobs();
var oldStatus = jobs.get(0).f1;
jobs.get(0).f1 =
new JobStatusMessage(
oldStatus.getJobId(),
oldStatus.getJobName(),
org.apache.flink.api.common.JobStatus.FINISHED,
oldStatus.getStartTime());

flinkService.setThrowCheckpointingDisabledError(true);
observer.observe(deployment, readyContext);

assertEquals(
0,
countErrorEvents(
EventRecorder.Reason.CheckpointError.name(),
deployment.getMetadata().getNamespace(),
"Checkpointing has not been enabled"));
}
}
Loading