forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-25431][runtime] Implement a file-based JobResultStore
This commit contains an implementation of the JobResultStore backed by a file-system. By default, this filesystem is based on the settings for high-availability.storageDir and the cluster-id, but can be overriden with the job-result-store.storage-path setting. Alongside this setting, it also introduces a new option job-result-store.delete-on-commit, which determines whether the underlying file for the job result entry is deleted when it is clean, or whether it is renamed and persisted. By default, this option is true.
- Loading branch information
Showing
18 changed files
with
938 additions
and
295 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
24 changes: 24 additions & 0 deletions
24
docs/layouts/shortcodes/generated/common_high_availability_jrs_section.html
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,24 @@ | ||
<table class="configuration table table-bordered"> | ||
<thead> | ||
<tr> | ||
<th class="text-left" style="width: 20%">Key</th> | ||
<th class="text-left" style="width: 15%">Default</th> | ||
<th class="text-left" style="width: 10%">Type</th> | ||
<th class="text-left" style="width: 55%">Description</th> | ||
</tr> | ||
</thead> | ||
<tbody> | ||
<tr> | ||
<td><h5>job-result-store.delete-on-commit</h5></td> | ||
<td style="word-wrap: break-word;">true</td> | ||
<td>Boolean</td> | ||
<td>Determines whether job results should be automatically removed from the underlying job result store when the corresponding entity transitions into a clean state. If false, the cleaned job results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.</td> | ||
</tr> | ||
<tr> | ||
<td><h5>job-result-store.storage-path</h5></td> | ||
<td style="word-wrap: break-word;">(none)</td> | ||
<td>String</td> | ||
<td>Defines where job results should be stored. This should be an underlying file-system that provides read-after-write consistency. By default, this is <code class="highlighter-rouge">{high-availability.storageDir}/job-result-store/{high-availability.cluster-id}</code>.</td> | ||
</tr> | ||
</tbody> | ||
</table> |
24 changes: 24 additions & 0 deletions
24
docs/layouts/shortcodes/generated/job_result_store_configuration.html
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,24 @@ | ||
<table class="configuration table table-bordered"> | ||
<thead> | ||
<tr> | ||
<th class="text-left" style="width: 20%">Key</th> | ||
<th class="text-left" style="width: 15%">Default</th> | ||
<th class="text-left" style="width: 10%">Type</th> | ||
<th class="text-left" style="width: 55%">Description</th> | ||
</tr> | ||
</thead> | ||
<tbody> | ||
<tr> | ||
<td><h5>job-result-store.delete-on-commit</h5></td> | ||
<td style="word-wrap: break-word;">true</td> | ||
<td>Boolean</td> | ||
<td>Determines whether job results should be automatically removed from the underlying job result store when the corresponding entity transitions into a clean state. If false, the cleaned job results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.</td> | ||
</tr> | ||
<tr> | ||
<td><h5>job-result-store.storage-path</h5></td> | ||
<td style="word-wrap: break-word;">(none)</td> | ||
<td>String</td> | ||
<td>Defines where job results should be stored. This should be an underlying file-system that provides read-after-write consistency. By default, this is <code class="highlighter-rouge">{high-availability.storageDir}/job-result-store/{high-availability.cluster-id}</code>.</td> | ||
</tr> | ||
</tbody> | ||
</table> |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
123 changes: 123 additions & 0 deletions
123
...main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeJobResultStore.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,123 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.runtime.highavailability; | ||
|
||
import org.apache.flink.api.common.JobID; | ||
import org.apache.flink.runtime.jobmaster.JobResult; | ||
import org.apache.flink.util.Preconditions; | ||
import org.apache.flink.util.function.SupplierWithException; | ||
import org.apache.flink.util.function.ThrowingRunnable; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import javax.annotation.concurrent.GuardedBy; | ||
|
||
import java.io.IOException; | ||
import java.util.NoSuchElementException; | ||
import java.util.Set; | ||
import java.util.concurrent.locks.ReadWriteLock; | ||
import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
|
||
/** An abstract class for threadsafe implementations of the {@link JobResultStore}. */ | ||
public abstract class AbstractThreadsafeJobResultStore implements JobResultStore { | ||
|
||
private static final Logger LOG = | ||
LoggerFactory.getLogger(AbstractThreadsafeJobResultStore.class); | ||
|
||
private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); | ||
|
||
@Override | ||
public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException { | ||
Preconditions.checkState( | ||
!hasJobResultEntry(jobResultEntry.getJobId()), | ||
"Job result store already contains an entry for for job %s", | ||
jobResultEntry.getJobId()); | ||
|
||
withWriteLock(() -> createDirtyResultInternal(jobResultEntry)); | ||
} | ||
|
||
@GuardedBy("readWriteLock") | ||
protected abstract void createDirtyResultInternal(JobResultEntry jobResultEntry) | ||
throws IOException; | ||
|
||
@Override | ||
public void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException { | ||
if (hasCleanJobResultEntry(jobId)) { | ||
LOG.debug("The job {} is already marked as clean. No action required.", jobId); | ||
return; | ||
} | ||
|
||
withWriteLock(() -> markResultAsCleanInternal(jobId)); | ||
} | ||
|
||
@GuardedBy("readWriteLock") | ||
protected abstract void markResultAsCleanInternal(JobID jobId) | ||
throws IOException, NoSuchElementException; | ||
|
||
@Override | ||
public boolean hasJobResultEntry(JobID jobId) throws IOException { | ||
return withReadLock( | ||
() -> | ||
hasDirtyJobResultEntryInternal(jobId) | ||
|| hasCleanJobResultEntryInternal(jobId)); | ||
} | ||
|
||
@Override | ||
public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException { | ||
return withReadLock(() -> hasDirtyJobResultEntryInternal(jobId)); | ||
} | ||
|
||
@GuardedBy("readWriteLock") | ||
protected abstract boolean hasDirtyJobResultEntryInternal(JobID jobId) throws IOException; | ||
|
||
@Override | ||
public boolean hasCleanJobResultEntry(JobID jobId) throws IOException { | ||
return withReadLock(() -> hasCleanJobResultEntryInternal(jobId)); | ||
} | ||
|
||
@GuardedBy("readWriteLock") | ||
protected abstract boolean hasCleanJobResultEntryInternal(JobID jobId) throws IOException; | ||
|
||
@Override | ||
public Set<JobResult> getDirtyResults() throws IOException { | ||
return withReadLock(this::getDirtyResultsInternal); | ||
} | ||
|
||
@GuardedBy("readWriteLock") | ||
protected abstract Set<JobResult> getDirtyResultsInternal() throws IOException; | ||
|
||
private void withWriteLock(ThrowingRunnable<IOException> runnable) throws IOException { | ||
readWriteLock.writeLock().lock(); | ||
try { | ||
runnable.run(); | ||
} finally { | ||
readWriteLock.writeLock().unlock(); | ||
} | ||
} | ||
|
||
private <T> T withReadLock(SupplierWithException<T, IOException> runnable) throws IOException { | ||
readWriteLock.readLock().lock(); | ||
try { | ||
return runnable.get(); | ||
} finally { | ||
readWriteLock.readLock().unlock(); | ||
} | ||
} | ||
} |
Oops, something went wrong.