|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
@@ -25,6 +26,7 @@ import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
import java.net.URI;
|
|
|
+import java.net.URL;
|
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -33,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
|
@@ -310,6 +313,66 @@ public class TestStandbyCheckpoints {
|
|
|
assertTrue("SBN should have finished checkpointing.",
|
|
|
answerer.getFireCount() == 1 && answerer.getResultCount() == 1);
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout=300000)
|
|
|
+ public void testReadsAllowedDuringCheckpoint() throws Exception {
|
|
|
+
|
|
|
+ // Set it up so that we know when the SBN checkpoint starts and ends.
|
|
|
+ FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
|
|
|
+ DelayAnswer answerer = new DelayAnswer(LOG);
|
|
|
+ Mockito.doAnswer(answerer).when(spyImage1)
|
|
|
+ .saveNamespace(Mockito.any(FSNamesystem.class),
|
|
|
+ Mockito.any(Canceler.class));
|
|
|
+
|
|
|
+ // Perform some edits and wait for a checkpoint to start on the SBN.
|
|
|
+ doEdits(0, 1000);
|
|
|
+ nn0.getRpcServer().rollEditLog();
|
|
|
+ answerer.waitForCall();
|
|
|
+ assertTrue("SBN is not performing checkpoint but it should be.",
|
|
|
+ answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
|
|
|
+
|
|
|
+ // Make sure that the lock has actually been taken by the checkpointing
|
|
|
+ // thread.
|
|
|
+ ThreadUtil.sleepAtLeastIgnoreInterrupts(1000);
|
|
|
+
|
|
|
+ // Perform an RPC that needs to take the write lock.
|
|
|
+ Thread t = new Thread() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ nn1.getRpcServer().restoreFailedStorage("false");
|
|
|
+ } catch (IOException e) {
|
|
|
+ e.printStackTrace();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ t.start();
|
|
|
+
|
|
|
+ // Make sure that our thread is waiting for the lock.
|
|
|
+ ThreadUtil.sleepAtLeastIgnoreInterrupts(1000);
|
|
|
+
|
|
|
+ assertFalse(nn1.getNamesystem().getFsLockForTests().hasQueuedThreads());
|
|
|
+ assertFalse(nn1.getNamesystem().getFsLockForTests().isWriteLocked());
|
|
|
+ assertTrue(nn1.getNamesystem().getLongReadLockForTests().hasQueuedThreads());
|
|
|
+
|
|
|
+ // Get /jmx of the standby NN web UI, which will cause the FSNS read lock to
|
|
|
+ // be taken.
|
|
|
+ String pageContents = DFSTestUtil.urlGet(new URL("http://" +
|
|
|
+ nn1.getHttpAddress().getHostName() + ":" +
|
|
|
+ nn1.getHttpAddress().getPort() + "/jmx"));
|
|
|
+ assertTrue(pageContents.contains("NumLiveDataNodes"));
|
|
|
+
|
|
|
+ // Make sure that the checkpoint is still going on, implying that the client
|
|
|
+ // RPC to the SBN happened during the checkpoint.
|
|
|
+ assertTrue("SBN should have still been checkpointing.",
|
|
|
+ answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
|
|
|
+ answerer.proceed();
|
|
|
+ answerer.waitForResult();
|
|
|
+ assertTrue("SBN should have finished checkpointing.",
|
|
|
+ answerer.getFireCount() == 1 && answerer.getResultCount() == 1);
|
|
|
+
|
|
|
+ t.join();
|
|
|
+ }
|
|
|
|
|
|
private void doEdits(int start, int stop) throws IOException {
|
|
|
for (int i = start; i < stop; i++) {
|