Skip to content
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
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,10 @@ private long getCurrentTime() {
return clock.millis();
}

public long getExpiryDurationMillis() {
return expiryDurationMillis;
}

/**
* A custom monotonic clock implementation.
* Implementation of Clock that uses System.nanoTime() for real usage.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,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 +100,8 @@ 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(60);

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 +407,16 @@ public class DatanodeConfiguration extends ReconfigurableConfig {
)
private Duration diskCheckTimeout = DISK_CHECK_TIMEOUT_DEFAULT;

@Config(key = "disk.check.sliding.window.timeout",
defaultValue = "60m",
type = ConfigType.TIME,
tags = {ConfigTag.DATANODE},
description = "Time interval after which a disk check"
+ " failure result stored in the sliding window will expire."
+ " Unit could be defined with postfix (ns,ms,s,m,h,d)."
)
private Duration diskCheckSlidingWindowTimeout = DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT;

@Config(key = "chunk.data.validation.check",
defaultValue = "false",
type = ConfigType.BOOLEAN,
Expand Down Expand Up @@ -688,6 +701,13 @@ 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;
}

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 +927,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 @@ -25,12 +25,9 @@
import jakarta.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
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 @@ -106,11 +103,6 @@ public class HddsVolume extends StorageVolume {
private 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 @@ -143,20 +135,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 @@ -318,38 +303,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 readOnlyDb = 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 @@ -26,13 +26,10 @@
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.LinkedList;
import java.util.Objects;
import java.util.Optional;
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 @@ -41,6 +38,7 @@
import org.apache.hadoop.hdds.fs.SpaceUsageCheckFactory;
import org.apache.hadoop.hdds.fs.SpaceUsageCheckParams;
import org.apache.hadoop.hdds.fs.SpaceUsageSource;
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 @@ -109,9 +107,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 @@ -161,9 +157,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());
this.healthCheckFileSize = dnConf.getVolumeHealthCheckFileSize();
} else {
storageDir = new File(b.volumeRootStr);
Expand All @@ -172,7 +167,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 @@ -538,6 +532,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 @@ -635,7 +637,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 @@ -672,7 +674,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 @@ -691,39 +692,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());

return VolumeCheckResult.HEALTHY;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,16 +21,20 @@
import static org.junit.jupiter.api.Assertions.assertEquals;

import java.io.File;
import java.lang.reflect.Field;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.time.Duration;
import java.util.UUID;
import java.util.stream.Stream;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.fs.MockSpaceUsageCheckFactory;
import org.apache.hadoop.hdds.utils.SlidingWindow;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
import org.apache.hadoop.ozone.container.common.utils.DiskCheckUtil;
import org.apache.ozone.test.TestClock;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Named;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -304,13 +308,23 @@ private void testCheckIOUntilFailure(StorageVolume.Builder<?> builder,
DatanodeConfiguration dnConf = CONF.getObject(DatanodeConfiguration.class);
dnConf.setVolumeIOTestCount(ioTestCount);
dnConf.setVolumeIOFailureTolerance(ioFailureTolerance);
dnConf.setDiskCheckSlidingWindowTimeout(Duration.ofMillis(ioTestCount));
CONF.setFromObject(dnConf);
builder.conf(CONF);
StorageVolume volume = builder.build();
volume.format(CLUSTER_ID);
volume.createTmpDirs(CLUSTER_ID);
// Sliding window protocol transitioned from count-based to a time-based system
// Update the default failure duration of the window from 60 minutes to a shorter duration for the test
long eventRate = 1L;
TestClock testClock = TestClock.newInstance();
Field clock = SlidingWindow.class.getDeclaredField("clock");
clock.setAccessible(true);
clock.set(volume.getIoTestSlidingWindow(), testClock);

for (int i = 0; i < checkResults.length; i++) {
// Sleep to allow entries in the sliding window to eventually timeout
testClock.fastForward(eventRate);
final boolean result = checkResults[i];
final DiskCheckUtil.DiskChecks ioResult = new DiskCheckUtil.DiskChecks() {
@Override
Expand Down