-
Notifications
You must be signed in to change notification settings - Fork 13.7k
[FLINK-36429] [runtime-web] Enhancing Flink History Server File Storage and Retrieval with RocksDB #25838
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
Closed
Closed
[FLINK-36429] [runtime-web] Enhancing Flink History Server File Storage and Retrieval with RocksDB #25838
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
0146904
Add KVStore interface, RocksDB impl, and unit tests
Shawnsuun 54a0e48
Add RocksDB configuration to HistoryServerOptions
Shawnsuun 7c682d4
Add ArchiveFetcher abstract class and extension for file system and R…
Shawnsuun a6346d8
add tests for archiveFetcher
Shawnsuun 5e25748
Add server handlers for KVStore and static file support
Shawnsuun f87328d
Update HistoryServer and tests for RocksDB integration
Shawnsuun e1b6164
Add missing Apache license header
Shawnsuun a478cbc
fixed testUpdateJobOverview fail
b17c658
fixed spotless
Shawnsuun 5967804
Add storage backend config to History Server docs
Shawnsuun File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 hidden or 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 hidden or 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
265 changes: 265 additions & 0 deletions
265
...runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/ArchiveFetcher.java
This file contains hidden or 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,265 @@ | ||
/* | ||
* 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.webmonitor.history; | ||
|
||
import org.apache.flink.api.common.JobID; | ||
import org.apache.flink.api.common.JobStatus; | ||
import org.apache.flink.core.fs.FileStatus; | ||
import org.apache.flink.core.fs.FileSystem; | ||
import org.apache.flink.core.fs.Path; | ||
import org.apache.flink.runtime.execution.ExecutionState; | ||
import org.apache.flink.runtime.messages.webmonitor.JobDetails; | ||
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; | ||
import org.apache.flink.util.jackson.JacksonMapperFactory; | ||
|
||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.io.IOException; | ||
import java.io.StringWriter; | ||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.Collections; | ||
import java.util.Comparator; | ||
import java.util.HashMap; | ||
import java.util.HashSet; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.function.Consumer; | ||
|
||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
|
||
/** | ||
* The {@code ArchiveFetcher} class is an abstract class that defines the basic structure for | ||
* fetching job archives in the Flink History Server. Implementations of this class should provide | ||
* the logic for retrieving and processing job archives. | ||
* | ||
* <p>This class may be extended by specific implementations to support different storage backends | ||
* for job archives, such as local file systems or key-value stores. | ||
*/ | ||
public abstract class ArchiveFetcher { | ||
|
||
/** Possible job archive operations in history-server. */ | ||
public enum ArchiveEventType { | ||
/** Job archive was found in one refresh location and created in history server. */ | ||
CREATED, | ||
/** | ||
* Job archive was deleted from one of refresh locations and deleted from history server. | ||
*/ | ||
DELETED | ||
} | ||
|
||
/** Representation of job archive event. */ | ||
public static class ArchiveEvent { | ||
private final String jobID; | ||
private final ArchiveFetcher.ArchiveEventType operation; | ||
|
||
ArchiveEvent(String jobID, ArchiveFetcher.ArchiveEventType operation) { | ||
this.jobID = jobID; | ||
this.operation = operation; | ||
} | ||
|
||
public String getJobID() { | ||
return jobID; | ||
} | ||
|
||
public ArchiveFetcher.ArchiveEventType getType() { | ||
return operation; | ||
} | ||
} | ||
|
||
protected static final Logger LOG = LoggerFactory.getLogger(ArchiveFetcher.class); | ||
|
||
protected static final ObjectMapper MAPPER = JacksonMapperFactory.createObjectMapper(); | ||
|
||
protected final List<HistoryServer.RefreshLocation> refreshDirs; | ||
protected final Consumer<ArchiveEvent> jobArchiveEventListener; | ||
protected final boolean processExpiredArchiveDeletion; | ||
protected final boolean processBeyondLimitArchiveDeletion; | ||
protected final int maxHistorySize; | ||
|
||
/** Cache of all available jobs identified by their id. */ | ||
protected final Map<Path, Set<String>> cachedArchivesPerRefreshDirectory; | ||
|
||
protected ArchiveFetcher( | ||
List<HistoryServer.RefreshLocation> refreshDirs, | ||
Consumer<ArchiveEvent> jobArchiveEventListener, | ||
boolean cleanupExpiredArchives, | ||
int maxHistorySize) { | ||
this.refreshDirs = checkNotNull(refreshDirs); | ||
this.jobArchiveEventListener = jobArchiveEventListener; | ||
this.processExpiredArchiveDeletion = cleanupExpiredArchives; | ||
this.maxHistorySize = maxHistorySize; | ||
this.processBeyondLimitArchiveDeletion = this.maxHistorySize > 0; | ||
this.cachedArchivesPerRefreshDirectory = new HashMap<>(); | ||
for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { | ||
cachedArchivesPerRefreshDirectory.put(refreshDir.getPath(), new HashSet<>()); | ||
} | ||
} | ||
|
||
abstract void fetchArchives(); | ||
|
||
static FileStatus[] listArchives(FileSystem refreshFS, Path refreshDir) throws IOException { | ||
// contents of /:refreshDir | ||
FileStatus[] jobArchives = refreshFS.listStatus(refreshDir); | ||
if (jobArchives == null) { | ||
// the entire refreshDirectory was removed | ||
return new FileStatus[0]; | ||
} | ||
|
||
Arrays.sort( | ||
jobArchives, Comparator.comparingLong(FileStatus::getModificationTime).reversed()); | ||
|
||
return jobArchives; | ||
} | ||
|
||
static boolean isValidJobID(String jobId, Path refreshDir) { | ||
try { | ||
JobID.fromHexString(jobId); | ||
return true; | ||
} catch (IllegalArgumentException iae) { | ||
LOG.debug( | ||
"Archive directory {} contained file with unexpected name {}. Ignoring file.", | ||
refreshDir, | ||
jobId, | ||
iae); | ||
return false; | ||
} | ||
} | ||
|
||
abstract void processArchive(String jobID, Path jobArchivePath) throws IOException; | ||
|
||
List<ArchiveFetcher.ArchiveEvent> cleanupJobsBeyondSizeLimit( | ||
Map<Path, Set<Path>> jobArchivesToRemove) { | ||
Map<Path, Set<String>> allJobIdsToRemoveFromOverview = new HashMap<>(); | ||
|
||
for (Map.Entry<Path, Set<Path>> pathSetEntry : jobArchivesToRemove.entrySet()) { | ||
HashSet<String> jobIdsToRemoveFromOverview = new HashSet<>(); | ||
|
||
for (Path archive : pathSetEntry.getValue()) { | ||
jobIdsToRemoveFromOverview.add(archive.getName()); | ||
try { | ||
archive.getFileSystem().delete(archive, false); | ||
} catch (IOException ioe) { | ||
LOG.warn("Could not delete old archive " + archive, ioe); | ||
} | ||
} | ||
allJobIdsToRemoveFromOverview.put(pathSetEntry.getKey(), jobIdsToRemoveFromOverview); | ||
} | ||
|
||
return cleanupExpiredJobs(allJobIdsToRemoveFromOverview); | ||
} | ||
|
||
List<ArchiveFetcher.ArchiveEvent> cleanupExpiredJobs(Map<Path, Set<String>> jobsToRemove) { | ||
|
||
List<ArchiveFetcher.ArchiveEvent> deleteLog = new ArrayList<>(); | ||
LOG.info("Archive directories for jobs {} were deleted.", jobsToRemove); | ||
|
||
jobsToRemove.forEach( | ||
(refreshDir, archivesToRemove) -> { | ||
cachedArchivesPerRefreshDirectory.get(refreshDir).removeAll(archivesToRemove); | ||
}); | ||
jobsToRemove.values().stream() | ||
.flatMap(Set::stream) | ||
.forEach( | ||
removedJobID -> { | ||
deleteJobFiles(removedJobID); | ||
deleteLog.add( | ||
new ArchiveFetcher.ArchiveEvent( | ||
removedJobID, ArchiveFetcher.ArchiveEventType.DELETED)); | ||
}); | ||
|
||
return deleteLog; | ||
} | ||
|
||
abstract void deleteJobFiles(String jobID); | ||
|
||
static String convertLegacyJobOverview(String legacyOverview) throws IOException { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I this javadoc would be helpful here, including the definition of a legacy server and the expected output. I thin all the methods would benefit from javadoc. |
||
JsonNode root = MAPPER.readTree(legacyOverview); | ||
JsonNode finishedJobs = root.get("finished"); | ||
JsonNode job = finishedJobs.get(0); | ||
|
||
JobID jobId = JobID.fromHexString(job.get("jid").asText()); | ||
String name = job.get("name").asText(); | ||
JobStatus state = JobStatus.valueOf(job.get("state").asText()); | ||
|
||
long startTime = job.get("start-time").asLong(); | ||
long endTime = job.get("end-time").asLong(); | ||
long duration = job.get("duration").asLong(); | ||
long lastMod = job.get("last-modification").asLong(); | ||
|
||
JsonNode tasks = job.get("tasks"); | ||
int numTasks = tasks.get("total").asInt(); | ||
JsonNode pendingNode = tasks.get("pending"); | ||
// for flink version < 1.4 we have pending field, | ||
// when version >= 1.4 pending has been split into scheduled, deploying, and created. | ||
boolean versionLessThan14 = pendingNode != null; | ||
int created = 0; | ||
int scheduled; | ||
int deploying = 0; | ||
|
||
if (versionLessThan14) { | ||
// pending is a mix of CREATED/SCHEDULED/DEPLOYING | ||
// to maintain the correct number of task states we pick SCHEDULED | ||
scheduled = pendingNode.asInt(); | ||
} else { | ||
created = tasks.get("created").asInt(); | ||
scheduled = tasks.get("scheduled").asInt(); | ||
deploying = tasks.get("deploying").asInt(); | ||
} | ||
int running = tasks.get("running").asInt(); | ||
int finished = tasks.get("finished").asInt(); | ||
int canceling = tasks.get("canceling").asInt(); | ||
int canceled = tasks.get("canceled").asInt(); | ||
int failed = tasks.get("failed").asInt(); | ||
|
||
int[] tasksPerState = new int[ExecutionState.values().length]; | ||
tasksPerState[ExecutionState.CREATED.ordinal()] = created; | ||
tasksPerState[ExecutionState.SCHEDULED.ordinal()] = scheduled; | ||
tasksPerState[ExecutionState.DEPLOYING.ordinal()] = deploying; | ||
tasksPerState[ExecutionState.RUNNING.ordinal()] = running; | ||
tasksPerState[ExecutionState.FINISHED.ordinal()] = finished; | ||
tasksPerState[ExecutionState.CANCELING.ordinal()] = canceling; | ||
tasksPerState[ExecutionState.CANCELED.ordinal()] = canceled; | ||
tasksPerState[ExecutionState.FAILED.ordinal()] = failed; | ||
|
||
JobDetails jobDetails = | ||
new JobDetails( | ||
jobId, | ||
name, | ||
startTime, | ||
endTime, | ||
duration, | ||
state, | ||
lastMod, | ||
tasksPerState, | ||
numTasks, | ||
new HashMap<>()); | ||
MultipleJobsDetails multipleJobsDetails = | ||
new MultipleJobsDetails(Collections.singleton(jobDetails)); | ||
|
||
StringWriter sw = new StringWriter(); | ||
MAPPER.writeValue(sw, multipleJobsDetails); | ||
return sw.toString(); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
I suggest adding something to the documentation around this new option in https://nightlies.apache.org/flink/flink-docs-release-1.20/docs/deployment/advanced/historyserver/. It would be good to indicate the pos and cons of file vs kv servers.
The docs could include details on how to extend ArchiveFetcher - as per the javadoc at the top of that class.