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 @@ -149,12 +149,17 @@ private long getCurrentTime() {
return clock.millis();
}

public long getExpiryDurationMillis() {
return expiryDurationMillis;
}

/**
* A custom monotonic clock implementation.
* A custom monotonic clock implementation to allow overriding the current time for testing purposes.
* Implementation of Clock that uses System.nanoTime() for real usage.
* See {@see org.apache.ozone.test.TestClock}
* The class {@code org.apache.ozone.test.TestClock} provides a mock clock which can be used
* to manipulate the current time in tests.
*/
private static final class MonotonicClock extends Clock {
public static final class MonotonicClock extends Clock {
@Override
public long millis() {
return TimeUnit.NANOSECONDS.toMillis(System.nanoTime());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.CONFIG_PREFIX;

import java.time.Duration;
import org.apache.commons.lang3.time.DurationFormatUtils;
import org.apache.hadoop.hdds.conf.Config;
import org.apache.hadoop.hdds.conf.ConfigGroup;
import org.apache.hadoop.hdds.conf.ConfigTag;
Expand Down Expand Up @@ -61,6 +62,7 @@ public class DatanodeConfiguration extends ReconfigurableConfig {
public static final String FAILED_DB_VOLUMES_TOLERATED_KEY = "hdds.datanode.failed.db.volumes.tolerated";
public static final String DISK_CHECK_MIN_GAP_KEY = "hdds.datanode.disk.check.min.gap";
public static final String DISK_CHECK_TIMEOUT_KEY = "hdds.datanode.disk.check.timeout";
public static final String DISK_CHECK_SLIDING_WINDOW_TIMEOUT_KEY = "hdds.datanode.disk.check.sliding.window.timeout";

// Minimum space should be left on volume.
// Ex: If volume has 1000GB and minFreeSpace is configured as 10GB,
Expand Down Expand Up @@ -99,6 +101,9 @@ public class DatanodeConfiguration extends ReconfigurableConfig {

static final Duration DISK_CHECK_TIMEOUT_DEFAULT = Duration.ofMinutes(10);

static final Duration DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT =
Duration.ofMinutes(PERIODIC_DISK_CHECK_INTERVAL_MINUTES_DEFAULT).plus(DISK_CHECK_TIMEOUT_DEFAULT);

static final boolean CONTAINER_SCHEMA_V3_ENABLED_DEFAULT = true;
static final long ROCKSDB_LOG_MAX_FILE_SIZE_BYTES_DEFAULT = 32 * 1024 * 1024;
static final int ROCKSDB_LOG_MAX_FILE_NUM_DEFAULT = 64;
Expand Down Expand Up @@ -404,6 +409,19 @@ public class DatanodeConfiguration extends ReconfigurableConfig {
)
private Duration diskCheckTimeout = DISK_CHECK_TIMEOUT_DEFAULT;

@Config(key = "hdds.datanode.disk.check.sliding.window.timeout",
defaultValue = "70m",
type = ConfigType.TIME,
tags = {ConfigTag.DATANODE},
description = "Time interval after which a disk check"
+ " failure result stored in the sliding window will expire."
+ " Do not set the window timeout period to less than or equal to the disk check interval period"
+ " or failures can be missed across sparse checks"
+ " e.g., every 120m interval with a 60m window rarely accumulates enough failed events"
+ " Unit could be defined with postfix (ns,ms,s,m,h,d)."
)
private Duration diskCheckSlidingWindowTimeout = DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the recommendation of value relationship between this new property and PERIODIC_DISK_CHECK_INTERVAL_MINUTES_DEFAULT? Say if user reconfigured PERIODIC_DISK_CHECK_INTERVAL_MINUTES_DEFAULT to 2h, or 30m, shall we suggest user to reconfigure this property too?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a good question.

The period disk check currently runs every 1 hour and the sliding window coverage is also for 1 hour.

The window should likely cover the period between and inclusive of the two periodic checks such that if two periodic checks fail, then they get counted in the same window.

Otherwise the only opportunity for a failure will be by a combination of periodic and on-demand checks and never due to two periodic checks.

I have updated the sliding window to be as long as the periodic disk check interval plus the timeout value for the disk check.


@Config(key = "hdds.datanode.chunk.data.validation.check",
defaultValue = "false",
type = ConfigType.BOOLEAN,
Expand Down Expand Up @@ -688,6 +706,23 @@ public void validate() {
diskCheckTimeout = DISK_CHECK_TIMEOUT_DEFAULT;
}

if (diskCheckSlidingWindowTimeout.isNegative()) {
LOG.warn("{} must be greater than zero and was set to {}. Defaulting to {}",
DISK_CHECK_SLIDING_WINDOW_TIMEOUT_KEY, diskCheckSlidingWindowTimeout,
DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT);
diskCheckSlidingWindowTimeout = DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT;
}

// Do not set window timeout <= periodic disk check interval period, or failures can be missed across sparse checks
// e.g., every 120m interval with a 60m window rarely accumulates enough failed events
if (diskCheckSlidingWindowTimeout.compareTo(Duration.ofMinutes(periodicDiskCheckIntervalMinutes)) < 0) {
LOG.warn("{} must be greater than or equal to {} minutes and was set to {} minutes. Defaulting to {}",
DISK_CHECK_SLIDING_WINDOW_TIMEOUT_KEY, periodicDiskCheckIntervalMinutes,
diskCheckSlidingWindowTimeout.toMinutes(),
DurationFormatUtils.formatDurationHMS(DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT.toMillis()));
diskCheckSlidingWindowTimeout = DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT;
}

if (blockDeleteCommandWorkerInterval.isNegative()) {
LOG.warn(BLOCK_DELETE_COMMAND_WORKER_INTERVAL +
" must be greater than zero and was set to {}. Defaulting to {}",
Expand Down Expand Up @@ -907,6 +942,14 @@ public void setDiskCheckTimeout(Duration duration) {
diskCheckTimeout = duration;
}

public Duration getDiskCheckSlidingWindowTimeout() {
return diskCheckSlidingWindowTimeout;
}

public void setDiskCheckSlidingWindowTimeout(Duration duration) {
diskCheckSlidingWindowTimeout = duration;
}

public int getBlockDeleteThreads() {
return blockDeleteThreads;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,10 @@
import java.io.File;
import java.io.IOException;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import org.apache.commons.io.FileUtils;
Expand Down Expand Up @@ -110,11 +107,6 @@ public class HddsVolume extends StorageVolume {
private final AtomicBoolean dbLoaded = new AtomicBoolean(false);
private final AtomicBoolean dbLoadFailure = new AtomicBoolean(false);

private final int volumeTestCount;
private final int volumeTestFailureTolerance;
private AtomicInteger volumeTestFailureCount;
private Queue<Boolean> volumeTestResultQueue;

/**
* Builder for HddsVolume.
*/
Expand Down Expand Up @@ -147,20 +139,13 @@ private HddsVolume(Builder b) throws IOException {
this.volumeInfoMetrics =
new VolumeInfoMetrics(b.getVolumeRootStr(), this);

this.volumeTestCount = getDatanodeConfig().getVolumeIOTestCount();
this.volumeTestFailureTolerance = getDatanodeConfig().getVolumeIOFailureTolerance();
this.volumeTestFailureCount = new AtomicInteger(0);
this.volumeTestResultQueue = new LinkedList<>();

initialize();
} else {
// Builder is called with failedVolume set, so create a failed volume
// HddsVolume Object.
this.setState(VolumeState.FAILED);
volumeIOStats = null;
volumeInfoMetrics = new VolumeInfoMetrics(b.getVolumeRootStr(), this);
this.volumeTestCount = 0;
this.volumeTestFailureTolerance = 0;
}

LOG.info("HddsVolume: {}", getReport());
Expand Down Expand Up @@ -322,38 +307,32 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused)

@VisibleForTesting
public VolumeCheckResult checkDbHealth(File dbFile) throws InterruptedException {
if (volumeTestCount == 0) {
if (getIoTestCount() == 0) {
return VolumeCheckResult.HEALTHY;
}

final boolean isVolumeTestResultHealthy = true;
try (ManagedOptions managedOptions = new ManagedOptions();
ManagedRocksDB ignored = ManagedRocksDB.openReadOnly(managedOptions, dbFile.toString())) {
volumeTestResultQueue.add(isVolumeTestResultHealthy);
// Do nothing. Only check if rocksdb is accessible.
LOG.debug("Successfully opened the database at \"{}\" for HDDS volume {}.", dbFile, getStorageDir());
} catch (Exception e) {
if (Thread.currentThread().isInterrupted()) {
throw new InterruptedException("Check of database for volume " + this + " interrupted.");
}
LOG.warn("Could not open Volume DB located at {}", dbFile, e);
volumeTestResultQueue.add(!isVolumeTestResultHealthy);
volumeTestFailureCount.incrementAndGet();
}

if (volumeTestResultQueue.size() > volumeTestCount
&& (Boolean.TRUE.equals(volumeTestResultQueue.poll()) != isVolumeTestResultHealthy)) {
volumeTestFailureCount.decrementAndGet();
getIoTestSlidingWindow().add();
}

if (volumeTestFailureCount.get() > volumeTestFailureTolerance) {
if (getIoTestSlidingWindow().isExceeded()) {
LOG.error("Failed to open the database at \"{}\" for HDDS volume {}: " +
"the last {} runs encountered {} out of {} tolerated failures.",
dbFile, this, volumeTestResultQueue.size(), volumeTestFailureCount.get(), volumeTestFailureTolerance);
"encountered more than the {} tolerated failures.",
dbFile, this, getIoTestSlidingWindow().getWindowSize());
return VolumeCheckResult.FAILED;
}

LOG.debug("Successfully opened the database at \"{}\" for HDDS volume {}: " +
"the last {} runs encountered {} out of {} tolerated failures",
dbFile, this, volumeTestResultQueue.size(), volumeTestFailureTolerance, volumeTestFailureTolerance);
"encountered {} out of {} tolerated failures",
dbFile, this, getIoTestSlidingWindow().getNumEventsInWindow(), getIoTestSlidingWindow().getWindowSize());
return VolumeCheckResult.HEALTHY;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,10 @@
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.LinkedList;
import java.time.Clock;
import java.util.Objects;
import java.util.Properties;
import java.util.Queue;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Stream;
Expand All @@ -43,6 +41,7 @@
import org.apache.hadoop.hdds.fs.SpaceUsageSource;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.server.ServerUtils;
import org.apache.hadoop.hdds.utils.SlidingWindow;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
Expand Down Expand Up @@ -111,9 +110,7 @@ public abstract class StorageVolume implements Checkable<Boolean, VolumeCheckRes
tests run, then the volume is considered failed.
*/
private final int ioTestCount;
private final int ioFailureTolerance;
private AtomicInteger currentIOFailureCount;
private Queue<Boolean> ioTestSlidingWindow;
private SlidingWindow ioTestSlidingWindow;
private int healthCheckFileSize;

/**
Expand Down Expand Up @@ -163,9 +160,8 @@ protected StorageVolume(Builder<?> b) throws IOException {
this.conf = b.conf;
this.dnConf = conf.getObject(DatanodeConfiguration.class);
this.ioTestCount = dnConf.getVolumeIOTestCount();
this.ioFailureTolerance = dnConf.getVolumeIOFailureTolerance();
this.ioTestSlidingWindow = new LinkedList<>();
this.currentIOFailureCount = new AtomicInteger(0);
this.ioTestSlidingWindow = new SlidingWindow(dnConf.getVolumeIOFailureTolerance(),
dnConf.getDiskCheckSlidingWindowTimeout(), b.getClock());
this.healthCheckFileSize = dnConf.getVolumeHealthCheckFileSize();
} else {
storageDir = new File(b.volumeRootStr);
Expand All @@ -174,7 +170,6 @@ protected StorageVolume(Builder<?> b) throws IOException {
this.storageID = UUID.randomUUID().toString();
this.state = VolumeState.FAILED;
this.ioTestCount = 0;
this.ioFailureTolerance = 0;
this.conf = null;
this.dnConf = null;
}
Expand Down Expand Up @@ -411,6 +406,7 @@ public abstract static class Builder<T extends Builder<T>> {
private boolean failedVolume = false;
private String datanodeUuid;
private String clusterID;
private Clock clock = new SlidingWindow.MonotonicClock();

public Builder(String volumeRootStr, String storageDirStr) {
this.volumeRootStr = volumeRootStr;
Expand Down Expand Up @@ -457,6 +453,11 @@ public T clusterID(String cid) {
return this.getThis();
}

public T clock(Clock c) {
this.clock = c;
return this.getThis();
}

public abstract StorageVolume build() throws IOException;

public String getVolumeRootStr() {
Expand All @@ -474,6 +475,10 @@ public StorageType getStorageType() {
public String getStorageDirStr() {
return this.storageDirStr;
}

public Clock getClock() {
return this.clock;
}
}

public String getVolumeRootDir() {
Expand Down Expand Up @@ -554,6 +559,14 @@ public VolumeSet getVolumeSet() {
return this.volumeSet;
}

public int getIoTestCount() {
return ioTestCount;
}

public SlidingWindow getIoTestSlidingWindow() {
return ioTestSlidingWindow;
}

public StorageType getStorageType() {
return storageType;
}
Expand Down Expand Up @@ -661,7 +674,7 @@ private void cleanTmpDiskCheckDir() {
* check consists of a directory check and an IO check.
*
* If the directory check fails, the volume check fails immediately.
* The IO check is allows to fail up to {@code ioFailureTolerance} times
* The IO check is allowed to fail up to {@code ioFailureTolerance} times
* out of the last {@code ioTestCount} IO checks before this volume check is
* failed. Each call to this method runs one IO check.
*
Expand Down Expand Up @@ -698,7 +711,6 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused)
// to avoid volume failure we can ignore checking disk read/write
int minimumDiskSpace = healthCheckFileSize * 2;
if (getCurrentUsage().getAvailable() < minimumDiskSpace) {
ioTestSlidingWindow.add(true);
return VolumeCheckResult.HEALTHY;
}

Expand All @@ -717,39 +729,25 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused)
// We can check again if disk is full. If it is full,
// in this case keep volume as healthy so that READ can still be served
if (!diskChecksPassed && getCurrentUsage().getAvailable() < minimumDiskSpace) {
ioTestSlidingWindow.add(true);
return VolumeCheckResult.HEALTHY;
}

// Move the sliding window of IO test results forward 1 by adding the
// latest entry and removing the oldest entry from the window.
// Update the failure counter for the new window.
ioTestSlidingWindow.add(diskChecksPassed);
if (!diskChecksPassed) {
currentIOFailureCount.incrementAndGet();
}
if (ioTestSlidingWindow.size() > ioTestCount &&
Objects.equals(ioTestSlidingWindow.poll(), Boolean.FALSE)) {
currentIOFailureCount.decrementAndGet();
ioTestSlidingWindow.add();
}

// If the failure threshold has been crossed, fail the volume without
// further scans.
// If the failure threshold has been crossed, fail the volume without further scans.
// Once the volume is failed, it will not be checked anymore.
// The failure counts can be left as is.
if (currentIOFailureCount.get() > ioFailureTolerance) {
LOG.error("Failed IO test for volume {}: the last {} runs " +
"encountered {} out of {} tolerated failures.", this,
ioTestSlidingWindow.size(), currentIOFailureCount,
ioFailureTolerance);
if (ioTestSlidingWindow.isExceeded()) {
LOG.error("Failed IO test for volume {}: encountered more than the {} tolerated failures within the past {} ms.",
this, ioTestSlidingWindow.getWindowSize(), ioTestSlidingWindow.getExpiryDurationMillis());
return VolumeCheckResult.FAILED;
} else if (LOG.isDebugEnabled()) {
LOG.debug("IO test results for volume {}: the last {} runs encountered " +
"{} out of {} tolerated failures", this,
ioTestSlidingWindow.size(),
currentIOFailureCount, ioFailureTolerance);
}

LOG.debug("IO test results for volume {}: encountered {} out of {} tolerated failures",
this, ioTestSlidingWindow.getNumEventsInWindow(), ioTestSlidingWindow.getWindowSize());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we call getNumEvents() here, instead of getNumEventsInWindow()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

getNumEventsInWindow is correct here. It will show how many failures are in the actual window. The window size is the number of failures that are allowed.

This log line is logged every time in debug mode. Since our queue is larger than the window, logging getNumEvents may give us the false impression that we got more failures as it will include the errors outside the window as well.

We are using getNumEvents only for logging in tests.


return VolumeCheckResult.HEALTHY;
}

Expand Down
Loading