Skip to content

Commit 7eaf46f

Browse files
authored
HDDS-14871. DataNode: tolerate per-volume health-check latch timeouts before marking volumes failed. (#9954)
1 parent b3b4f32 commit 7eaf46f

5 files changed

Lines changed: 514 additions & 24 deletions

File tree

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java

Lines changed: 101 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,12 @@ public class DatanodeConfiguration extends ReconfigurableConfig {
6363
public static final String DISK_CHECK_MIN_GAP_KEY = "hdds.datanode.disk.check.min.gap";
6464
public static final String DISK_CHECK_TIMEOUT_KEY = "hdds.datanode.disk.check.timeout";
6565
public static final String DISK_CHECK_SLIDING_WINDOW_TIMEOUT_KEY = "hdds.datanode.disk.check.sliding.window.timeout";
66+
public static final String DISK_CHECK_TIMEOUT_TEST_ENABLED_KEY =
67+
"hdds.datanode.disk.check.timeout.test.enabled";
68+
public static final String DISK_CHECK_TIMEOUT_FAILURES_TOLERATED_KEY =
69+
"hdds.datanode.disk.check.timeout.failures.tolerated";
70+
public static final String DISK_CHECK_TIMEOUT_SLIDING_WINDOW_TIMEOUT_KEY =
71+
"hdds.datanode.disk.check.timeout.sliding.window.timeout";
6672

6773
// Minimum space should be left on volume.
6874
// Ex: If volume has 1000GB and minFreeSpace is configured as 10GB,
@@ -103,6 +109,11 @@ public class DatanodeConfiguration extends ReconfigurableConfig {
103109

104110
static final Duration DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT =
105111
Duration.ofMinutes(PERIODIC_DISK_CHECK_INTERVAL_MINUTES_DEFAULT).plus(DISK_CHECK_TIMEOUT_DEFAULT);
112+
static final boolean DISK_CHECK_TIMEOUT_TEST_ENABLED_DEFAULT = true;
113+
static final int DISK_CHECK_TIMEOUT_FAILURES_TOLERATED_DEFAULT = 1;
114+
static final Duration DISK_CHECK_TIMEOUT_SLIDING_WINDOW_TIMEOUT_DEFAULT =
115+
Duration.ofMinutes(PERIODIC_DISK_CHECK_INTERVAL_MINUTES_DEFAULT)
116+
.plus(DISK_CHECK_TIMEOUT_DEFAULT);
106117

107118
static final boolean CONTAINER_SCHEMA_V3_ENABLED_DEFAULT = true;
108119
static final long ROCKSDB_LOG_MAX_FILE_SIZE_BYTES_DEFAULT = 32 * 1024 * 1024;
@@ -430,6 +441,39 @@ public class DatanodeConfiguration extends ReconfigurableConfig {
430441
)
431442
private Duration diskCheckSlidingWindowTimeout = DISK_CHECK_SLIDING_WINDOW_TIMEOUT_DEFAULT;
432443

444+
@Config(key = "hdds.datanode.disk.check.timeout.test.enabled",
445+
defaultValue = "true",
446+
type = ConfigType.BOOLEAN,
447+
tags = { DATANODE },
448+
description = "Enable or disable timeout-based volume failure tracking "
449+
+ "independently of the disk IO health checks."
450+
)
451+
private boolean isDiskCheckTimeoutTestEnabled =
452+
DISK_CHECK_TIMEOUT_TEST_ENABLED_DEFAULT;
453+
454+
@Config(key = "hdds.datanode.disk.check.timeout.failures.tolerated",
455+
defaultValue = "1",
456+
type = ConfigType.INT,
457+
tags = { DATANODE },
458+
description = "The number of volume check timeouts allowed within the "
459+
+ "timeout sliding window before the volume is marked as failed."
460+
)
461+
private int diskCheckTimeoutFailureTolerance =
462+
DISK_CHECK_TIMEOUT_FAILURES_TOLERATED_DEFAULT;
463+
464+
@Config(key = "hdds.datanode.disk.check.timeout.sliding.window.timeout",
465+
defaultValue = "90m",
466+
type = ConfigType.TIME,
467+
tags = { DATANODE },
468+
description = "Time interval after which a timed out volume check "
469+
+ "event stored in the timeout sliding window will expire. "
470+
+ "Do not set this below the periodic disk check interval or "
471+
+ "timeout events may not accumulate across sparse checks. "
472+
+ "Unit could be defined with postfix (ns,ms,s,m,h,d)."
473+
)
474+
private Duration diskCheckTimeoutSlidingWindowTimeout =
475+
DISK_CHECK_TIMEOUT_SLIDING_WINDOW_TIMEOUT_DEFAULT;
476+
433477
@Config(key = "hdds.datanode.chunk.data.validation.check",
434478
defaultValue = "false",
435479
type = ConfigType.BOOLEAN,
@@ -676,6 +720,17 @@ public void validate() {
676720
}
677721
}
678722

723+
if (!isDiskCheckTimeoutTestEnabled) {
724+
LOG.info("Disk check timeout tracking has been disabled.");
725+
} else if (diskCheckTimeoutFailureTolerance < 0) {
726+
LOG.warn("{} must be greater than or equal to 0 but was set to {}. Defaulting to {}",
727+
DISK_CHECK_TIMEOUT_FAILURES_TOLERATED_KEY,
728+
diskCheckTimeoutFailureTolerance,
729+
DISK_CHECK_TIMEOUT_FAILURES_TOLERATED_DEFAULT);
730+
diskCheckTimeoutFailureTolerance =
731+
DISK_CHECK_TIMEOUT_FAILURES_TOLERATED_DEFAULT;
732+
}
733+
679734
if (diskCheckMinGap.isNegative()) {
680735
LOG.warn(DISK_CHECK_MIN_GAP_KEY +
681736
" must be greater than zero and was set to {}. Defaulting to {}",
@@ -709,6 +764,28 @@ public void validate() {
709764
diskCheckSlidingWindowTimeout = defaultTimeout;
710765
}
711766

767+
if (diskCheckTimeoutSlidingWindowTimeout.isNegative()) {
768+
Duration defaultTimeout =
769+
Duration.ofMinutes(periodicDiskCheckIntervalMinutes).plus(diskCheckTimeout);
770+
LOG.warn("{} must be greater than zero and was set to {}. Defaulting to {}",
771+
DISK_CHECK_TIMEOUT_SLIDING_WINDOW_TIMEOUT_KEY,
772+
diskCheckTimeoutSlidingWindowTimeout,
773+
defaultTimeout);
774+
diskCheckTimeoutSlidingWindowTimeout = defaultTimeout;
775+
}
776+
777+
if (diskCheckTimeoutSlidingWindowTimeout
778+
.compareTo(Duration.ofMinutes(periodicDiskCheckIntervalMinutes)) < 0) {
779+
Duration defaultTimeout =
780+
Duration.ofMinutes(periodicDiskCheckIntervalMinutes).plus(diskCheckTimeout);
781+
LOG.warn("{} must be greater than or equal to {} minutes and was set to {} minutes. Defaulting to {}",
782+
DISK_CHECK_TIMEOUT_SLIDING_WINDOW_TIMEOUT_KEY,
783+
periodicDiskCheckIntervalMinutes,
784+
diskCheckTimeoutSlidingWindowTimeout.toMinutes(),
785+
DurationFormatUtils.formatDurationHMS(defaultTimeout.toMillis()));
786+
diskCheckTimeoutSlidingWindowTimeout = defaultTimeout;
787+
}
788+
712789
if (blockDeleteCommandWorkerInterval.isNegative()) {
713790
LOG.warn(BLOCK_DELETE_COMMAND_WORKER_INTERVAL +
714791
" must be greater than zero and was set to {}. Defaulting to {}",
@@ -944,6 +1021,30 @@ public void setDiskCheckSlidingWindowTimeout(Duration duration) {
9441021
diskCheckSlidingWindowTimeout = duration;
9451022
}
9461023

1024+
public boolean isDiskCheckTimeoutTestEnabled() {
1025+
return isDiskCheckTimeoutTestEnabled;
1026+
}
1027+
1028+
public void setDiskCheckTimeoutTestEnabled(boolean enabled) {
1029+
isDiskCheckTimeoutTestEnabled = enabled;
1030+
}
1031+
1032+
public int getDiskCheckTimeoutFailureTolerance() {
1033+
return diskCheckTimeoutFailureTolerance;
1034+
}
1035+
1036+
public void setDiskCheckTimeoutFailureTolerance(int tolerance) {
1037+
diskCheckTimeoutFailureTolerance = tolerance;
1038+
}
1039+
1040+
public Duration getDiskCheckTimeoutSlidingWindowTimeout() {
1041+
return diskCheckTimeoutSlidingWindowTimeout;
1042+
}
1043+
1044+
public void setDiskCheckTimeoutSlidingWindowTimeout(Duration duration) {
1045+
diskCheckTimeoutSlidingWindowTimeout = duration;
1046+
}
1047+
9471048
public int getBlockDeleteThreads() {
9481049
return blockDeleteThreads;
9491050
}

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java

Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -110,7 +110,9 @@ public abstract class StorageVolume implements Checkable<Boolean, VolumeCheckRes
110110
tests run, then the volume is considered failed.
111111
*/
112112
private final boolean isDiskCheckEnabled;
113+
private final boolean isTimeoutCheckEnabled;
113114
private SlidingWindow ioTestSlidingWindow;
115+
private SlidingWindow timeoutFailureSlidingWindow;
114116
private int healthCheckFileSize;
115117

116118
/**
@@ -160,8 +162,12 @@ protected StorageVolume(Builder<?> b) throws IOException {
160162
this.conf = b.conf;
161163
this.dnConf = conf.getObject(DatanodeConfiguration.class);
162164
this.isDiskCheckEnabled = dnConf.isDiskCheckEnabled();
165+
this.isTimeoutCheckEnabled = dnConf.isDiskCheckTimeoutTestEnabled();
163166
this.ioTestSlidingWindow = new SlidingWindow(dnConf.getVolumeIOFailureTolerance(),
164167
dnConf.getDiskCheckSlidingWindowTimeout(), b.getClock());
168+
this.timeoutFailureSlidingWindow = new SlidingWindow(
169+
dnConf.getDiskCheckTimeoutFailureTolerance(),
170+
dnConf.getDiskCheckTimeoutSlidingWindowTimeout(), b.getClock());
165171
this.healthCheckFileSize = dnConf.getVolumeHealthCheckFileSize();
166172
} else {
167173
storageDir = new File(b.volumeRootStr);
@@ -170,6 +176,7 @@ protected StorageVolume(Builder<?> b) throws IOException {
170176
this.storageID = UUID.randomUUID().toString();
171177
this.state = VolumeState.FAILED;
172178
this.isDiskCheckEnabled = false;
179+
this.isTimeoutCheckEnabled = false;
173180
this.conf = null;
174181
this.dnConf = null;
175182
}
@@ -567,6 +574,11 @@ public SlidingWindow getIoTestSlidingWindow() {
567574
return ioTestSlidingWindow;
568575
}
569576

577+
@VisibleForTesting
578+
public SlidingWindow getTimeoutFailureSlidingWindow() {
579+
return timeoutFailureSlidingWindow;
580+
}
581+
570582
public StorageType getStorageType() {
571583
return storageType;
572584
}
@@ -752,6 +764,38 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused)
752764
return VolumeCheckResult.HEALTHY;
753765
}
754766

767+
/**
768+
* Records a volume-check timeout in the timeout failure window.
769+
*
770+
* <p>This is intentionally separate from the normal IO failure window.
771+
* Timeouts are tracked as "more than N timeout events within the timeout
772+
* window" rather than as a consecutive counter. The time-based expiry
773+
* automatically removes old timeout events.
774+
*
775+
* @return {@code true} if the number of timeout events in the timeout window
776+
* now exceeds the tolerated threshold and the volume should be
777+
* marked failed; {@code false} otherwise.
778+
*/
779+
public boolean recordTimeoutAndCheckFailure() {
780+
if (!isTimeoutCheckEnabled) {
781+
return false;
782+
}
783+
timeoutFailureSlidingWindow.add();
784+
if (timeoutFailureSlidingWindow.isExceeded()) {
785+
LOG.error("Volume {} check timed out more than the {} tolerated times "
786+
+ "within the past {} ms. Marking FAILED.",
787+
this, timeoutFailureSlidingWindow.getWindowSize(),
788+
timeoutFailureSlidingWindow.getExpiryDurationMillis());
789+
return true;
790+
}
791+
LOG.warn("Volume {} check timed out. Encountered {} out of {} tolerated "
792+
+ "timeouts within the past {} ms.",
793+
this, timeoutFailureSlidingWindow.getNumEventsInWindow(),
794+
timeoutFailureSlidingWindow.getWindowSize(),
795+
timeoutFailureSlidingWindow.getExpiryDurationMillis());
796+
return false;
797+
}
798+
755799
@Override
756800
public int hashCode() {
757801
return Objects.hash(storageDir);

0 commit comments

Comments
 (0)