Skip to content
Open
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 @@ -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.
*/
Expand Down Expand Up @@ -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;
Expand All @@ -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();
}
Expand Down Expand Up @@ -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.
Comment on lines +722 to +726
Copy link
Copy Markdown
Contributor

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 StorageVolume class to use the new SlidingWindow implementation.

LOG.error("Failed IO test for volume {}: the last {} runs " +
"encountered {} out of {} tolerated failures.", this,
ioTestSlidingWindow.size(), currentIOFailureCount,
Expand All @@ -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
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are using AtomicInteger consecutiveTimeoutCount to essentially fail if we see two consecutive failures.

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 checkAllVolumes to complete.


@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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -224,6 +225,13 @@ public Set<? extends StorageVolume> checkAllVolumes(
final AtomicLong numVolumes = new AtomicLong(volumes.size());
final CountDownLatch latch = new CountDownLatch(1);

// Shared set used to guarantee exactly-one call to
// recordTimeoutAsIOFailure() per volume, regardless of whether the
// per-check timeout (ResultHandler.onFailure) or the global latch timeout
// (pending-volumes loop below) fires first. The first path to CAS-add the
// volume owns the tolerance decision; the other path skips it.
final Set<StorageVolume> timeoutHandledSet = ConcurrentHashMap.newKeySet();

for (StorageVolume v : volumes) {
Optional<ListenableFuture<VolumeCheckResult>> olf =
delegateChecker.schedule(v, null);
Expand All @@ -232,7 +240,8 @@ public Set<? extends StorageVolume> checkAllVolumes(
allVolumes.add(v);
Futures.addCallback(olf.get(),
new ResultHandler(v, healthyVolumes, failedVolumes,
numVolumes, (ignored1, ignored2) -> latch.countDown()),
numVolumes, (ignored1, ignored2) -> latch.countDown(),
timeoutHandledSet),
MoreExecutors.directExecutor());
} else {
if (v instanceof HddsVolume) {
Expand All @@ -246,18 +255,51 @@ 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<StorageVolume> result = new HashSet<>(failedVolumes);

// Volumes that completed healthy: reset their consecutive-timeout
// counter so a single transient timeout is not combined with an
// unrelated future one after a healthy gap.
for (StorageVolume v : healthyVolumes) {
v.resetTimeoutCount();
}

// Volumes still pending (neither healthy nor explicitly failed) at
// latch-timeout time. onFailure() may have already handled some of
// these via timeoutHandledSet; skip those to avoid double-counting.
final Set<StorageVolume> pendingVolumes =
new HashSet<>(Sets.difference(allVolumes,
Sets.union(healthyVolumes, failedVolumes)));

for (StorageVolume v : pendingVolumes) {
if (!timeoutHandledSet.add(v)) {
// onFailure() already handled this volume's timeout (per-check
// timeout fired before the latch). The tolerance decision was
// already made there; nothing left to do.
continue;
}
// Latch fired first — this is the first (and only) handler.
if (v.recordTimeoutAsIOFailure()) {
// Tolerance exceeded — mark as failed.
result.add(v);
}
// else: within tolerance this round — omit from failed set.
}

return result;
}
}

Expand Down Expand Up @@ -298,7 +340,7 @@ public boolean checkVolume(final StorageVolume volume, Callback callback) {
Futures.addCallback(olf.get(),
new ResultHandler(volume,
ConcurrentHashMap.newKeySet(), ConcurrentHashMap.newKeySet(),
new AtomicLong(1), callback),
new AtomicLong(1), callback, null),
checkVolumeResultHandlerExecutorService
);
return true;
Expand All @@ -320,23 +362,39 @@ private static class ResultHandler
private final Callback callback;

/**
* @param healthyVolumes set of healthy volumes. If the disk check is
* successful, add the volume here.
* @param failedVolumes set of failed volumes. If the disk check fails,
* add the volume here.
* @param volumeCounter volumeCounter used to trigger callback invocation.
* @param callback invoked when the volumeCounter reaches 0.
* Shared set used to guarantee exactly-one call to
* {@link StorageVolume#recordTimeoutAsIOFailure()} per volume when both
* the per-check timeout ({@link #onFailure}) and the global latch timeout
* (pending-volumes loop in {@code checkAllVolumes}) can race for the same
* volume.
* <p>
* {@code null} for the {@code checkVolume()} path, where no latch exists
* and {@link #onFailure} is the sole timeout handler.
*/
@Nullable
private final Set<StorageVolume> timeoutHandledSet;

/**
* @param healthyVolumes set of healthy volumes.
* @param failedVolumes set of failed volumes.
* @param volumeCounter triggers callback when it reaches 0.
* @param callback invoked when volumeCounter reaches 0.
* @param timeoutHandledSet shared CAS set for exactly-once timeout
* handling; {@code null} for
* {@code checkVolume()}.
*/
ResultHandler(StorageVolume volume,
Set<StorageVolume> healthyVolumes,
Set<StorageVolume> failedVolumes,
AtomicLong volumeCounter,
@Nullable Callback callback) {
@Nullable Callback callback,
@Nullable Set<StorageVolume> timeoutHandledSet) {
this.volume = volume;
this.healthyVolumes = healthyVolumes;
this.failedVolumes = failedVolumes;
this.volumeCounter = volumeCounter;
this.callback = callback;
this.timeoutHandledSet = timeoutHandledSet;
}

@Override
Expand Down Expand Up @@ -376,14 +434,41 @@ 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;
}
// Detect a per-check timeout from ThrottledAsyncChecker.
// Guava 28+ (including 33.5.0-jre used here) fails the TimeoutFuture
// with TimeoutException on timeout.
boolean isTimeout = exception instanceof TimeoutException;
if (isTimeout) {
// timeoutHandledSet is null for checkVolume() (sole timeout handler).
// For checkAllVolumes(), the set is shared with the pending-volumes
// loop; CAS-add determines which path owns the tolerance decision.
boolean firstToHandle =
(timeoutHandledSet == null) || timeoutHandledSet.add(volume);
if (firstToHandle) {
if (!volume.recordTimeoutAsIOFailure()) {
// Within tolerance: do NOT trigger the failure callback.
// The volume is not marked failed; the next check cycle will
// re-evaluate its health. cleanup() is intentionally not called
// to avoid firing handleVolumeFailures() with an empty failed set.
return;
}
// Tolerance exceeded — fall through to markFailed()/cleanup().
}
// else: the pending-volumes loop already handled this timeout.
// Fall through to markFailed()/cleanup() for counter bookkeeping only.
}
markFailed();
cleanup();
}

private void markHealthy() {
healthyVolumes.add(volume);
// A successful completion resets any accumulated timeout count so that
// an earlier transient timeout does not carry over to future cycles.
volume.resetTimeoutCount();
}

private void markFailed() {
Expand Down
Loading