|
@@ -34,6 +34,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.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
|
|
|
import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
|
|
@@ -122,11 +123,17 @@ public class TestObserverWithRouter {
|
|
|
|
|
|
cluster.waitActiveNamespaces();
|
|
|
routerContext = cluster.getRandomRouter();
|
|
|
- fileSystem = routerContext.getFileSystemWithObserverReadsEnabled();
|
|
|
+ }
|
|
|
+
|
|
|
+ private static Configuration getConfToEnableObserverReads() {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean(HdfsClientConfigKeys.DFS_RBF_OBSERVER_READ_ENABLE, true);
|
|
|
+ return conf;
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testObserverRead() throws Exception {
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
internalTestObserverRead();
|
|
|
}
|
|
|
|
|
@@ -137,7 +144,6 @@ public class TestObserverWithRouter {
|
|
|
*/
|
|
|
@Test
|
|
|
public void testReadWithoutObserverClientConfigurations() throws Exception {
|
|
|
- fileSystem.close();
|
|
|
fileSystem = routerContext.getFileSystem();
|
|
|
assertThrows(AssertionError.class, this::internalTestObserverRead);
|
|
|
}
|
|
@@ -173,6 +179,7 @@ public class TestObserverWithRouter {
|
|
|
Configuration confOverrides = new Configuration(false);
|
|
|
confOverrides.setInt(RBFConfigKeys.DFS_ROUTER_OBSERVER_FEDERATED_STATE_PROPAGATION_MAXSIZE, 0);
|
|
|
startUpCluster(2, confOverrides);
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
List<? extends FederationNamenodeContext> namenodes = routerContext
|
|
|
.getRouter().getNamenodeResolver()
|
|
|
.getNamenodesForNameserviceId(cluster.getNameservices().get(0), true);
|
|
@@ -202,6 +209,7 @@ public class TestObserverWithRouter {
|
|
|
Configuration confOverrides = new Configuration(false);
|
|
|
confOverrides.set(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_OVERRIDES, "ns0");
|
|
|
startUpCluster(2, confOverrides);
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
|
|
|
Path path = new Path("/testFile");
|
|
|
fileSystem.create(path).close();
|
|
@@ -219,6 +227,7 @@ public class TestObserverWithRouter {
|
|
|
|
|
|
@Test
|
|
|
public void testReadWhenObserverIsDown() throws Exception {
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
Path path = new Path("/testFile1");
|
|
|
// Send Create call to active
|
|
|
fileSystem.create(path).close();
|
|
@@ -246,6 +255,7 @@ public class TestObserverWithRouter {
|
|
|
|
|
|
@Test
|
|
|
public void testMultipleObserver() throws Exception {
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
Path path = new Path("/testFile1");
|
|
|
// Send Create call to active
|
|
|
fileSystem.create(path).close();
|
|
@@ -384,6 +394,7 @@ public class TestObserverWithRouter {
|
|
|
|
|
|
@Test
|
|
|
public void testUnavailableObserverNN() throws Exception {
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
stopObserver(2);
|
|
|
|
|
|
Path path = new Path("/testFile");
|
|
@@ -417,10 +428,9 @@ public class TestObserverWithRouter {
|
|
|
assertTrue("There must be unavailable namenodes", hasUnavailable);
|
|
|
}
|
|
|
|
|
|
-
|
|
|
-
|
|
|
@Test
|
|
|
public void testRouterMsync() throws Exception {
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
Path path = new Path("/testFile");
|
|
|
|
|
|
// Send Create call to active
|
|
@@ -439,4 +449,60 @@ public class TestObserverWithRouter {
|
|
|
assertEquals("Four calls should be sent to active", 4,
|
|
|
rpcCountForActive);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testSingleRead() throws Exception {
|
|
|
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
|
|
|
+ List<? extends FederationNamenodeContext> namenodes = routerContext
|
|
|
+ .getRouter().getNamenodeResolver()
|
|
|
+ .getNamenodesForNameserviceId(cluster.getNameservices().get(0), true);
|
|
|
+ assertEquals("First namenode should be observer", namenodes.get(0).getState(),
|
|
|
+ FederationNamenodeServiceState.OBSERVER);
|
|
|
+ Path path = new Path("/");
|
|
|
+
|
|
|
+ long rpcCountForActive;
|
|
|
+ long rpcCountForObserver;
|
|
|
+
|
|
|
+ // Send read request
|
|
|
+ fileSystem.listFiles(path, false);
|
|
|
+ fileSystem.close();
|
|
|
+
|
|
|
+ rpcCountForActive = routerContext.getRouter().getRpcServer()
|
|
|
+ .getRPCMetrics().getActiveProxyOps();
|
|
|
+ // getListingCall sent to active.
|
|
|
+ assertEquals("Only one call should be sent to active", 1, rpcCountForActive);
|
|
|
+
|
|
|
+ rpcCountForObserver = routerContext.getRouter().getRpcServer()
|
|
|
+ .getRPCMetrics().getObserverProxyOps();
|
|
|
+ // getList call should be sent to observer
|
|
|
+ assertEquals("No calls should be sent to observer", 0, rpcCountForObserver);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testSingleReadUsingObserverReadProxyProvider() throws Exception {
|
|
|
+ fileSystem = routerContext.getFileSystemWithObserverReadProxyProvider();
|
|
|
+ List<? extends FederationNamenodeContext> namenodes = routerContext
|
|
|
+ .getRouter().getNamenodeResolver()
|
|
|
+ .getNamenodesForNameserviceId(cluster.getNameservices().get(0), true);
|
|
|
+ assertEquals("First namenode should be observer", namenodes.get(0).getState(),
|
|
|
+ FederationNamenodeServiceState.OBSERVER);
|
|
|
+ Path path = new Path("/");
|
|
|
+
|
|
|
+ long rpcCountForActive;
|
|
|
+ long rpcCountForObserver;
|
|
|
+
|
|
|
+ // Send read request
|
|
|
+ fileSystem.listFiles(path, false);
|
|
|
+ fileSystem.close();
|
|
|
+
|
|
|
+ rpcCountForActive = routerContext.getRouter().getRpcServer()
|
|
|
+ .getRPCMetrics().getActiveProxyOps();
|
|
|
+ // Two msync calls to the active namenodes.
|
|
|
+ assertEquals("Two calls should be sent to active", 2, rpcCountForActive);
|
|
|
+
|
|
|
+ rpcCountForObserver = routerContext.getRouter().getRpcServer()
|
|
|
+ .getRPCMetrics().getObserverProxyOps();
|
|
|
+ // getList call should be sent to observer
|
|
|
+ assertEquals("One call should be sent to observer", 1, rpcCountForObserver);
|
|
|
+ }
|
|
|
}
|