Skip to content
Draft
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
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,12 @@
<td>Integer</td>
<td>The maximum number of jobs to retain in each archive directory defined by `historyserver.archive.fs.dir`. If set to `-1`(default), there is no limit to the number of archives. If set to `0` or less than `-1` HistoryServer will throw an <code class="highlighter-rouge">IllegalConfigurationException</code>. </td>
</tr>
<tr>
<td><h5>historyserver.archive.retained-ttl</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>Duration</td>
<td>The time-to-live duration to retain the jobs archived in each archive directory defined by `historyserver.archive.fs.dir`. Note, when there are multiple history server instances with different configurations, they are working independently today and may have conflict configs. This is an existing problem. For the configuration, when there are multiple history servers instances. You can enable the configuration option like following: <ul><li>Using the same `<code class="highlighter-rouge">historyserver.archive.fs.dir</code>` directory as the refresh directories, you should enable and configure this feature in only one HistoryServer instance to avoid errors caused by multiple instances simultaneously cleaning up remote files,</li><li>Or you can keep the value of this configuration consistent across them.</li></ul></td>
</tr>
<tr>
<td><h5>historyserver.log.jobmanager.url-pattern</h5></td>
<td style="word-wrap: break-word;">(none)</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@

import static org.apache.flink.configuration.ConfigOptions.key;
import static org.apache.flink.configuration.description.TextElement.code;
import static org.apache.flink.configuration.description.TextElement.text;

/** The set of configuration options relating to the HistoryServer. */
@PublicEvolving
Expand Down Expand Up @@ -143,5 +144,29 @@ public class HistoryServerOptions {
code("IllegalConfigurationException"))
.build());

public static final ConfigOption<Duration> HISTORY_SERVER_RETAINED_TTL =
key("historyserver.archive.retained-ttl")
.durationType()
.noDefaultValue()
.withDescription(
Description.builder()
.text(
String.format(
"The time-to-live duration to retain the jobs archived in each archive directory defined by `%s`. ",
HISTORY_SERVER_ARCHIVE_DIRS.key()))
.text(
"Note, when there are multiple history server instances with different configurations, "
+ "they are working independently today and may have conflict configs. "
+ "This is an existing problem. "
+ "For the configuration, when there are multiple history servers instances. You can enable the configuration option like following: ")
.list(
text(
"Using the same `%s` directory as the refresh directories, "
+ "you should enable and configure this feature in only one HistoryServer instance to avoid errors caused by multiple instances simultaneously cleaning up remote files,",
code(HISTORY_SERVER_ARCHIVE_DIRS.key())),
text(
"Or you can keep the value of this configuration consistent across them."))
.build());

private HistoryServerOptions() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.configuration.HistoryServerOptions;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.core.plugin.PluginUtils;
Expand All @@ -38,6 +37,7 @@
import org.apache.flink.runtime.security.SecurityUtils;
import org.apache.flink.runtime.util.EnvironmentInformation;
import org.apache.flink.runtime.util.Runnables;
import org.apache.flink.runtime.webmonitor.history.retaining.JobArchivedRetainedStrategyImpl;
import org.apache.flink.runtime.webmonitor.utils.LogUrlUtil;
import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
import org.apache.flink.util.ExceptionUtils;
Expand Down Expand Up @@ -238,19 +238,13 @@ public HistoryServer(

refreshIntervalMillis =
config.get(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_REFRESH_INTERVAL).toMillis();
int maxHistorySize = config.get(HistoryServerOptions.HISTORY_SERVER_RETAINED_JOBS);
if (maxHistorySize == 0 || maxHistorySize < -1) {
throw new IllegalConfigurationException(
"Cannot set %s to 0 or less than -1",
HistoryServerOptions.HISTORY_SERVER_RETAINED_JOBS.key());
}
archiveFetcher =
new HistoryServerArchiveFetcher(
refreshDirs,
webDir,
jobArchiveEventListener,
cleanupExpiredArchives,
maxHistorySize);
JobArchivedRetainedStrategyImpl.createFrom(config));

this.shutdownHook =
ShutdownHookUtil.addShutdownHook(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders;
import org.apache.flink.runtime.webmonitor.history.retaining.JobArchivesRetainedStrategy;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.jackson.JacksonMapperFactory;

Expand Down Expand Up @@ -112,8 +113,7 @@ public ArchiveEventType getType() {
private final List<HistoryServer.RefreshLocation> refreshDirs;
private final Consumer<ArchiveEvent> jobArchiveEventListener;
private final boolean processExpiredArchiveDeletion;
private final boolean processBeyondLimitArchiveDeletion;
private final int maxHistorySize;
private final JobArchivesRetainedStrategy jobRetainedStrategy;

/** Cache of all available jobs identified by their id. */
private final Map<Path, Set<String>> cachedArchivesPerRefreshDirectory;
Expand All @@ -127,13 +127,12 @@ public ArchiveEventType getType() {
File webDir,
Consumer<ArchiveEvent> jobArchiveEventListener,
boolean cleanupExpiredArchives,
int maxHistorySize)
JobArchivesRetainedStrategy jobRetainedStrategy)
throws IOException {
this.refreshDirs = checkNotNull(refreshDirs);
this.jobArchiveEventListener = jobArchiveEventListener;
this.processExpiredArchiveDeletion = cleanupExpiredArchives;
this.maxHistorySize = maxHistorySize;
this.processBeyondLimitArchiveDeletion = this.maxHistorySize > 0;
this.jobRetainedStrategy = checkNotNull(jobRetainedStrategy);
this.cachedArchivesPerRefreshDirectory = new HashMap<>();
for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
cachedArchivesPerRefreshDirectory.put(refreshDir.getPath(), new HashSet<>());
Expand All @@ -159,7 +158,7 @@ void fetchArchives() {
Map<Path, Set<String>> jobsToRemove = new HashMap<>();
cachedArchivesPerRefreshDirectory.forEach(
(path, archives) -> jobsToRemove.put(path, new HashSet<>(archives)));
Map<Path, Set<Path>> archivesBeyondSizeLimit = new HashMap<>();
Map<Path, Set<Path>> archivesBeyondRetainedLimit = new HashMap<>();
for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) {
Path refreshDir = refreshLocation.getPath();
LOG.debug("Checking archive directory {}.", refreshDir);
Expand All @@ -176,7 +175,7 @@ void fetchArchives() {
continue;
}

int historySize = 0;
int fileOrderedIndexOnModifiedTime = 0;
for (FileStatus jobArchive : jobArchives) {
Path jobArchivePath = jobArchive.getPath();
String jobID = jobArchivePath.getName();
Expand All @@ -186,9 +185,10 @@ void fetchArchives() {

jobsToRemove.get(refreshDir).remove(jobID);

historySize++;
if (historySize > maxHistorySize && processBeyondLimitArchiveDeletion) {
archivesBeyondSizeLimit
fileOrderedIndexOnModifiedTime++;
if (!jobRetainedStrategy.shouldRetain(
jobArchive, fileOrderedIndexOnModifiedTime)) {
archivesBeyondRetainedLimit
.computeIfAbsent(refreshDir, ignored -> new HashSet<>())
.add(jobArchivePath);
continue;
Expand Down Expand Up @@ -220,8 +220,8 @@ void fetchArchives() {
&& processExpiredArchiveDeletion) {
events.addAll(cleanupExpiredJobs(jobsToRemove));
}
if (!archivesBeyondSizeLimit.isEmpty() && processBeyondLimitArchiveDeletion) {
events.addAll(cleanupJobsBeyondSizeLimit(archivesBeyondSizeLimit));
if (!archivesBeyondRetainedLimit.isEmpty()) {
events.addAll(cleanupJobsBeyondSizeLimit(archivesBeyondRetainedLimit));
}
if (!events.isEmpty()) {
updateJobOverview(webOverviewDir, webDir);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* 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.retaining;

import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.core.fs.FileStatus;

import javax.annotation.Nullable;

import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Optional;

import static org.apache.flink.configuration.HistoryServerOptions.HISTORY_SERVER_RETAINED_JOBS;
import static org.apache.flink.configuration.HistoryServerOptions.HISTORY_SERVER_RETAINED_TTL;

/** The retained strategy. */
public class JobArchivedRetainedStrategyImpl implements JobArchivesRetainedStrategy {

public static JobArchivesRetainedStrategy createFrom(ReadableConfig config) {
int maxHistorySizeByOldKey = config.get(HISTORY_SERVER_RETAINED_JOBS);
if (maxHistorySizeByOldKey == 0 || maxHistorySizeByOldKey < -1) {
throw new IllegalConfigurationException(
"Cannot set %s to 0 or less than -1", HISTORY_SERVER_RETAINED_JOBS.key());
}
Optional<Duration> retainedTtlOpt = config.getOptional(HISTORY_SERVER_RETAINED_TTL);
return new JobArchivedRetainedStrategyImpl(
new QuantityBasedJobRetainedStrategy(maxHistorySizeByOldKey),
new TimeToLiveBasedJobRetainedStrategy(retainedTtlOpt.orElse(null)));
}

private final List<JobArchivesRetainedStrategy> strategies;

JobArchivedRetainedStrategyImpl(JobArchivesRetainedStrategy... strategies) {
this.strategies =
strategies == null || strategies.length == 0
? Collections.emptyList()
: Arrays.asList(strategies);
}

@Override
public boolean shouldRetain(FileStatus file, int fileOrderedIndex) {
if (strategies.isEmpty()) {
return true;
}
return strategies.stream().allMatch(s -> s.shouldRetain(file, fileOrderedIndex));
}
}

/** The time to live based retained strategy. */
class TimeToLiveBasedJobRetainedStrategy implements JobArchivesRetainedStrategy {

@Nullable private final Duration ttlThreshold;

TimeToLiveBasedJobRetainedStrategy(Duration ttlThreshold) {
this.ttlThreshold = ttlThreshold;
}

@Override
public boolean shouldRetain(FileStatus file, int fileOrderedIndex) {
if (ttlThreshold == null || ttlThreshold.toMillis() <= 0L) {
return true;
}
return Instant.now().toEpochMilli() - file.getModificationTime() < ttlThreshold.toMillis();
}
}

/** The job quantity based retained strategy. */
class QuantityBasedJobRetainedStrategy implements JobArchivesRetainedStrategy {

private final int quantityThreshold;

QuantityBasedJobRetainedStrategy(int quantityThreshold) {
this.quantityThreshold = quantityThreshold;
}

@Override
public boolean shouldRetain(FileStatus file, int fileOrderedIndex) {
if (quantityThreshold <= 0) {
return true;
}
return quantityThreshold >= fileOrderedIndex;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.retaining;

import org.apache.flink.core.fs.FileStatus;

/** To define the strategy interface to judge whether the file should be retained. */
public interface JobArchivesRetainedStrategy {

/**
* Judge whether the file should be retained.
*
* @param file the target file to judge.
* @param fileOrderedIndex the specified order index position of the target file,
* @return The result that indicates whether the file should be retained.
*/
boolean shouldRetain(FileStatus file, int fileOrderedIndex);
}
Loading