|
@@ -178,8 +178,12 @@ public class TestConsistentReadsObserver {
|
|
|
// Therefore, the subsequent getFileStatus call should succeed.
|
|
|
dfs2.getClient().msync();
|
|
|
dfs2.getFileStatus(testPath);
|
|
|
- readStatus.set(1);
|
|
|
- } catch (IOException e) {
|
|
|
+ if (HATestUtil.isSentToAnyOfNameNodes(dfs2, dfsCluster, 2)) {
|
|
|
+ readStatus.set(1);
|
|
|
+ } else {
|
|
|
+ readStatus.set(-1);
|
|
|
+ }
|
|
|
+ } catch (Exception e) {
|
|
|
e.printStackTrace();
|
|
|
readStatus.set(-1);
|
|
|
}
|
|
@@ -196,6 +200,71 @@ public class TestConsistentReadsObserver {
|
|
|
assertEquals(1, readStatus.get());
|
|
|
}
|
|
|
|
|
|
+ // A new client should first contact the active, before using an observer,
|
|
|
+ // to ensure that it is up-to-date with the current state
|
|
|
+ @Test
|
|
|
+ public void testCallFromNewClient() throws Exception {
|
|
|
+ // Set the order of nodes: Observer, Standby, Active
|
|
|
+ // This is to ensure that test doesn't pass trivially because the active is
|
|
|
+ // the first node contacted
|
|
|
+ dfsCluster.transitionToStandby(0);
|
|
|
+ dfsCluster.transitionToObserver(0);
|
|
|
+ dfsCluster.transitionToStandby(2);
|
|
|
+ dfsCluster.transitionToActive(2);
|
|
|
+ try {
|
|
|
+ // 0 == not completed, 1 == succeeded, -1 == failed
|
|
|
+ AtomicInteger readStatus = new AtomicInteger(0);
|
|
|
+
|
|
|
+ // Initialize the proxies for Observer Node.
|
|
|
+ dfs.getClient().getHAServiceState();
|
|
|
+
|
|
|
+ // Advance Observer's state ID so it is ahead of client's.
|
|
|
+ dfs.mkdir(new Path("/test"), FsPermission.getDefault());
|
|
|
+ dfsCluster.getNameNode(2).getRpcServer().rollEditLog();
|
|
|
+ dfsCluster.getNameNode(0)
|
|
|
+ .getNamesystem().getEditLogTailer().doTailEdits();
|
|
|
+
|
|
|
+ dfs.mkdir(testPath, FsPermission.getDefault());
|
|
|
+ assertSentTo(2);
|
|
|
+
|
|
|
+ Configuration conf2 = new Configuration(conf);
|
|
|
+
|
|
|
+ // Disable FS cache so two different DFS clients will be used.
|
|
|
+ conf2.setBoolean("fs.hdfs.impl.disable.cache", true);
|
|
|
+ DistributedFileSystem dfs2 =
|
|
|
+ (DistributedFileSystem) FileSystem.get(conf2);
|
|
|
+ dfs2.getClient().getHAServiceState();
|
|
|
+
|
|
|
+ Thread reader = new Thread(() -> {
|
|
|
+ try {
|
|
|
+ dfs2.getFileStatus(testPath);
|
|
|
+ readStatus.set(1);
|
|
|
+ } catch (Exception e) {
|
|
|
+ e.printStackTrace();
|
|
|
+ readStatus.set(-1);
|
|
|
+ }
|
|
|
+ });
|
|
|
+
|
|
|
+ reader.start();
|
|
|
+
|
|
|
+ Thread.sleep(100);
|
|
|
+ assertEquals(0, readStatus.get());
|
|
|
+
|
|
|
+ dfsCluster.getNameNode(2).getRpcServer().rollEditLog();
|
|
|
+ dfsCluster.getNameNode(0)
|
|
|
+ .getNamesystem().getEditLogTailer().doTailEdits();
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> readStatus.get() != 0, 100, 10000);
|
|
|
+ assertEquals(1, readStatus.get());
|
|
|
+ } finally {
|
|
|
+ // Put the cluster back the way it was when the test started
|
|
|
+ dfsCluster.transitionToStandby(2);
|
|
|
+ dfsCluster.transitionToObserver(2);
|
|
|
+ dfsCluster.transitionToStandby(0);
|
|
|
+ dfsCluster.transitionToActive(0);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testUncoordinatedCall() throws Exception {
|
|
|
// make a write call so that client will be ahead of
|