Skip to content

[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
wants to merge 10 commits into from
Closed
Show file tree
Hide file tree
Changes from all 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
30 changes: 29 additions & 1 deletion docs/content/docs/deployment/advanced/historyserver.md
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,35 @@ historyserver.archive.fs.dir: hdfs:///completed-jobs
historyserver.archive.fs.refresh-interval: 10000
```

The contained archives are downloaded and cached in the local filesystem. The local directory for this is configured via `historyserver.web.tmpdir`.
The contained archives are downloaded and cached in the local filesystem or a key-value store, depending on the configured storage backend. The local directory for cache storage is configured via `historyserver.web.tmpdir`.

**Storage Backend Configuration**

The History Server supports multiple storage backends for storing completed job archives.

***Available Options***
- **`file` (default)**: Stores job archives in a file system directory.
- **`kvstore`**: Stores job archives in a key-value store (e.g., RocksDB), improving retrieval speed.

***Choosing Between `file` and `kvstore`***

| Backend | Pros | Cons |
|----------|-----------------------------|------------------------------|
| `file` (default) | Simple, easy to configure | Slower retrieval for large archives, high inode usage |
| `kvstore` | More scalable, lower file system overhead, faster retrieval | Requires a key-value store backend, more complex setup |

***Configuration Example***

To configure the storage backend, update `flink-conf.yaml`:

```yaml
# Use file-based storage (default)
historyserver.storage.backend: file

# Use key-value store for storage
historyserver.storage.backend: kvstore

```

Check out the configuration page for a [complete list of configuration options]({{< ref "docs/deployment/config" >}}#history-server).

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,5 +74,13 @@
<td>String</td>
<td>Local directory that is used by the history server REST API for temporary files.</td>
</tr>
<tr>
<td><h5>historyserver.storage.backend</h5></td>
<td style="word-wrap: break-word;">file</td>
<td>String</td>
<td>Defines the storage backend for the History Server where completed job archives are downloaded and cached.
The default value is <code>file</code>, which stores archives in the file system.
If set to <code>kvstore</code>, archives are stored in a key-value store backend.</td>
</tr>
</tbody>
</table>
Original file line number Diff line number Diff line change
Expand Up @@ -143,5 +143,20 @@ public class HistoryServerOptions {
code("IllegalConfigurationException"))
.build());

public static final ConfigOption<String> HISTORY_SERVER_STORAGE_BACKEND =
Copy link
Contributor

@davidradl davidradl Jan 9, 2025

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.

key("historyserver.storage.backend")
.stringType()
.defaultValue("file")
.withDescription(
Description.builder()
.text("Defines the storage backend for the History Server.")
.text(
"The default value is `file`, which stores archives in the file system. ")
.text(
"If set to `kvstore`, archives are stored in a key-value store backend. ")
.text(
"Note that switching storage backends requires configuration compatibility.")
.build());

private HistoryServerOptions() {}
}
6 changes: 6 additions & 0 deletions flink-runtime-web/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,12 @@ under the License.
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>com.ververica</groupId>
<artifactId>frocksdbjni</artifactId>
<version>8.10.0-ververica-beta-1.0</version>
</dependency>

<!-- ===================================================
Dependencies for the Web Server
=================================================== -->
Expand Down
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 {
Copy link
Contributor

@davidradl davidradl Jan 9, 2025

Choose a reason for hiding this comment

The 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();
}
}
Loading