|
@@ -29,9 +29,9 @@ import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Map;
|
|
|
-import java.util.Queue;
|
|
|
import java.util.Set;
|
|
|
-import java.util.concurrent.ArrayBlockingQueue;
|
|
|
+import java.util.Deque;
|
|
|
+import java.util.LinkedList;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
@@ -40,8 +40,6 @@ import java.util.concurrent.Future;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_DEAD_NODE_QUEUE_MAX_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_DEAD_NODE_QUEUE_MAX_KEY;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_CONNECTION_TIMEOUT_MS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_CONNECTION_TIMEOUT_MS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_DEAD_NODE_INTERVAL_MS_DEFAULT;
|
|
@@ -54,9 +52,9 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_THREADS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_RPC_THREADS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_RPC_THREADS_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_KEY;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_DEFAULT;
|
|
|
|
|
|
/**
|
|
|
* Detect the dead nodes in advance, and share this information among all the
|
|
@@ -74,7 +72,7 @@ public class DeadNodeDetector extends Daemon {
|
|
|
/**
|
|
|
* Waiting time when DeadNodeDetector's state is idle.
|
|
|
*/
|
|
|
- private static final long IDLE_SLEEP_MS = 10000;
|
|
|
+ private final long idleSleepMs;
|
|
|
|
|
|
/**
|
|
|
* Client context name.
|
|
@@ -113,16 +111,6 @@ public class DeadNodeDetector extends Daemon {
|
|
|
*/
|
|
|
private long suspectNodeDetectInterval = 0;
|
|
|
|
|
|
- /**
|
|
|
- * The max queue size of probing dead node.
|
|
|
- */
|
|
|
- private int maxDeadNodesProbeQueueLen = 0;
|
|
|
-
|
|
|
- /**
|
|
|
- * The max queue size of probing suspect node.
|
|
|
- */
|
|
|
- private int maxSuspectNodesProbeQueueLen;
|
|
|
-
|
|
|
/**
|
|
|
* Connection timeout for probing dead node in milliseconds.
|
|
|
*/
|
|
@@ -131,12 +119,12 @@ public class DeadNodeDetector extends Daemon {
|
|
|
/**
|
|
|
* The dead node probe queue.
|
|
|
*/
|
|
|
- private Queue<DatanodeInfo> deadNodesProbeQueue;
|
|
|
+ private UniqueQueue<DatanodeInfo> deadNodesProbeQueue;
|
|
|
|
|
|
/**
|
|
|
* The suspect node probe queue.
|
|
|
*/
|
|
|
- private Queue<DatanodeInfo> suspectNodesProbeQueue;
|
|
|
+ private UniqueQueue<DatanodeInfo> suspectNodesProbeQueue;
|
|
|
|
|
|
/**
|
|
|
* The thread pool of probing dead node.
|
|
@@ -181,6 +169,32 @@ public class DeadNodeDetector extends Daemon {
|
|
|
INIT, CHECK_DEAD, IDLE, ERROR
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The thread safe unique queue.
|
|
|
+ */
|
|
|
+ static class UniqueQueue<T> {
|
|
|
+ private Deque<T> queue = new LinkedList<>();
|
|
|
+ private Set<T> set = new HashSet<>();
|
|
|
+
|
|
|
+ synchronized boolean offer(T dn) {
|
|
|
+ if (set.add(dn)) {
|
|
|
+ queue.addLast(dn);
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized T poll() {
|
|
|
+ T dn = queue.pollFirst();
|
|
|
+ set.remove(dn);
|
|
|
+ return dn;
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized int size() {
|
|
|
+ return set.size();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Disabled start probe suspect/dead thread for the testing.
|
|
|
*/
|
|
@@ -203,20 +217,14 @@ public class DeadNodeDetector extends Daemon {
|
|
|
DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_INTERVAL_MS_DEFAULT);
|
|
|
socketTimeout =
|
|
|
conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, HdfsConstants.READ_TIMEOUT);
|
|
|
- maxDeadNodesProbeQueueLen =
|
|
|
- conf.getInt(DFS_CLIENT_DEAD_NODE_DETECTION_DEAD_NODE_QUEUE_MAX_KEY,
|
|
|
- DFS_CLIENT_DEAD_NODE_DETECTION_DEAD_NODE_QUEUE_MAX_DEFAULT);
|
|
|
- maxSuspectNodesProbeQueueLen =
|
|
|
- conf.getInt(DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_KEY,
|
|
|
- DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_DEFAULT);
|
|
|
probeConnectionTimeoutMs = conf.getLong(
|
|
|
DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_CONNECTION_TIMEOUT_MS_KEY,
|
|
|
DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_CONNECTION_TIMEOUT_MS_DEFAULT);
|
|
|
+ this.deadNodesProbeQueue = new UniqueQueue<>();
|
|
|
+ this.suspectNodesProbeQueue = new UniqueQueue<>();
|
|
|
|
|
|
- this.deadNodesProbeQueue =
|
|
|
- new ArrayBlockingQueue<DatanodeInfo>(maxDeadNodesProbeQueueLen);
|
|
|
- this.suspectNodesProbeQueue =
|
|
|
- new ArrayBlockingQueue<DatanodeInfo>(maxSuspectNodesProbeQueueLen);
|
|
|
+ idleSleepMs = conf.getLong(DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_KEY,
|
|
|
+ DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_DEFAULT);
|
|
|
|
|
|
int deadNodeDetectDeadThreads =
|
|
|
conf.getInt(DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_DEAD_NODE_THREADS_KEY,
|
|
@@ -447,8 +455,7 @@ public class DeadNodeDetector extends Daemon {
|
|
|
for (DatanodeInfo datanodeInfo : datanodeInfos) {
|
|
|
if (!deadNodesProbeQueue.offer(datanodeInfo)) {
|
|
|
LOG.debug("Skip to add dead node {} to check " +
|
|
|
- "since the probe queue is full.", datanodeInfo);
|
|
|
- break;
|
|
|
+ "since the node is already in the probe queue.", datanodeInfo);
|
|
|
} else {
|
|
|
LOG.debug("Add dead node to check: {}.", datanodeInfo);
|
|
|
}
|
|
@@ -458,7 +465,7 @@ public class DeadNodeDetector extends Daemon {
|
|
|
|
|
|
private void idle() {
|
|
|
try {
|
|
|
- Thread.sleep(IDLE_SLEEP_MS);
|
|
|
+ Thread.sleep(idleSleepMs);
|
|
|
} catch (InterruptedException e) {
|
|
|
LOG.debug("Got interrupted while DeadNodeDetector is idle.", e);
|
|
|
Thread.currentThread().interrupt();
|
|
@@ -483,14 +490,24 @@ public class DeadNodeDetector extends Daemon {
|
|
|
deadNodes.remove(datanodeInfo.getDatanodeUuid());
|
|
|
}
|
|
|
|
|
|
- public Queue<DatanodeInfo> getDeadNodesProbeQueue() {
|
|
|
+ public UniqueQueue<DatanodeInfo> getDeadNodesProbeQueue() {
|
|
|
return deadNodesProbeQueue;
|
|
|
}
|
|
|
|
|
|
- public Queue<DatanodeInfo> getSuspectNodesProbeQueue() {
|
|
|
+ public UniqueQueue<DatanodeInfo> getSuspectNodesProbeQueue() {
|
|
|
return suspectNodesProbeQueue;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ void setSuspectQueue(UniqueQueue<DatanodeInfo> queue) {
|
|
|
+ this.suspectNodesProbeQueue = queue;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ void setDeadQueue(UniqueQueue<DatanodeInfo> queue) {
|
|
|
+ this.deadNodesProbeQueue = queue;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Add datanode to suspectNodes and suspectAndDeadNodes.
|
|
|
*/
|