ptlrs commented on code in PR #9954:
URL: https://github.com/apache/ozone/pull/9954#discussion_r3005501000


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java:
##########
@@ -721,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.

Review Comment:
   Can we remove all changes not related to `consecutiveTimeoutCount`.
   
   These changes conflict with the PR https://github.com/apache/ozone/pull/8843 
which transitions the `StorageVolume` class to use the new `SlidingWindow` 
implementation.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java:
##########
@@ -753,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);
+    }
+  }

Review Comment:
   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](https://github.com/apache/ozone/blob/master/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/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. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to