HDFS-15149. TestDeadNodeDetection test cases time-out. Contributed by Lisheng Sun.

This commit is contained in:
Inigo Goiri 2020-02-28 18:47:22 -08:00
parent 9a3a28018a
commit 97b797c314
4 changed files with 102 additions and 33 deletions

View File

@ -293,7 +293,7 @@ public class ClientContext {
if (deadNodeDetectorThr != null) { if (deadNodeDetectorThr != null) {
deadNodeDetectorThr.interrupt(); deadNodeDetectorThr.interrupt();
try { try {
deadNodeDetectorThr.join(3000); deadNodeDetectorThr.join();
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.warn("Encountered exception while waiting to join on dead " + LOG.warn("Encountered exception while waiting to join on dead " +
"node detector thread.", e); "node detector thread.", e);

View File

@ -247,6 +247,20 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
private final int smallBufferSize; private final int smallBufferSize;
private final long serverDefaultsValidityPeriod; private final long serverDefaultsValidityPeriod;
/**
* Disabled stop DeadNodeDetectorThread for the testing when MiniDFSCluster
* start.
*/
private static volatile boolean disabledStopDeadNodeDetectorThreadForTest =
false;
@VisibleForTesting
public static void setDisabledStopDeadNodeDetectorThreadForTest(
boolean disabledStopDeadNodeDetectorThreadForTest) {
DFSClient.disabledStopDeadNodeDetectorThreadForTest =
disabledStopDeadNodeDetectorThreadForTest;
}
public DfsClientConf getConf() { public DfsClientConf getConf() {
return dfsClientConf; return dfsClientConf;
} }
@ -637,7 +651,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
closeAllFilesBeingWritten(false); closeAllFilesBeingWritten(false);
clientRunning = false; clientRunning = false;
// close dead node detector thread // close dead node detector thread
if (!disabledStopDeadNodeDetectorThreadForTest) {
clientContext.stopDeadNodeDetectorThread(); clientContext.stopDeadNodeDetectorThread();
}
// close connections to the namenode // close connections to the namenode
closeConnectionToNamenode(); closeConnectionToNamenode();
} }

View File

@ -243,7 +243,7 @@ public class DeadNodeDetector implements Runnable {
@Override @Override
public void run() { public void run() {
while (true) { while (!Thread.currentThread().isInterrupted()) {
clearAndGetDetectedDeadNodes(); clearAndGetDetectedDeadNodes();
LOG.debug("Current detector state {}, the detected nodes: {}.", state, LOG.debug("Current detector state {}, the detected nodes: {}.", state,
deadNodes.values()); deadNodes.values());
@ -261,6 +261,8 @@ public class DeadNodeDetector implements Runnable {
try { try {
Thread.sleep(ERROR_SLEEP_MS); Thread.sleep(ERROR_SLEEP_MS);
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.debug("Got interrupted while DeadNodeDetector is error.", e);
Thread.currentThread().interrupt();
} }
return; return;
default: default:
@ -270,8 +272,9 @@ public class DeadNodeDetector implements Runnable {
} }
@VisibleForTesting @VisibleForTesting
static void disabledProbeThreadForTest() { static void setDisabledProbeThreadForTest(
disabledProbeThreadForTest = true; boolean disabledProbeThreadForTest) {
DeadNodeDetector.disabledProbeThreadForTest = disabledProbeThreadForTest;
} }
/** /**
@ -426,7 +429,8 @@ public class DeadNodeDetector implements Runnable {
try { try {
Thread.sleep(IDLE_SLEEP_MS); Thread.sleep(IDLE_SLEEP_MS);
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.debug("Got interrupted while DeadNodeDetector is idle.", e);
Thread.currentThread().interrupt();
} }
state = State.CHECK_DEAD; state = State.CHECK_DEAD;
@ -548,7 +552,9 @@ public class DeadNodeDetector implements Runnable {
try { try {
Thread.sleep(time); Thread.sleep(time);
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.debug("Got interrupted while probe is scheduling.", e);
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
return;
} }
} }
@ -566,7 +572,7 @@ public class DeadNodeDetector implements Runnable {
@Override @Override
public void run() { public void run() {
while (true) { while (!Thread.currentThread().isInterrupted()) {
deadNodeDetector.scheduleProbe(type); deadNodeDetector.scheduleProbe(type);
if (type == ProbeType.CHECK_SUSPECT) { if (type == ProbeType.CHECK_SUSPECT) {
probeSleep(deadNodeDetector.suspectNodeDetectInterval); probeSleep(deadNodeDetector.suspectNodeDetectInterval);

View File

@ -31,12 +31,17 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.Queue;
import java.util.concurrent.LinkedBlockingQueue;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
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_DEAD_NODE_QUEUE_MAX_KEY;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_ENABLED_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_ENABLED_KEY;
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_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_DEAD_NODE_INTERVAL_MS_KEY;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_INTERVAL_MS_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_INTERVAL_MS_KEY;
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_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_KEY;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
/** /**
@ -53,9 +58,15 @@ public class TestDeadNodeDetection {
conf = new HdfsConfiguration(); conf = new HdfsConfiguration();
conf.setBoolean(DFS_CLIENT_DEAD_NODE_DETECTION_ENABLED_KEY, true); conf.setBoolean(DFS_CLIENT_DEAD_NODE_DETECTION_ENABLED_KEY, true);
conf.setLong( conf.setLong(
DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_DEAD_NODE_INTERVAL_MS_KEY, 1000); DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_DEAD_NODE_INTERVAL_MS_KEY,
1000);
conf.setLong( conf.setLong(
DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_INTERVAL_MS_KEY, 100); DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_INTERVAL_MS_KEY,
100);
conf.setLong(
DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_CONNECTION_TIMEOUT_MS_KEY,
1000);
conf.setInt(DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, 0);
} }
@After @After
@ -67,6 +78,7 @@ public class TestDeadNodeDetection {
@Test @Test
public void testDeadNodeDetectionInBackground() throws Exception { public void testDeadNodeDetectionInBackground() throws Exception {
conf.set(DFS_CLIENT_CONTEXT, "testDeadNodeDetectionInBackground");
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
cluster.waitActive(); cluster.waitActive();
@ -102,7 +114,10 @@ public class TestDeadNodeDetection {
} catch (BlockMissingException e) { } catch (BlockMissingException e) {
} }
waitForDeadNode(dfsClient, 3); DefaultCoordination defaultCoordination = new DefaultCoordination();
defaultCoordination.startWaitForDeadNodeThread(dfsClient, 3);
defaultCoordination.sync();
assertEquals(3, dfsClient.getDeadNodes(din).size()); assertEquals(3, dfsClient.getDeadNodes(din).size());
assertEquals(3, dfsClient.getClientContext().getDeadNodeDetector() assertEquals(3, dfsClient.getClientContext().getDeadNodeDetector()
.clearAndGetDetectedDeadNodes().size()); .clearAndGetDetectedDeadNodes().size());
@ -143,6 +158,10 @@ public class TestDeadNodeDetection {
din2 = (DFSInputStream) in2.getWrappedStream(); din2 = (DFSInputStream) in2.getWrappedStream();
dfsClient2 = din2.getDFSClient(); dfsClient2 = din2.getDFSClient();
DefaultCoordination defaultCoordination = new DefaultCoordination();
defaultCoordination.startWaitForDeadNodeThread(dfsClient2, 1);
defaultCoordination.sync();
assertEquals(dfsClient1.toString(), dfsClient2.toString()); assertEquals(dfsClient1.toString(), dfsClient2.toString());
assertEquals(1, dfsClient1.getDeadNodes(din1).size()); assertEquals(1, dfsClient1.getDeadNodes(din1).size());
assertEquals(1, dfsClient2.getDeadNodes(din2).size()); assertEquals(1, dfsClient2.getDeadNodes(din2).size());
@ -173,9 +192,13 @@ public class TestDeadNodeDetection {
@Test @Test
public void testDeadNodeDetectionDeadNodeRecovery() throws Exception { public void testDeadNodeDetectionDeadNodeRecovery() throws Exception {
// prevent interrupt deadNodeDetectorThr in cluster.waitActive()
DFSClient.setDisabledStopDeadNodeDetectorThreadForTest(true);
conf.set(DFS_CLIENT_CONTEXT, "testDeadNodeDetectionDeadNodeRecovery");
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
cluster.waitActive(); cluster.waitActive();
DFSClient.setDisabledStopDeadNodeDetectorThreadForTest(false);
FileSystem fs = cluster.getFileSystem(); FileSystem fs = cluster.getFileSystem();
Path filePath = new Path("/testDeadNodeDetectionDeadNodeRecovery"); Path filePath = new Path("/testDeadNodeDetectionDeadNodeRecovery");
createFile(fs, filePath); createFile(fs, filePath);
@ -193,14 +216,18 @@ public class TestDeadNodeDetection {
in.read(); in.read();
} catch (BlockMissingException e) { } catch (BlockMissingException e) {
} }
DefaultCoordination defaultCoordination = new DefaultCoordination();
waitForDeadNode(dfsClient, 3); defaultCoordination.startWaitForDeadNodeThread(dfsClient, 3);
defaultCoordination.sync();
assertEquals(3, dfsClient.getDeadNodes(din).size()); assertEquals(3, dfsClient.getDeadNodes(din).size());
assertEquals(3, dfsClient.getClientContext().getDeadNodeDetector() assertEquals(3, dfsClient.getClientContext().getDeadNodeDetector()
.clearAndGetDetectedDeadNodes().size()); .clearAndGetDetectedDeadNodes().size());
cluster.restartDataNode(one, true); cluster.restartDataNode(one, true);
waitForDeadNode(dfsClient, 2);
defaultCoordination = new DefaultCoordination();
defaultCoordination.startWaitForDeadNodeThread(dfsClient, 2);
defaultCoordination.sync();
assertEquals(2, dfsClient.getDeadNodes(din).size()); assertEquals(2, dfsClient.getDeadNodes(din).size());
assertEquals(2, dfsClient.getClientContext().getDeadNodeDetector() assertEquals(2, dfsClient.getClientContext().getDeadNodeDetector()
.clearAndGetDetectedDeadNodes().size()); .clearAndGetDetectedDeadNodes().size());
@ -250,7 +277,7 @@ public class TestDeadNodeDetection {
@Test @Test
public void testDeadNodeDetectionSuspectNode() throws Exception { public void testDeadNodeDetectionSuspectNode() throws Exception {
conf.setInt(DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_KEY, 1); conf.setInt(DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_KEY, 1);
DeadNodeDetector.disabledProbeThreadForTest(); DeadNodeDetector.setDisabledProbeThreadForTest(true);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive(); cluster.waitActive();
@ -288,6 +315,8 @@ public class TestDeadNodeDetection {
assertEquals(0, dfsClient.getDeadNodes(din).size()); assertEquals(0, dfsClient.getDeadNodes(din).size());
assertEquals(0, dfsClient.getClientContext().getDeadNodeDetector() assertEquals(0, dfsClient.getClientContext().getDeadNodeDetector()
.clearAndGetDetectedDeadNodes().size()); .clearAndGetDetectedDeadNodes().size());
// reset disabledProbeThreadForTest
DeadNodeDetector.setDisabledProbeThreadForTest(false);
} }
} }
@ -317,24 +346,6 @@ public class TestDeadNodeDetection {
fs.delete(filePath, true); fs.delete(filePath, true);
} }
private void waitForDeadNode(DFSClient dfsClient, int size) throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
try {
if (dfsClient.getClientContext().getDeadNodeDetector()
.clearAndGetDetectedDeadNodes().size() == size) {
return true;
}
} catch (Exception e) {
// Ignore the exception
}
return false;
}
}, 5000, 100000);
}
private void waitForSuspectNode(DFSClient dfsClient) throws Exception { private void waitForSuspectNode(DFSClient dfsClient) throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() { GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override @Override
@ -350,6 +361,41 @@ public class TestDeadNodeDetection {
return false; return false;
} }
}, 5000, 100000); }, 500, 5000);
}
class DefaultCoordination {
private Queue<Object> queue = new LinkedBlockingQueue<Object>(1);
public boolean addToQueue() {
return queue.offer(new Object());
}
public Object removeFromQueue() {
return queue.poll();
}
public void sync() {
while (removeFromQueue() == null) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
}
}
private void startWaitForDeadNodeThread(DFSClient dfsClient, int size) {
new Thread(() -> {
DeadNodeDetector deadNodeDetector =
dfsClient.getClientContext().getDeadNodeDetector();
while (deadNodeDetector.clearAndGetDetectedDeadNodes().size() != size) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
}
addToQueue();
}).start();
}
} }
} }