|
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
@@ -34,6 +35,7 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
@@ -256,7 +258,8 @@ public class TestClientProtocolForPipelineRecovery {
|
|
|
final String[] args1 = {"-shutdownDatanode", dnAddr, "upgrade" };
|
|
|
Assert.assertEquals(0, dfsadmin.run(args1));
|
|
|
// Wait long enough to receive an OOB ack before closing the file.
|
|
|
- Thread.sleep(4000);
|
|
|
+ GenericTestUtils.waitForThreadTermination(
|
|
|
+ "Async datanode shutdown thread", 100, 10000);
|
|
|
// Retart the datanode
|
|
|
cluster.restartDataNode(0, true);
|
|
|
// The following forces a data packet and end of block packets to be sent.
|
|
@@ -293,7 +296,8 @@ public class TestClientProtocolForPipelineRecovery {
|
|
|
// issue shutdown to the datanode.
|
|
|
final String[] args1 = {"-shutdownDatanode", dnAddr1, "upgrade" };
|
|
|
Assert.assertEquals(0, dfsadmin.run(args1));
|
|
|
- Thread.sleep(4000);
|
|
|
+ GenericTestUtils.waitForThreadTermination(
|
|
|
+ "Async datanode shutdown thread", 100, 10000);
|
|
|
// This should succeed without restarting the node. The restart will
|
|
|
// expire and regular pipeline recovery will kick in.
|
|
|
out.close();
|
|
@@ -309,7 +313,8 @@ public class TestClientProtocolForPipelineRecovery {
|
|
|
// issue shutdown to the datanode.
|
|
|
final String[] args2 = {"-shutdownDatanode", dnAddr2, "upgrade" };
|
|
|
Assert.assertEquals(0, dfsadmin.run(args2));
|
|
|
- Thread.sleep(4000);
|
|
|
+ GenericTestUtils.waitForThreadTermination(
|
|
|
+ "Async datanode shutdown thread", 100, 10000);
|
|
|
try {
|
|
|
// close should fail
|
|
|
out.close();
|
|
@@ -321,4 +326,53 @@ public class TestClientProtocolForPipelineRecovery {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * HDFS-9752. The client keeps sending heartbeat packets during datanode
|
|
|
+ * rolling upgrades. The client should be able to retry pipeline recovery
|
|
|
+ * more times than the default.
|
|
|
+ * (in a row for the same packet, including the heartbeat packet)
|
|
|
+ * (See{@link DataStreamer#pipelineRecoveryCount})
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testPipelineRecoveryOnDatanodeUpgrade() throws Exception {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
|
|
+ cluster.waitActive();
|
|
|
+ FileSystem fileSys = cluster.getFileSystem();
|
|
|
+
|
|
|
+ Path file = new Path("/testPipelineRecoveryOnDatanodeUpgrade");
|
|
|
+ DFSTestUtil.createFile(fileSys, file, 10240L, (short) 2, 0L);
|
|
|
+ final DFSOutputStream out = (DFSOutputStream) (fileSys.append(file).
|
|
|
+ getWrappedStream());
|
|
|
+ out.write(1);
|
|
|
+ out.hflush();
|
|
|
+
|
|
|
+ final long oldGs = out.getBlock().getGenerationStamp();
|
|
|
+ MiniDFSCluster.DataNodeProperties dnProps =
|
|
|
+ cluster.stopDataNodeForUpgrade(0);
|
|
|
+ GenericTestUtils.waitForThreadTermination(
|
|
|
+ "Async datanode shutdown thread", 100, 10000);
|
|
|
+ cluster.restartDataNode(dnProps, true);
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ // wait pipeline to be recovered
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ return out.getBlock().getGenerationStamp() > oldGs;
|
|
|
+ }
|
|
|
+ }, 100, 10000);
|
|
|
+ Assert.assertEquals("The pipeline recovery count shouldn't increase",
|
|
|
+ 0, out.getStreamer().getPipelineRecoveryCount());
|
|
|
+ out.write(1);
|
|
|
+ out.close();
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|