@Internal
public interface JobResultStore
dirty - indicating that the corresponding job is not properly cleaned up, yet.
clean - indicating that the cleanup of the corresponding job is performed and no
further actions need to be applied.
| Modifier and Type | Method and Description |
|---|---|
CompletableFuture<Void> |
createDirtyResultAsync(JobResultEntry jobResultEntry)
Registers the passed
JobResultEntry instance as dirty which indicates that
clean-up operations still need to be performed. |
Set<JobResult> |
getDirtyResults()
Get the persisted
JobResult instances that are marked as dirty. |
CompletableFuture<Boolean> |
hasCleanJobResultEntryAsync(org.apache.flink.api.common.JobID jobId)
Returns the future of whether the store contains a
clean entry for the given JobID. |
CompletableFuture<Boolean> |
hasDirtyJobResultEntryAsync(org.apache.flink.api.common.JobID jobId)
Returns the future of whether the store contains a
dirty entry for the given JobID. |
default CompletableFuture<Boolean> |
hasJobResultEntryAsync(org.apache.flink.api.common.JobID jobId)
Returns the future of whether the store already contains an entry for a job.
|
CompletableFuture<Void> |
markResultAsCleanAsync(org.apache.flink.api.common.JobID jobId)
Marks an existing
JobResultEntry as clean. |
CompletableFuture<Void> createDirtyResultAsync(JobResultEntry jobResultEntry)
JobResultEntry instance as dirty which indicates that
clean-up operations still need to be performed. Once the job resource cleanup has been
finalized, we can mark the JobResultEntry as clean result using markResultAsCleanAsync(JobID).jobResultEntry - The job result we wish to persist.IllegalStateException if the passed jobResultEntry has a JobID attached that is already registered in this JobResultStore.CompletableFuture<Void> markResultAsCleanAsync(org.apache.flink.api.common.JobID jobId)
JobResultEntry as clean. This indicates that no more
resource cleanup steps need to be performed. No actions should be triggered if the passed
JobID belongs to a job that was already marked as clean.jobId - Ident of the job we wish to mark as clean.NoSuchElementException. i.e. there is no
corresponding dirty job present in the store for the given JobID.default CompletableFuture<Boolean> hasJobResultEntryAsync(org.apache.flink.api.common.JobID jobId)
jobId - Ident of the job we wish to check the store for.true if a dirty or clean
JobResultEntry exists for the given JobID; otherwise false.CompletableFuture<Boolean> hasDirtyJobResultEntryAsync(org.apache.flink.api.common.JobID jobId)
dirty entry for the given JobID.jobId - Ident of the job we wish to check the store for.true, if a dirty entry exists
for the given JobID; otherwise false.CompletableFuture<Boolean> hasCleanJobResultEntryAsync(org.apache.flink.api.common.JobID jobId)
clean entry for the given JobID.jobId - Ident of the job we wish to check the store for.true, if a clean entry exists
for the given JobID; otherwise a successfully completed future with false.Set<JobResult> getDirtyResults() throws IOException
JobResult instances that are marked as dirty. This is
useful for recovery of finalization steps.JobResults from the store.IOException - if collecting the set of dirty results failed for IO reasons.Copyright © 2014–2025 The Apache Software Foundation. All rights reserved.