-
Notifications
You must be signed in to change notification settings - Fork 597
HDDS-14871. DataNode: tolerate per-volume health-check latch timeouts before marking volumes failed. #9954
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
HDDS-14871. DataNode: tolerate per-volume health-check latch timeouts before marking volumes failed. #9954
Changes from all commits
c702794
d91cf3d
de0a027
16bdb26
452cd11
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -113,6 +113,15 @@ public abstract class StorageVolume implements Checkable<Boolean, VolumeCheckRes | |
| private Queue<Boolean> ioTestSlidingWindow; | ||
| private int healthCheckFileSize; | ||
|
|
||
| /* | ||
| Counter for consecutive latch or per-check timeouts. Incremented by | ||
| recordTimeoutAsIOFailure() which must NOT be synchronized (check() may be | ||
| holding the lock — that is exactly why the timeout fired). AtomicInteger | ||
| provides the necessary thread safety without locking. Reset to 0 by | ||
| resetTimeoutCount() whenever a check completes successfully. | ||
| */ | ||
| private final AtomicInteger consecutiveTimeoutCount; | ||
|
|
||
| /** | ||
| * Type for StorageVolume. | ||
| */ | ||
|
|
@@ -164,6 +173,7 @@ protected StorageVolume(Builder<?> b) throws IOException { | |
| this.ioTestSlidingWindow = new LinkedList<>(); | ||
| this.currentIOFailureCount = new AtomicInteger(0); | ||
| this.healthCheckFileSize = dnConf.getVolumeHealthCheckFileSize(); | ||
| this.consecutiveTimeoutCount = new AtomicInteger(0); | ||
| } else { | ||
| storageDir = new File(b.volumeRootStr); | ||
| volumeUsage = null; | ||
|
|
@@ -174,6 +184,7 @@ protected StorageVolume(Builder<?> b) throws IOException { | |
| this.ioFailureTolerance = 0; | ||
| this.conf = null; | ||
| this.dnConf = null; | ||
| this.consecutiveTimeoutCount = new AtomicInteger(0); | ||
| } | ||
| this.storageDirStr = storageDir.getAbsolutePath(); | ||
| } | ||
|
|
@@ -708,23 +719,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 +739,92 @@ 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. | ||
| * | ||
| * <p><b>Must not be {@code synchronized}.</b> When a timeout fires, | ||
| * {@link #check} may still be executing and holding the object lock — that | ||
| * is precisely why the timeout occurred. Acquiring the same lock here would | ||
| * deadlock or stall {@link StorageVolumeChecker#checkAllVolumes} until the | ||
| * hung check finally returns. | ||
| * | ||
| * <p>Instead, a dedicated {@link AtomicInteger} ({@code | ||
| * consecutiveTimeoutCount}) tracks consecutive timeouts without any locking. | ||
| * The threshold reuses the existing {@code ioFailureTolerance} so no | ||
| * additional configuration key is required. | ||
| * | ||
| * <p>Recovery: call {@link #resetTimeoutCount()} when a check completes | ||
| * successfully to break the timeout streak. | ||
| * | ||
| * @return {@code true} if {@code consecutiveTimeoutCount > ioFailureTolerance}, | ||
| * meaning the volume should now be marked FAILED; {@code false} if | ||
| * the timeout is still within tolerance this round. | ||
| */ | ||
| public boolean recordTimeoutAsIOFailure() { | ||
| int count = consecutiveTimeoutCount.incrementAndGet(); | ||
| if (count > ioFailureTolerance) { | ||
| LOG.error("Volume {} check timed out {} consecutive time(s)," | ||
| + " exceeding tolerance of {}. Marking FAILED.", | ||
| this, count, ioFailureTolerance); | ||
| return true; | ||
| } | ||
| LOG.warn("Volume {} check timed out ({}/{} consecutive timeouts tolerated)." | ||
| + " Common transient causes: kernel I/O scheduler saturation" | ||
| + " or JVM GC pressure. Volume will be failed if the next check" | ||
| + " also times out.", | ||
| this, count, ioFailureTolerance); | ||
| return false; | ||
| } | ||
|
|
||
| /** | ||
| * Resets the consecutive-timeout counter to 0. | ||
| * | ||
| * <p>Called by {@link StorageVolumeChecker} when this volume's check | ||
| * completes successfully, indicating that the transient stall has resolved | ||
| * and any accumulated timeout count should not carry over to the next cycle. | ||
| * | ||
| * <p>No synchronization needed — operates on an {@link AtomicInteger}. | ||
| */ | ||
| public void resetTimeoutCount() { | ||
| int prev = consecutiveTimeoutCount.getAndSet(0); | ||
| if (prev > 0 && LOG.isDebugEnabled()) { | ||
| LOG.debug("Volume {} completed a healthy check. Consecutive timeout" | ||
| + " count reset from {} to 0.", this, prev); | ||
| } | ||
| } | ||
|
Comment on lines
+791
to
+797
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We are using This can also be modeled using a Sliding Window similar to what we do for tracking volume check failures. We can create a new sliding window which keeps track of the timeouts with a max toleration of 1. If we use the new SlidingWindow.java implementation, we will also not have to worry about resetting the count as the time based policy will automatically take care of it. The time validity of the window can be 70 minutes, sufficient for two |
||
|
|
||
| @VisibleForTesting | ||
| public int getConsecutiveTimeoutCount() { | ||
| return consecutiveTimeoutCount.get(); | ||
| } | ||
|
|
||
| /** | ||
| * Advances the IO-test sliding window by one entry and updates the rolling | ||
| * failure counter. | ||
| * | ||
| * <p>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); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we remove all changes not related to
consecutiveTimeoutCount.These changes conflict with the PR #8843 which transitions the
StorageVolumeclass to use the newSlidingWindowimplementation.