|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.hdds.scm.node;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -43,6 +44,7 @@ import org.slf4j.LoggerFactory;
|
|
|
import java.io.Closeable;
|
|
|
import java.util.*;
|
|
|
import java.util.concurrent.ScheduledExecutorService;
|
|
|
+import java.util.concurrent.ScheduledFuture;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.function.Predicate;
|
|
|
|
|
@@ -116,6 +118,26 @@ public class NodeStateManager implements Runnable, Closeable {
|
|
|
*/
|
|
|
private final long deadNodeIntervalMs;
|
|
|
|
|
|
+ /**
|
|
|
+ * The future is used to pause/unpause the scheduled checks.
|
|
|
+ */
|
|
|
+ private ScheduledFuture<?> healthCheckFuture;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test utility - tracks if health check has been paused (unit tests).
|
|
|
+ */
|
|
|
+ private boolean checkPaused;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * timestamp of the latest heartbeat check process.
|
|
|
+ */
|
|
|
+ private long lastHealthCheck;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * number of times the heart beat check was skipped.
|
|
|
+ */
|
|
|
+ private long skippedHealthChecks;
|
|
|
+
|
|
|
/**
|
|
|
* Constructs a NodeStateManager instance with the given configuration.
|
|
|
*
|
|
@@ -143,10 +165,11 @@ public class NodeStateManager implements Runnable, Closeable {
|
|
|
executorService = HadoopExecutors.newScheduledThreadPool(1,
|
|
|
new ThreadFactoryBuilder().setDaemon(true)
|
|
|
.setNameFormat("SCM Heartbeat Processing Thread - %d").build());
|
|
|
- //BUG:BUG TODO: The return value is ignored, if an exception is thrown in
|
|
|
- // the executing funtion, it will be ignored.
|
|
|
- executorService.schedule(this, heartbeatCheckerIntervalMs,
|
|
|
- TimeUnit.MILLISECONDS);
|
|
|
+
|
|
|
+ skippedHealthChecks = 0;
|
|
|
+ checkPaused = false; // accessed only from test functions
|
|
|
+
|
|
|
+ scheduleNextHealthCheck();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -464,6 +487,42 @@ public class NodeStateManager implements Runnable, Closeable {
|
|
|
@Override
|
|
|
public void run() {
|
|
|
|
|
|
+ if (shouldSkipCheck()) {
|
|
|
+ skippedHealthChecks++;
|
|
|
+ LOG.info("Detected long delay in scheduling HB processing thread. "
|
|
|
+ + "Skipping heartbeat checks for one iteration.");
|
|
|
+ } else {
|
|
|
+ checkNodesHealth();
|
|
|
+ }
|
|
|
+
|
|
|
+ // we purposefully make this non-deterministic. Instead of using a
|
|
|
+ // scheduleAtFixedFrequency we will just go to sleep
|
|
|
+ // and wake up at the next rendezvous point, which is currentTime +
|
|
|
+ // heartbeatCheckerIntervalMs. This leads to the issue that we are now
|
|
|
+ // heart beating not at a fixed cadence, but clock tick + time taken to
|
|
|
+ // work.
|
|
|
+ //
|
|
|
+ // This time taken to work can skew the heartbeat processor thread.
|
|
|
+ // The reason why we don't care is because of the following reasons.
|
|
|
+ //
|
|
|
+ // 1. checkerInterval is general many magnitudes faster than datanode HB
|
|
|
+ // frequency.
|
|
|
+ //
|
|
|
+ // 2. if we have too much nodes, the SCM would be doing only HB
|
|
|
+ // processing, this could lead to SCM's CPU starvation. With this
|
|
|
+ // approach we always guarantee that HB thread sleeps for a little while.
|
|
|
+ //
|
|
|
+ // 3. It is possible that we will never finish processing the HB's in the
|
|
|
+ // thread. But that means we have a mis-configured system. We will warn
|
|
|
+ // the users by logging that information.
|
|
|
+ //
|
|
|
+ // 4. And the most important reason, heartbeats are not blocked even if
|
|
|
+ // this thread does not run, they will go into the processing queue.
|
|
|
+ scheduleNextHealthCheck();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void checkNodesHealth() {
|
|
|
+
|
|
|
/*
|
|
|
*
|
|
|
* staleNodeDeadline healthyNodeDeadline
|
|
@@ -558,41 +617,36 @@ public class NodeStateManager implements Runnable, Closeable {
|
|
|
heartbeatCheckerIntervalMs);
|
|
|
}
|
|
|
|
|
|
- // we purposefully make this non-deterministic. Instead of using a
|
|
|
- // scheduleAtFixedFrequency we will just go to sleep
|
|
|
- // and wake up at the next rendezvous point, which is currentTime +
|
|
|
- // heartbeatCheckerIntervalMs. This leads to the issue that we are now
|
|
|
- // heart beating not at a fixed cadence, but clock tick + time taken to
|
|
|
- // work.
|
|
|
- //
|
|
|
- // This time taken to work can skew the heartbeat processor thread.
|
|
|
- // The reason why we don't care is because of the following reasons.
|
|
|
- //
|
|
|
- // 1. checkerInterval is general many magnitudes faster than datanode HB
|
|
|
- // frequency.
|
|
|
- //
|
|
|
- // 2. if we have too much nodes, the SCM would be doing only HB
|
|
|
- // processing, this could lead to SCM's CPU starvation. With this
|
|
|
- // approach we always guarantee that HB thread sleeps for a little while.
|
|
|
- //
|
|
|
- // 3. It is possible that we will never finish processing the HB's in the
|
|
|
- // thread. But that means we have a mis-configured system. We will warn
|
|
|
- // the users by logging that information.
|
|
|
- //
|
|
|
- // 4. And the most important reason, heartbeats are not blocked even if
|
|
|
- // this thread does not run, they will go into the processing queue.
|
|
|
+ }
|
|
|
+
|
|
|
+ private void scheduleNextHealthCheck() {
|
|
|
|
|
|
if (!Thread.currentThread().isInterrupted() &&
|
|
|
!executorService.isShutdown()) {
|
|
|
//BUGBUG: The return future needs to checked here to make sure the
|
|
|
// exceptions are handled correctly.
|
|
|
- executorService.schedule(this, heartbeatCheckerIntervalMs,
|
|
|
- TimeUnit.MILLISECONDS);
|
|
|
+ healthCheckFuture = executorService.schedule(this,
|
|
|
+ heartbeatCheckerIntervalMs, TimeUnit.MILLISECONDS);
|
|
|
} else {
|
|
|
- LOG.info("Current Thread is interrupted, shutting down HB processing " +
|
|
|
+ LOG.warn("Current Thread is interrupted, shutting down HB processing " +
|
|
|
"thread for Node Manager.");
|
|
|
}
|
|
|
|
|
|
+ lastHealthCheck = Time.monotonicNow();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * if the time since last check exceeds the stale|dead node interval, skip.
|
|
|
+ * such long delays might be caused by a JVM pause. SCM cannot make reliable
|
|
|
+ * conclusions about datanode health in such situations.
|
|
|
+ * @return : true indicates skip HB checks
|
|
|
+ */
|
|
|
+ private boolean shouldSkipCheck() {
|
|
|
+
|
|
|
+ long currentTime = Time.monotonicNow();
|
|
|
+ long minInterval = Math.min(staleNodeIntervalMs, deadNodeIntervalMs);
|
|
|
+
|
|
|
+ return ((currentTime - lastHealthCheck) >= minInterval);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -640,4 +694,57 @@ public class NodeStateManager implements Runnable, Closeable {
|
|
|
Thread.currentThread().interrupt();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test Utility : return number of times heartbeat check was skipped.
|
|
|
+ * @return : count of times HB process was skipped
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ long getSkippedHealthChecks() {
|
|
|
+ return skippedHealthChecks;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test Utility : Pause the periodic node hb check.
|
|
|
+ * @return ScheduledFuture for the scheduled check that got cancelled.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ ScheduledFuture pause() {
|
|
|
+
|
|
|
+ if (executorService.isShutdown() || checkPaused) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ checkPaused = healthCheckFuture.cancel(false);
|
|
|
+
|
|
|
+ return healthCheckFuture;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test utility : unpause the periodic node hb check.
|
|
|
+ * @return ScheduledFuture for the next scheduled check
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ ScheduledFuture unpause() {
|
|
|
+
|
|
|
+ if (executorService.isShutdown()) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (checkPaused) {
|
|
|
+ Preconditions.checkState(((healthCheckFuture == null)
|
|
|
+ || healthCheckFuture.isCancelled()
|
|
|
+ || healthCheckFuture.isDone()));
|
|
|
+
|
|
|
+ checkPaused = false;
|
|
|
+ /**
|
|
|
+ * We do not call scheduleNextHealthCheck because we are
|
|
|
+ * not updating the lastHealthCheck timestamp.
|
|
|
+ */
|
|
|
+ healthCheckFuture = executorService.schedule(this,
|
|
|
+ heartbeatCheckerIntervalMs, TimeUnit.MILLISECONDS);
|
|
|
+ }
|
|
|
+
|
|
|
+ return healthCheckFuture;
|
|
|
+ }
|
|
|
}
|