diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java index 41f6d36971ff..482dcb6bff5d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java @@ -61,7 +61,6 @@ 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"; - // Minimum space should be left on volume. // Ex: If volume has 1000GB and minFreeSpace is configured as 10GB, // In this case when availableSpace is 10GB or below, volume is assumed as full diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java index b39468318311..728f5b64ccfd 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java @@ -708,23 +708,11 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) 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(); - } - - // 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) { + // Move the sliding window of IO test results forward 1 and check threshold. + if (advanceIOWindow(diskChecksPassed)) { + // 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. LOG.error("Failed IO test for volume {}: the last {} runs " + "encountered {} out of {} tolerated failures.", this, ioTestSlidingWindow.size(), currentIOFailureCount, @@ -740,6 +728,65 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.HEALTHY; } + /** + * Called by {@link StorageVolumeChecker} when a volume check times out — + * either because the global {@code checkAllVolumes()} latch expired before + * this volume's async check completed, or because the per-check timeout + * inside {@link ThrottledAsyncChecker} fired. + * + *
Records a synthetic IO-test failure in the existing sliding window, + * making latch timeouts subject to the same {@code ioFailureTolerance} + * threshold as genuine read/write failures. No separate configuration key + * is required: the existing + * {@code hdds.datanode.disk.check.io.failures.tolerated} governs both. + * + *
Recovery is automatic: each successful {@link #check} call records a + * {@code true} entry in the window, gradually evicting the synthetic + * failure once {@code ioTestCount} healthy results have accumulated. + * + * @return {@code true} if {@code currentIOFailureCount > ioFailureTolerance}, + * meaning the volume should now be marked FAILED; {@code false} if + * the failure is still within tolerance this round. + */ + public synchronized boolean recordTimeoutAsIOFailure() { + if (advanceIOWindow(false)) { + LOG.error("Volume {} check timed out: IO-failure count ({}) exceeds" + + " tolerance ({}). Marking FAILED.", + this, currentIOFailureCount, ioFailureTolerance); + return true; + } + LOG.warn("Volume {} check timed out. IO-failure count: {} / tolerance: {}." + + " Volume will not be failed until tolerance is exceeded." + + " Common transient causes: kernel I/O scheduler saturation" + + " or JVM GC pressure.", + this, currentIOFailureCount, ioFailureTolerance); + return false; + } + + /** + * Advances the IO-test sliding window by one entry and updates the rolling + * failure counter. + * + *
Called by both {@link #check} (genuine IO test result) and
+ * {@link #recordTimeoutAsIOFailure} (synthetic failure for a check timeout),
+ * keeping the window-update logic in a single place.
+ *
+ * @param passed {@code true} if the IO test passed; {@code false} otherwise.
+ * @return {@code true} if {@code currentIOFailureCount} now exceeds
+ * {@code ioFailureTolerance}; {@code false} if still within bounds.
+ */
+ private boolean advanceIOWindow(boolean passed) {
+ ioTestSlidingWindow.add(passed);
+ if (!passed) {
+ currentIOFailureCount.incrementAndGet();
+ }
+ if (ioTestSlidingWindow.size() > ioTestCount &&
+ Objects.equals(ioTestSlidingWindow.poll(), Boolean.FALSE)) {
+ currentIOFailureCount.decrementAndGet();
+ }
+ return currentIOFailureCount.get() > ioFailureTolerance;
+ }
+
@Override
public int hashCode() {
return Objects.hash(storageDir);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeChecker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeChecker.java
index b48b0dac1180..b3384514d253 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeChecker.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolumeChecker.java
@@ -42,6 +42,7 @@
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -246,18 +247,40 @@ public Set extends StorageVolume> checkAllVolumes(
// Wait until our timeout elapses, after which we give up on
// the remaining volumes.
- if (!latch.await(maxAllowedTimeForCheckMs, TimeUnit.MILLISECONDS)) {
- LOG.warn("checkAllVolumes timed out after {} ms",
- maxAllowedTimeForCheckMs);
- }
+ boolean completedOnTime =
+ latch.await(maxAllowedTimeForCheckMs, TimeUnit.MILLISECONDS);
synchronized (this) {
- // All volumes that have not been detected as healthy should be
- // considered failed. This is a superset of 'failedVolumes'.
- //
- // Make a copy under the mutex as Sets.difference() returns a view
- // of a potentially changing set.
- return new HashSet<>(Sets.difference(allVolumes, healthyVolumes));
+ if (!completedOnTime) {
+ LOG.warn("checkAllVolumes timed out after {} ms."
+ + " Evaluating per-volume latch-timeout tolerance.",
+ maxAllowedTimeForCheckMs);
+ }
+
+ // Volumes that explicitly reported FAILED via check() are always
+ // returned — the IO-failure sliding window in StorageVolume.check()
+ // already applied its own tolerance.
+ final Set With the defaults (ioTestCount=3, ioFailureTolerance=1):
+ *
+ *
+ */
+ @ParameterizedTest
+ @MethodSource("volumeBuilders")
+ public void testHealthyChecksEvictTimeoutFromSlidingWindow(
+ StorageVolume.Builder> builder) throws Exception {
+ StorageVolume volume = builder.build();
+ volume.format(CLUSTER_ID);
+ volume.createTmpDirs(CLUSTER_ID);
+
+ // Simulate one tolerated timeout.
+ assertFalse(volume.recordTimeoutAsIOFailure(),
+ "First timeout should be tolerated");
+
+ // Three healthy checks push TRUE entries into the sliding window,
+ // eventually evicting the synthetic FALSE.
+ DiskCheckUtil.DiskChecks alwaysPass = new DiskCheckUtil.DiskChecks() {
+ @Override
+ public boolean checkReadWrite(File storageDir, File testFileDir,
+ int numBytesToWrite) {
+ return true;
+ }
+ };
+ DiskCheckUtil.setTestImpl(alwaysPass);
+ assertEquals(VolumeCheckResult.HEALTHY, volume.check(false));
+ assertEquals(VolumeCheckResult.HEALTHY, volume.check(false));
+ assertEquals(VolumeCheckResult.HEALTHY, volume.check(false));
+
+ // After recovery a new single timeout is tolerated again.
+ assertFalse(volume.recordTimeoutAsIOFailure(),
+ "Timeout after recovery should be tolerated again");
+ }
+
/**
* Asserts that the disk checks are being done on the correct directory for
* each volume type.