|
@@ -17,83 +17,94 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY;
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY;
|
|
|
|
+import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState;
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
|
+import static org.mockito.Matchers.any;
|
|
|
|
+import static org.mockito.Matchers.anyBoolean;
|
|
|
|
+import static org.mockito.Matchers.anyLong;
|
|
|
|
+import static org.mockito.Mockito.doAnswer;
|
|
|
|
+
|
|
|
|
+import java.io.File;
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.net.URI;
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.List;
|
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
+
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
|
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
|
|
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
-import org.apache.hadoop.io.retry.FailoverProxyProvider;
|
|
|
|
-import org.apache.hadoop.io.retry.RetryInvocationHandler;
|
|
|
|
-import org.apache.hadoop.test.GenericTestUtils;
|
|
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
|
|
+import org.junit.AfterClass;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
-import org.junit.Ignore;
|
|
|
|
|
|
+import org.junit.BeforeClass;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
+import org.mockito.Mockito;
|
|
|
|
+import org.slf4j.Logger;
|
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import java.io.File;
|
|
|
|
-import java.io.IOException;
|
|
|
|
-import java.lang.reflect.Proxy;
|
|
|
|
-import java.net.URI;
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
-import java.util.List;
|
|
|
|
-import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
|
-
|
|
|
|
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY;
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
|
-import static org.junit.Assert.fail;
|
|
|
|
|
|
+/**
|
|
|
|
+ * Test main functionality of ObserverNode.
|
|
|
|
+ */
|
|
|
|
+public class TestObserverNode {
|
|
|
|
+ public static final Logger LOG =
|
|
|
|
+ LoggerFactory.getLogger(TestObserverNode.class.getName());
|
|
|
|
|
|
-import static org.mockito.Matchers.any;
|
|
|
|
-import static org.mockito.Matchers.anyBoolean;
|
|
|
|
-import static org.mockito.Matchers.anyLong;
|
|
|
|
-import static org.mockito.Matchers.anyShort;
|
|
|
|
-import static org.mockito.Matchers.anyString;
|
|
|
|
-import static org.mockito.Mockito.doAnswer;
|
|
|
|
-import static org.mockito.Mockito.doNothing;
|
|
|
|
|
|
+ private static Configuration conf;
|
|
|
|
+ private static MiniQJMHACluster qjmhaCluster;
|
|
|
|
+ private static MiniDFSCluster dfsCluster;
|
|
|
|
+ private static DistributedFileSystem dfs;
|
|
|
|
|
|
-// Main unit tests for ObserverNode
|
|
|
|
-public class TestObserverNode {
|
|
|
|
- private Configuration conf;
|
|
|
|
- private MiniQJMHACluster qjmhaCluster;
|
|
|
|
- private MiniDFSCluster dfsCluster;
|
|
|
|
- private NameNode[] namenodes;
|
|
|
|
- private Path testPath;
|
|
|
|
- private Path testPath2;
|
|
|
|
- private Path testPath3;
|
|
|
|
-
|
|
|
|
- /** These are set in each individual test case */
|
|
|
|
- private DistributedFileSystem dfs;
|
|
|
|
- private ObserverReadProxyProvider<?> provider;
|
|
|
|
|
|
+ private final Path testPath= new Path("/TestObserverNode");
|
|
|
|
|
|
- @Before
|
|
|
|
- public void setUp() throws Exception {
|
|
|
|
|
|
+ @BeforeClass
|
|
|
|
+ public static void startUpCluster() throws Exception {
|
|
conf = new Configuration();
|
|
conf = new Configuration();
|
|
|
|
+ // disable block scanner
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
|
|
conf.setBoolean(DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
|
|
conf.setBoolean(DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
|
|
conf.setTimeDuration(
|
|
conf.setTimeDuration(
|
|
DFS_HA_TAILEDITS_PERIOD_KEY, 100, TimeUnit.MILLISECONDS);
|
|
DFS_HA_TAILEDITS_PERIOD_KEY, 100, TimeUnit.MILLISECONDS);
|
|
|
|
|
|
- testPath = new Path("/test");
|
|
|
|
- testPath2 = new Path("/test2");
|
|
|
|
- testPath3 = new Path("/test3");
|
|
|
|
|
|
+ qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 1);
|
|
|
|
+ dfsCluster = qjmhaCluster.getDfsCluster();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Before
|
|
|
|
+ public void setUp() throws Exception {
|
|
|
|
+ setObserverRead(true);
|
|
}
|
|
}
|
|
|
|
|
|
@After
|
|
@After
|
|
public void cleanUp() throws IOException {
|
|
public void cleanUp() throws IOException {
|
|
|
|
+ dfs.delete(testPath, true);
|
|
|
|
+ assertEquals("NN[0] should be active", HAServiceState.ACTIVE,
|
|
|
|
+ getServiceState(dfsCluster.getNameNode(0)));
|
|
|
|
+ assertEquals("NN[1] should be standby", HAServiceState.STANDBY,
|
|
|
|
+ getServiceState(dfsCluster.getNameNode(1)));
|
|
|
|
+ assertEquals("NN[2] should be observer", HAServiceState.OBSERVER,
|
|
|
|
+ getServiceState(dfsCluster.getNameNode(2)));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @AfterClass
|
|
|
|
+ public static void shutDownCluster() throws IOException {
|
|
if (qjmhaCluster != null) {
|
|
if (qjmhaCluster != null) {
|
|
qjmhaCluster.shutdown();
|
|
qjmhaCluster.shutdown();
|
|
}
|
|
}
|
|
@@ -101,13 +112,12 @@ public class TestObserverNode {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testSimpleRead() throws Exception {
|
|
public void testSimpleRead() throws Exception {
|
|
- setUpCluster(1);
|
|
|
|
- setObserverRead(true);
|
|
|
|
|
|
+ Path testPath2 = new Path(testPath, "test2");
|
|
|
|
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
assertSentTo(0);
|
|
assertSentTo(0);
|
|
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
|
|
+ dfsCluster.rollEditLogAndTail(0);
|
|
dfs.getFileStatus(testPath);
|
|
dfs.getFileStatus(testPath);
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
|
|
|
|
@@ -117,7 +127,7 @@ public class TestObserverNode {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testFailover() throws Exception {
|
|
public void testFailover() throws Exception {
|
|
- setUpCluster(1);
|
|
|
|
|
|
+ Path testPath2 = new Path(testPath, "test2");
|
|
setObserverRead(false);
|
|
setObserverRead(false);
|
|
|
|
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
@@ -127,23 +137,26 @@ public class TestObserverNode {
|
|
|
|
|
|
dfsCluster.transitionToStandby(0);
|
|
dfsCluster.transitionToStandby(0);
|
|
dfsCluster.transitionToActive(1);
|
|
dfsCluster.transitionToActive(1);
|
|
- dfsCluster.waitActive();
|
|
|
|
|
|
+ dfsCluster.waitActive(1);
|
|
|
|
|
|
dfs.mkdir(testPath2, FsPermission.getDefault());
|
|
dfs.mkdir(testPath2, FsPermission.getDefault());
|
|
assertSentTo(1);
|
|
assertSentTo(1);
|
|
dfs.getFileStatus(testPath);
|
|
dfs.getFileStatus(testPath);
|
|
assertSentTo(1);
|
|
assertSentTo(1);
|
|
|
|
+
|
|
|
|
+ dfsCluster.transitionToStandby(1);
|
|
|
|
+ dfsCluster.transitionToActive(0);
|
|
|
|
+ dfsCluster.waitActive(0);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testDoubleFailover() throws Exception {
|
|
public void testDoubleFailover() throws Exception {
|
|
- setUpCluster(1);
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
|
|
+ Path testPath2 = new Path(testPath, "test2");
|
|
|
|
+ Path testPath3 = new Path(testPath, "test3");
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
assertSentTo(0);
|
|
assertSentTo(0);
|
|
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
|
|
+ dfsCluster.rollEditLogAndTail(0);
|
|
dfs.getFileStatus(testPath);
|
|
dfs.getFileStatus(testPath);
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
dfs.mkdir(testPath2, FsPermission.getDefault());
|
|
dfs.mkdir(testPath2, FsPermission.getDefault());
|
|
@@ -153,7 +166,7 @@ public class TestObserverNode {
|
|
dfsCluster.transitionToActive(1);
|
|
dfsCluster.transitionToActive(1);
|
|
dfsCluster.waitActive(1);
|
|
dfsCluster.waitActive(1);
|
|
|
|
|
|
- rollEditLogAndTail(1);
|
|
|
|
|
|
+ dfsCluster.rollEditLogAndTail(1);
|
|
dfs.getFileStatus(testPath2);
|
|
dfs.getFileStatus(testPath2);
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
dfs.mkdir(testPath3, FsPermission.getDefault());
|
|
dfs.mkdir(testPath3, FsPermission.getDefault());
|
|
@@ -163,51 +176,17 @@ public class TestObserverNode {
|
|
dfsCluster.transitionToActive(0);
|
|
dfsCluster.transitionToActive(0);
|
|
dfsCluster.waitActive(0);
|
|
dfsCluster.waitActive(0);
|
|
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
|
|
+ dfsCluster.rollEditLogAndTail(0);
|
|
dfs.getFileStatus(testPath3);
|
|
dfs.getFileStatus(testPath3);
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
dfs.delete(testPath3, false);
|
|
dfs.delete(testPath3, false);
|
|
assertSentTo(0);
|
|
assertSentTo(0);
|
|
}
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
|
- public void testObserverFailover() throws Exception {
|
|
|
|
- setUpCluster(2);
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
- dfs.mkdir(testPath, FsPermission.getDefault());
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
- dfs.getFileStatus(testPath);
|
|
|
|
- assertSentToAny(2, 3);
|
|
|
|
-
|
|
|
|
- // Transition observer #2 to standby, request should go to the #3.
|
|
|
|
- dfsCluster.transitionToStandby(2);
|
|
|
|
- dfs.getFileStatus(testPath);
|
|
|
|
- assertSentTo(3);
|
|
|
|
-
|
|
|
|
- // Transition observer #3 to standby, request should go to active
|
|
|
|
- dfsCluster.transitionToStandby(3);
|
|
|
|
- dfs.getFileStatus(testPath);
|
|
|
|
- assertSentTo(0);
|
|
|
|
-
|
|
|
|
- // Transition #2 back to observer, request should go to #2
|
|
|
|
- dfsCluster.transitionToObserver(2);
|
|
|
|
- dfs.getFileStatus(testPath);
|
|
|
|
- assertSentTo(2);
|
|
|
|
-
|
|
|
|
- // Transition #3 back to observer, request should go to either #2 or #3
|
|
|
|
- dfsCluster.transitionToObserver(3);
|
|
|
|
- dfs.getFileStatus(testPath);
|
|
|
|
- assertSentToAny(2, 3);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
@Test
|
|
@Test
|
|
public void testObserverShutdown() throws Exception {
|
|
public void testObserverShutdown() throws Exception {
|
|
- setUpCluster(1);
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
- rollEditLogAndTail(0);
|
|
|
|
|
|
+ dfsCluster.rollEditLogAndTail(0);
|
|
dfs.getFileStatus(testPath);
|
|
dfs.getFileStatus(testPath);
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
|
|
|
|
@@ -228,18 +207,14 @@ public class TestObserverNode {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testObserverFailOverAndShutdown() throws Exception {
|
|
public void testObserverFailOverAndShutdown() throws Exception {
|
|
- setUpCluster(1);
|
|
|
|
- // Test the case when there is a failover before ONN shutdown
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
dfs.mkdir(testPath, FsPermission.getDefault());
|
|
- rollEditLogAndTail(0);
|
|
|
|
|
|
+ dfsCluster.rollEditLogAndTail(0);
|
|
dfs.getFileStatus(testPath);
|
|
dfs.getFileStatus(testPath);
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
|
|
|
|
dfsCluster.transitionToStandby(0);
|
|
dfsCluster.transitionToStandby(0);
|
|
dfsCluster.transitionToActive(1);
|
|
dfsCluster.transitionToActive(1);
|
|
- dfsCluster.waitActive();
|
|
|
|
|
|
+ dfsCluster.waitActive(1);
|
|
|
|
|
|
// Shutdown the observer - requests should go to active
|
|
// Shutdown the observer - requests should go to active
|
|
dfsCluster.shutdownNameNode(2);
|
|
dfsCluster.shutdownNameNode(2);
|
|
@@ -257,54 +232,14 @@ public class TestObserverNode {
|
|
// the second will properly go to the observer
|
|
// the second will properly go to the observer
|
|
dfs.getFileStatus(testPath);
|
|
dfs.getFileStatus(testPath);
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void testMultiObserver() throws Exception {
|
|
|
|
- setUpCluster(2);
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
- dfs.mkdir(testPath, FsPermission.getDefault());
|
|
|
|
- assertSentTo(0);
|
|
|
|
-
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
- dfs.getFileStatus(testPath);
|
|
|
|
- assertSentToAny(2, 3);
|
|
|
|
-
|
|
|
|
- dfs.mkdir(testPath2, FsPermission.getDefault());
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
|
|
|
|
- // Shutdown first observer, request should go to the second one
|
|
|
|
- dfsCluster.shutdownNameNode(2);
|
|
|
|
- dfs.listStatus(testPath2);
|
|
|
|
- assertSentTo(3);
|
|
|
|
-
|
|
|
|
- // Restart the first observer
|
|
|
|
- dfsCluster.restartNameNode(2);
|
|
|
|
- dfs.listStatus(testPath);
|
|
|
|
- assertSentTo(3);
|
|
|
|
-
|
|
|
|
- dfsCluster.transitionToObserver(2);
|
|
|
|
- dfs.listStatus(testPath);
|
|
|
|
- assertSentToAny(2, 3);
|
|
|
|
-
|
|
|
|
- dfs.mkdir(testPath3, FsPermission.getDefault());
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
-
|
|
|
|
- // Now shutdown the second observer, request should go to the first one
|
|
|
|
- dfsCluster.shutdownNameNode(3);
|
|
|
|
- dfs.listStatus(testPath3);
|
|
|
|
- assertSentTo(2);
|
|
|
|
-
|
|
|
|
- // Shutdown both, request should go to active
|
|
|
|
- dfsCluster.shutdownNameNode(2);
|
|
|
|
- dfs.listStatus(testPath3);
|
|
|
|
- assertSentTo(0);
|
|
|
|
|
|
+ dfsCluster.transitionToStandby(1);
|
|
|
|
+ dfsCluster.transitionToActive(0);
|
|
|
|
+ dfsCluster.waitActive(0);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testBootstrap() throws Exception {
|
|
public void testBootstrap() throws Exception {
|
|
- setUpCluster(1);
|
|
|
|
for (URI u : dfsCluster.getNameDirs(2)) {
|
|
for (URI u : dfsCluster.getNameDirs(2)) {
|
|
File dir = new File(u.getPath());
|
|
File dir = new File(u.getPath());
|
|
assertTrue(FileUtil.fullyDelete(dir));
|
|
assertTrue(FileUtil.fullyDelete(dir));
|
|
@@ -323,20 +258,12 @@ public class TestObserverNode {
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
public void testObserverNodeSafeModeWithBlockLocations() throws Exception {
|
|
public void testObserverNodeSafeModeWithBlockLocations() throws Exception {
|
|
- setUpCluster(1);
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
- // Avoid starting DNs for the mini cluster.
|
|
|
|
- BlockManager bmSpy = NameNodeAdapter.spyOnBlockManager(namenodes[0]);
|
|
|
|
- doNothing().when(bmSpy)
|
|
|
|
- .verifyReplication(anyString(), anyShort(), anyString());
|
|
|
|
-
|
|
|
|
// Create a new file - the request should go to active.
|
|
// Create a new file - the request should go to active.
|
|
- dfs.createNewFile(testPath);
|
|
|
|
|
|
+ dfs.create(testPath, (short)1).close();
|
|
assertSentTo(0);
|
|
assertSentTo(0);
|
|
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
- dfs.open(testPath);
|
|
|
|
|
|
+ dfsCluster.rollEditLogAndTail(0);
|
|
|
|
+ dfs.open(testPath).close();
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
|
|
|
|
// Set observer to safe mode.
|
|
// Set observer to safe mode.
|
|
@@ -345,7 +272,8 @@ public class TestObserverNode {
|
|
// Mock block manager for observer to generate some fake blocks which
|
|
// Mock block manager for observer to generate some fake blocks which
|
|
// will trigger the (retriable) safe mode exception.
|
|
// will trigger the (retriable) safe mode exception.
|
|
final DatanodeInfo[] empty = {};
|
|
final DatanodeInfo[] empty = {};
|
|
- bmSpy = NameNodeAdapter.spyOnBlockManager(namenodes[2]);
|
|
|
|
|
|
+ BlockManager bmSpy =
|
|
|
|
+ NameNodeAdapter.spyOnBlockManager(dfsCluster.getNameNode(2));
|
|
doAnswer((invocation) -> {
|
|
doAnswer((invocation) -> {
|
|
ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
|
|
ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
|
|
LocatedBlock fakeBlock = new LocatedBlock(b, empty);
|
|
LocatedBlock fakeBlock = new LocatedBlock(b, empty);
|
|
@@ -357,158 +285,23 @@ public class TestObserverNode {
|
|
|
|
|
|
// Open the file again - it should throw retriable exception and then
|
|
// Open the file again - it should throw retriable exception and then
|
|
// failover to active.
|
|
// failover to active.
|
|
- dfs.open(testPath);
|
|
|
|
|
|
+ dfs.open(testPath).close();
|
|
assertSentTo(0);
|
|
assertSentTo(0);
|
|
|
|
|
|
// Remove safe mode on observer, request should still go to it.
|
|
// Remove safe mode on observer, request should still go to it.
|
|
dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
|
dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
|
- dfs.open(testPath);
|
|
|
|
|
|
+ dfs.open(testPath).close();
|
|
assertSentTo(2);
|
|
assertSentTo(2);
|
|
- }
|
|
|
|
-
|
|
|
|
- // TODO this does not currently work because fetching the service state from
|
|
|
|
- // e.g. the StandbyNameNode also waits for the transaction ID to catch up.
|
|
|
|
- // This is disabled pending HDFS-13872 and HDFS-13749.
|
|
|
|
- @Ignore("Disabled until HDFS-13872 and HDFS-13749 are committed")
|
|
|
|
- @Test
|
|
|
|
- public void testMsyncSimple() throws Exception {
|
|
|
|
- // disable fast path here because this test's assertions are based on the
|
|
|
|
- // timing of explicitly called rollEditLogAndTail. Although this means this
|
|
|
|
- // test takes some time to run
|
|
|
|
- // TODO: revisit if there is a better way.
|
|
|
|
- conf.setBoolean(DFS_HA_TAILEDITS_INPROGRESS_KEY, false);
|
|
|
|
- conf.setTimeDuration(DFS_HA_LOGROLL_PERIOD_KEY, 60, TimeUnit.SECONDS);
|
|
|
|
- conf.setTimeDuration(
|
|
|
|
- DFS_HA_TAILEDITS_PERIOD_KEY, 30, TimeUnit.SECONDS);
|
|
|
|
- setUpCluster(1);
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
- // 0 == not completed, 1 == succeeded, -1 == failed
|
|
|
|
- AtomicInteger readStatus = new AtomicInteger(0);
|
|
|
|
-
|
|
|
|
- dfs.mkdir(testPath, FsPermission.getDefault());
|
|
|
|
- assertSentTo(0);
|
|
|
|
-
|
|
|
|
- Thread reader = new Thread(() -> {
|
|
|
|
- try {
|
|
|
|
- // this read will block until roll and tail edits happen.
|
|
|
|
- dfs.getFileStatus(testPath);
|
|
|
|
- readStatus.set(1);
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- e.printStackTrace();
|
|
|
|
- readStatus.set(-1);
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
-
|
|
|
|
- reader.start();
|
|
|
|
- // the reader is still blocking, not succeeded yet.
|
|
|
|
- assertEquals(0, readStatus.get());
|
|
|
|
- rollEditLogAndTail(0);
|
|
|
|
- // wait a while for all the change to be done
|
|
|
|
- GenericTestUtils.waitFor(() -> readStatus.get() != 0, 100, 10000);
|
|
|
|
- // the reader should have succeed.
|
|
|
|
- assertEquals(1, readStatus.get());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Test
|
|
|
|
- public void testUncoordinatedCall() throws Exception {
|
|
|
|
- // disable fast tailing so that coordination takes time.
|
|
|
|
- conf.setBoolean(DFS_HA_TAILEDITS_INPROGRESS_KEY, false);
|
|
|
|
- conf.setTimeDuration(DFS_HA_LOGROLL_PERIOD_KEY, 300, TimeUnit.SECONDS);
|
|
|
|
- conf.setTimeDuration(
|
|
|
|
- DFS_HA_TAILEDITS_PERIOD_KEY, 200, TimeUnit.SECONDS);
|
|
|
|
- setUpCluster(1);
|
|
|
|
- setObserverRead(true);
|
|
|
|
-
|
|
|
|
- // make a write call so that client will be ahead of
|
|
|
|
- // observer for now.
|
|
|
|
- dfs.mkdir(testPath, FsPermission.getDefault());
|
|
|
|
|
|
|
|
- // a status flag, initialized to 0, after reader finished, this will be
|
|
|
|
- // updated to 1, -1 on error
|
|
|
|
- AtomicInteger readStatus = new AtomicInteger(0);
|
|
|
|
-
|
|
|
|
- // create a separate thread to make a blocking read.
|
|
|
|
- Thread reader = new Thread(() -> {
|
|
|
|
- try {
|
|
|
|
- // this read call will block until server state catches up. But due to
|
|
|
|
- // configuration, this will take a very long time.
|
|
|
|
- dfs.getClient().getFileInfo("/");
|
|
|
|
- readStatus.set(1);
|
|
|
|
- fail("Should have been interrupted before getting here.");
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- e.printStackTrace();
|
|
|
|
- readStatus.set(-1);
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
- reader.start();
|
|
|
|
-
|
|
|
|
- long before = System.currentTimeMillis();
|
|
|
|
- dfs.getClient().datanodeReport(HdfsConstants.DatanodeReportType.ALL);
|
|
|
|
- long after = System.currentTimeMillis();
|
|
|
|
-
|
|
|
|
- // should succeed immediately, because datanodeReport is marked an
|
|
|
|
- // uncoordinated call, and will not be waiting for server to catch up.
|
|
|
|
- assertTrue(after - before < 200);
|
|
|
|
- // by this time, reader thread should still be blocking, so the status not
|
|
|
|
- // updated
|
|
|
|
- assertEquals(0, readStatus.get());
|
|
|
|
- Thread.sleep(5000);
|
|
|
|
- // reader thread status should still be unchanged after 5 sec...
|
|
|
|
- assertEquals(0, readStatus.get());
|
|
|
|
- // and the reader thread is not dead, so it must be still waiting
|
|
|
|
- assertEquals(Thread.State.WAITING, reader.getState());
|
|
|
|
- reader.interrupt();
|
|
|
|
|
|
+ Mockito.reset(bmSpy);
|
|
}
|
|
}
|
|
|
|
|
|
- private void setUpCluster(int numObservers) throws Exception {
|
|
|
|
- qjmhaCluster = new MiniQJMHACluster.Builder(conf)
|
|
|
|
- .setNumNameNodes(2 + numObservers)
|
|
|
|
- .build();
|
|
|
|
- dfsCluster = qjmhaCluster.getDfsCluster();
|
|
|
|
-
|
|
|
|
- namenodes = new NameNode[2 + numObservers];
|
|
|
|
- for (int i = 0; i < namenodes.length; i++) {
|
|
|
|
- namenodes[i] = dfsCluster.getNameNode(i);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- dfsCluster.transitionToActive(0);
|
|
|
|
- dfsCluster.waitActive(0);
|
|
|
|
-
|
|
|
|
- for (int i = 0; i < numObservers; i++) {
|
|
|
|
- dfsCluster.transitionToObserver(2 + i);
|
|
|
|
- }
|
|
|
|
|
|
+ private void assertSentTo(int nnIdx) throws IOException {
|
|
|
|
+ assertTrue("Request was not sent to the expected namenode " + nnIdx,
|
|
|
|
+ HATestUtil.isSentToAnyOfNameNodes(dfs, dfsCluster, nnIdx));
|
|
}
|
|
}
|
|
|
|
|
|
- private void assertSentTo(int nnIdx) {
|
|
|
|
- assertSentToAny(nnIdx);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void assertSentToAny(int... nnIndices) {
|
|
|
|
- FailoverProxyProvider.ProxyInfo<?> pi = provider.getLastProxy();
|
|
|
|
- for (int nnIdx : nnIndices) {
|
|
|
|
- if (pi.proxyInfo.equals(
|
|
|
|
- dfsCluster.getNameNode(nnIdx).getNameNodeAddress().toString())) {
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- fail("Request was not sent to any of the expected namenodes");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void setObserverRead(boolean flag) throws Exception {
|
|
|
|
- dfs = HATestUtil.configureObserverReadFs(dfsCluster, conf, 0);
|
|
|
|
- RetryInvocationHandler<?> handler =
|
|
|
|
- (RetryInvocationHandler<?>) Proxy.getInvocationHandler(
|
|
|
|
- dfs.getClient().getNamenode());
|
|
|
|
- provider = (ObserverReadProxyProvider<?>) handler.getProxyProvider();
|
|
|
|
- provider.setObserverReadEnabled(flag);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void rollEditLogAndTail(int indexForActiveNN) throws Exception {
|
|
|
|
- dfsCluster.getNameNode(indexForActiveNN).getRpcServer().rollEditLog();
|
|
|
|
- for (int i = 2; i < namenodes.length; i++) {
|
|
|
|
- dfsCluster.getNameNode(i).getNamesystem().getEditLogTailer()
|
|
|
|
- .doTailEdits();
|
|
|
|
- }
|
|
|
|
|
|
+ private static void setObserverRead(boolean flag) throws Exception {
|
|
|
|
+ dfs = HATestUtil.configureObserverReadFs(dfsCluster, conf, flag);
|
|
}
|
|
}
|
|
}
|
|
}
|