|
@@ -18,9 +18,10 @@
|
|
|
package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
|
|
|
|
import static org.junit.Assert.*;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -31,19 +32,35 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.AppendTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
-import org.apache.hadoop.hdfs.TestDFSClientFailover;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
|
|
|
+import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
|
|
|
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
|
|
|
+
|
|
|
import org.apache.log4j.Level;
|
|
|
-import org.junit.Ignore;
|
|
|
+
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
+
|
|
|
+import com.google.common.base.Supplier;
|
|
|
|
|
|
/**
|
|
|
* Test cases regarding pipeline recovery during NN failover.
|
|
@@ -64,6 +81,9 @@ public class TestPipelinesFailover {
|
|
|
new Path("/test-file");
|
|
|
private static final int BLOCK_SIZE = 4096;
|
|
|
private static final int BLOCK_AND_A_HALF = BLOCK_SIZE * 3 / 2;
|
|
|
+
|
|
|
+ private static final int STRESS_NUM_THREADS = 25;
|
|
|
+ private static final int STRESS_RUNTIME = 40000;
|
|
|
|
|
|
/**
|
|
|
* Tests continuing a write pipeline over a failover.
|
|
@@ -216,22 +236,271 @@ public class TestPipelinesFailover {
|
|
|
cluster.transitionToActive(1);
|
|
|
|
|
|
assertTrue(fs.exists(TEST_PATH));
|
|
|
-
|
|
|
- FileSystem fsOtherUser = UserGroupInformation.createUserForTesting(
|
|
|
- "otheruser", new String[] { "othergroup"})
|
|
|
- .doAs(new PrivilegedExceptionAction<FileSystem>() {
|
|
|
- @Override
|
|
|
- public FileSystem run() throws Exception {
|
|
|
- return HATestUtil.configureFailoverFs(cluster, conf);
|
|
|
- }
|
|
|
- });
|
|
|
- ((DistributedFileSystem)fsOtherUser).recoverLease(TEST_PATH);
|
|
|
+
|
|
|
+ FileSystem fsOtherUser = createFsAsOtherUser(cluster, conf);
|
|
|
+ loopRecoverLease(fsOtherUser, TEST_PATH);
|
|
|
|
|
|
AppendTestUtil.check(fs, TEST_PATH, BLOCK_AND_A_HALF);
|
|
|
+
|
|
|
+ // Fail back to ensure that the block locations weren't lost on the
|
|
|
+ // original node.
|
|
|
+ cluster.transitionToStandby(1);
|
|
|
+ cluster.transitionToActive(0);
|
|
|
+ AppendTestUtil.check(fs, TEST_PATH, BLOCK_AND_A_HALF);
|
|
|
+ } finally {
|
|
|
+ IOUtils.closeStream(stm);
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test the scenario where the NN fails over after issuing a block
|
|
|
+ * synchronization request, but before it is committed. The
|
|
|
+ * DN running the recovery should then fail to commit the synchronization
|
|
|
+ * and a later retry will succeed.
|
|
|
+ */
|
|
|
+ @Test(timeout=30000)
|
|
|
+ public void testFailoverRightBeforeCommitSynchronization() throws Exception {
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
+ // Disable permissions so that another user can recover the lease.
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+
|
|
|
+ FSDataOutputStream stm = null;
|
|
|
+ final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleHATopology())
|
|
|
+ .numDataNodes(3)
|
|
|
+ .build();
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+ cluster.transitionToActive(0);
|
|
|
+ Thread.sleep(500);
|
|
|
+
|
|
|
+ LOG.info("Starting with NN 0 active");
|
|
|
+ FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
|
|
+ stm = fs.create(TEST_PATH);
|
|
|
+
|
|
|
+ // write a half block
|
|
|
+ AppendTestUtil.write(stm, 0, BLOCK_SIZE / 2);
|
|
|
+ stm.hflush();
|
|
|
+
|
|
|
+ // Look into the block manager on the active node for the block
|
|
|
+ // under construction.
|
|
|
+
|
|
|
+ NameNode nn0 = cluster.getNameNode(0);
|
|
|
+ ExtendedBlock blk = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
|
|
+ DatanodeDescriptor expectedPrimary = getExpectedPrimaryNode(nn0, blk);
|
|
|
+ LOG.info("Expecting block recovery to be triggered on DN " +
|
|
|
+ expectedPrimary);
|
|
|
+
|
|
|
+ // Find the corresponding DN daemon, and spy on its connection to the
|
|
|
+ // active.
|
|
|
+ DataNode primaryDN = cluster.getDataNode(expectedPrimary.getIpcPort());
|
|
|
+ DatanodeProtocolClientSideTranslatorPB nnSpy =
|
|
|
+ DataNodeAdapter.spyOnBposToNN(primaryDN, nn0);
|
|
|
+
|
|
|
+ // Delay the commitBlockSynchronization call
|
|
|
+ DelayAnswer delayer = new DelayAnswer(LOG);
|
|
|
+ Mockito.doAnswer(delayer).when(nnSpy).commitBlockSynchronization(
|
|
|
+ Mockito.eq(blk),
|
|
|
+ Mockito.anyInt(), // new genstamp
|
|
|
+ Mockito.anyLong(), // new length
|
|
|
+ Mockito.eq(true), // close file
|
|
|
+ Mockito.eq(false), // delete block
|
|
|
+ (DatanodeID[]) Mockito.anyObject()); // new targets
|
|
|
+
|
|
|
+ DistributedFileSystem fsOtherUser = createFsAsOtherUser(cluster, conf);
|
|
|
+ assertFalse(fsOtherUser.recoverLease(TEST_PATH));
|
|
|
+
|
|
|
+ LOG.info("Waiting for commitBlockSynchronization call from primary");
|
|
|
+ delayer.waitForCall();
|
|
|
+
|
|
|
+ LOG.info("Failing over to NN 1");
|
|
|
+
|
|
|
+ cluster.transitionToStandby(0);
|
|
|
+ cluster.transitionToActive(1);
|
|
|
+
|
|
|
+ // Let the commitBlockSynchronization call go through, and check that
|
|
|
+ // it failed with the correct exception.
|
|
|
+ delayer.proceed();
|
|
|
+ delayer.waitForResult();
|
|
|
+ Throwable t = delayer.getThrown();
|
|
|
+ if (t == null) {
|
|
|
+ fail("commitBlockSynchronization call did not fail on standby");
|
|
|
+ }
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
+ "Operation category WRITE is not supported",
|
|
|
+ t);
|
|
|
+
|
|
|
+ // Now, if we try again to recover the block, it should succeed on the new
|
|
|
+ // active.
|
|
|
+ loopRecoverLease(fsOtherUser, TEST_PATH);
|
|
|
+
|
|
|
+ AppendTestUtil.check(fs, TEST_PATH, BLOCK_SIZE/2);
|
|
|
} finally {
|
|
|
IOUtils.closeStream(stm);
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Stress test for pipeline/lease recovery. Starts a number of
|
|
|
+ * threads, each of which creates a file and has another client
|
|
|
+ * break the lease. While these threads run, failover proceeds
|
|
|
+ * back and forth between two namenodes.
|
|
|
+ */
|
|
|
+ @Test(timeout=STRESS_RUNTIME*3)
|
|
|
+ public void testPipelineRecoveryStress() throws Exception {
|
|
|
+ HAStressTestHarness harness = new HAStressTestHarness();
|
|
|
+ // Disable permissions so that another user can recover the lease.
|
|
|
+ harness.conf.setBoolean(
|
|
|
+ DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
|
|
|
|
|
|
+ final MiniDFSCluster cluster = harness.startCluster();
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+ cluster.transitionToActive(0);
|
|
|
+
|
|
|
+ FileSystem fs = harness.getFailoverFs();
|
|
|
+ DistributedFileSystem fsAsOtherUser = createFsAsOtherUser(
|
|
|
+ cluster, harness.conf);
|
|
|
+
|
|
|
+ TestContext testers = new TestContext();
|
|
|
+ for (int i = 0; i < STRESS_NUM_THREADS; i++) {
|
|
|
+ Path p = new Path("/test-" + i);
|
|
|
+ testers.addThread(new PipelineTestThread(
|
|
|
+ testers, fs, fsAsOtherUser, p));
|
|
|
+ }
|
|
|
+
|
|
|
+ // Start a separate thread which will make sure that replication
|
|
|
+ // happens quickly by triggering deletion reports and replication
|
|
|
+ // work calculation frequently.
|
|
|
+ harness.addReplicationTriggerThread(500);
|
|
|
+ harness.addFailoverThread(5000);
|
|
|
+ harness.startThreads();
|
|
|
+ testers.startThreads();
|
|
|
+
|
|
|
+ testers.waitFor(STRESS_RUNTIME);
|
|
|
+ testers.stop();
|
|
|
+ harness.stopThreads();
|
|
|
+ } finally {
|
|
|
+ System.err.println("===========================\n\n\n\n");
|
|
|
+ harness.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test thread which creates a file, has another fake user recover
|
|
|
+ * the lease on the file, and then ensures that the file's contents
|
|
|
+ * are properly readable. If any of these steps fails, propagates
|
|
|
+ * an exception back to the test context, causing the test case
|
|
|
+ * to fail.
|
|
|
+ */
|
|
|
+ private static class PipelineTestThread extends RepeatingTestThread {
|
|
|
+ private final FileSystem fs;
|
|
|
+ private final FileSystem fsOtherUser;
|
|
|
+ private final Path path;
|
|
|
+
|
|
|
+
|
|
|
+ public PipelineTestThread(TestContext ctx,
|
|
|
+ FileSystem fs, FileSystem fsOtherUser, Path p) {
|
|
|
+ super(ctx);
|
|
|
+ this.fs = fs;
|
|
|
+ this.fsOtherUser = fsOtherUser;
|
|
|
+ this.path = p;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void doAnAction() throws Exception {
|
|
|
+ FSDataOutputStream stm = fs.create(path, true);
|
|
|
+ try {
|
|
|
+ AppendTestUtil.write(stm, 0, 100);
|
|
|
+ stm.hflush();
|
|
|
+ loopRecoverLease(fsOtherUser, path);
|
|
|
+ AppendTestUtil.check(fs, path, 100);
|
|
|
+ } finally {
|
|
|
+ try {
|
|
|
+ stm.close();
|
|
|
+ } catch (IOException e) {
|
|
|
+ // should expect this since we lost the lease
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String toString() {
|
|
|
+ return "Pipeline test thread for " + path;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @return the node which is expected to run the recovery of the
|
|
|
+ * given block, which is known to be under construction inside the
|
|
|
+ * given NameNOde.
|
|
|
+ */
|
|
|
+ private DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
|
|
|
+ ExtendedBlock blk) {
|
|
|
+ BlockManager bm0 = nn.getNamesystem().getBlockManager();
|
|
|
+ BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
|
|
|
+ assertTrue("Block " + blk + " should be under construction, " +
|
|
|
+ "got: " + storedBlock,
|
|
|
+ storedBlock instanceof BlockInfoUnderConstruction);
|
|
|
+ BlockInfoUnderConstruction ucBlock =
|
|
|
+ (BlockInfoUnderConstruction)storedBlock;
|
|
|
+ // We expect that the first indexed replica will be the one
|
|
|
+ // to be in charge of the synchronization / recovery protocol.
|
|
|
+ DatanodeDescriptor expectedPrimary = ucBlock.getExpectedLocations()[0];
|
|
|
+ return expectedPrimary;
|
|
|
+ }
|
|
|
+
|
|
|
+ private DistributedFileSystem createFsAsOtherUser(
|
|
|
+ final MiniDFSCluster cluster, final Configuration conf)
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ return (DistributedFileSystem) UserGroupInformation.createUserForTesting(
|
|
|
+ "otheruser", new String[] { "othergroup"})
|
|
|
+ .doAs(new PrivilegedExceptionAction<FileSystem>() {
|
|
|
+ @Override
|
|
|
+ public FileSystem run() throws Exception {
|
|
|
+ return HATestUtil.configureFailoverFs(
|
|
|
+ cluster, conf);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Try to cover the lease on the given file for up to 30
|
|
|
+ * seconds.
|
|
|
+ * @param fsOtherUser the filesystem to use for the recoverLease call
|
|
|
+ * @param testPath the path on which to run lease recovery
|
|
|
+ * @throws TimeoutException if lease recover does not succeed within 30
|
|
|
+ * seconds
|
|
|
+ * @throws InterruptedException if the thread is interrupted
|
|
|
+ */
|
|
|
+ private static void loopRecoverLease(
|
|
|
+ final FileSystem fsOtherUser, final Path testPath)
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ try {
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ boolean success;
|
|
|
+ try {
|
|
|
+ success = ((DistributedFileSystem)fsOtherUser)
|
|
|
+ .recoverLease(testPath);
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ if (!success) {
|
|
|
+ LOG.info("Waiting to recover lease successfully");
|
|
|
+ }
|
|
|
+ return success;
|
|
|
+ }
|
|
|
+ }, 1000, 30000);
|
|
|
+ } catch (TimeoutException e) {
|
|
|
+ throw new TimeoutException("Timed out recovering lease for " +
|
|
|
+ testPath);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|