-
Notifications
You must be signed in to change notification settings - Fork 598
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
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 |
|---|---|---|
|
|
@@ -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. | ||
| * | ||
| * <p>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. | ||
| * | ||
| * <p>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() { | ||
|
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. check is synchronized. So if timeout, check doesn't return, then this recordTimeoutAsIOFailure will wait for check to complete. |
||
| 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. | ||
| * | ||
| * <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); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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) { | ||
devmadhuu marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 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<StorageVolume> result = new HashSet<>(failedVolumes); | ||
|
|
||
| // Volumes still pending (neither healthy nor explicitly failed): | ||
| // the latch expired before they reported a result. Record a synthetic | ||
| // IO failure in each volume's existing sliding window so latch timeouts | ||
| // share the same ioFailureTolerance threshold as genuine IO failures. | ||
| // Healthy volumes need no special action: their successful check() call | ||
| // already recorded TRUE in the sliding window. | ||
| final Set<StorageVolume> pendingVolumes = | ||
| new HashSet<>(Sets.difference(allVolumes, | ||
| Sets.union(healthyVolumes, failedVolumes))); | ||
|
|
||
| for (StorageVolume v : pendingVolumes) { | ||
| if (v.recordTimeoutAsIOFailure()) { | ||
| // Tolerance exceeded — mark as failed. | ||
| result.add(v); | ||
| } | ||
| // else: within tolerance this round — omit from failed set. | ||
| } | ||
|
|
||
| return result; | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -376,10 +399,22 @@ public void onFailure(@Nonnull Throwable t) { | |
| volume, exception); | ||
| // If the scan was interrupted, do not count it as a volume failure. | ||
| // This should only happen if the volume checker is being shut down. | ||
| if (!(t instanceof InterruptedException)) { | ||
| markFailed(); | ||
| cleanup(); | ||
| if (t instanceof InterruptedException) { | ||
| return; | ||
| } | ||
| if (exception instanceof TimeoutException) { | ||
|
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. Devesh, can you add a unit test of real timeout case for both check all volume and check one volume? The exception thrown out from ListenableFuture is not direct TimeoutException in my test. |
||
| // Per-check timeout from ThrottledAsyncChecker: apply the same | ||
| // IO-failure tolerance as a failed read/write test, rather than | ||
| // failing the volume immediately on the first timeout. | ||
| if (!volume.recordTimeoutAsIOFailure()) { | ||
| // Within tolerance this round. Still call cleanup() so numVolumes | ||
| // decrements correctly and the latch/callback fires on time. | ||
| cleanup(); | ||
|
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. one of the callback of checkVolume is We need to consider whether call the cleanup in this case. |
||
| return; | ||
| } | ||
| } | ||
| markFailed(); | ||
| cleanup(); | ||
| } | ||
|
|
||
| private void markHealthy() { | ||
|
|
||
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.
Please revert this change.