|
@@ -17,127 +17,251 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.federation.router;
|
|
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
|
|
|
-import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_MONITOR_NAMENODE;
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
+import static java.util.Arrays.asList;
|
|
|
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.TreeSet;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
+import org.apache.hadoop.hdfs.server.federation.MockNamenode;
|
|
|
import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
|
|
|
-import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.NamenodeContext;
|
|
|
-import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
|
|
|
-import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
|
|
|
import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
|
|
|
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
|
|
|
import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
|
|
|
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
|
|
|
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
/**
|
|
|
* Test namenodes monitor behavior in the Router.
|
|
|
*/
|
|
|
public class TestRouterNamenodeMonitoring {
|
|
|
|
|
|
- private static StateStoreDFSCluster cluster;
|
|
|
- private static RouterContext routerContext;
|
|
|
- private static MembershipNamenodeResolver resolver;
|
|
|
+ private static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(TestRouterNamenodeMonitoring.class);
|
|
|
|
|
|
- private String ns0;
|
|
|
- private String ns1;
|
|
|
- private long initializedTime;
|
|
|
|
|
|
- @Before
|
|
|
- public void setUp() throws Exception {
|
|
|
- // Build and start a federated cluster with HA enabled
|
|
|
- cluster = new StateStoreDFSCluster(true, 2);
|
|
|
- // Enable heartbeat service and local heartbeat
|
|
|
- Configuration routerConf = new RouterConfigBuilder()
|
|
|
- .stateStore()
|
|
|
- .admin()
|
|
|
- .rpc()
|
|
|
- .enableLocalHeartbeat(true)
|
|
|
- .heartbeat()
|
|
|
- .build();
|
|
|
+ /** Router for the test. */
|
|
|
+ private Router router;
|
|
|
+ /** Namenodes in the cluster. */
|
|
|
+ private Map<String, Map<String, MockNamenode>> nns = new HashMap<>();
|
|
|
+ /** Nameservices in the federated cluster. */
|
|
|
+ private List<String> nsIds = asList("ns0", "ns1");
|
|
|
|
|
|
- // Specify local node (ns0.nn1) to monitor
|
|
|
- StringBuilder sb = new StringBuilder();
|
|
|
- ns0 = cluster.getNameservices().get(0);
|
|
|
- NamenodeContext context = cluster.getNamenodes(ns0).get(1);
|
|
|
- routerConf.set(DFS_NAMESERVICE_ID, ns0);
|
|
|
- routerConf.set(DFS_HA_NAMENODE_ID_KEY, context.getNamenodeId());
|
|
|
+ /** Time the test starts. */
|
|
|
+ private long initializedTime;
|
|
|
|
|
|
- // Specify namenodes (ns1.nn0,ns1.nn1) to monitor
|
|
|
- sb = new StringBuilder();
|
|
|
- ns1 = cluster.getNameservices().get(1);
|
|
|
- for (NamenodeContext ctx : cluster.getNamenodes(ns1)) {
|
|
|
- String suffix = ctx.getConfSuffix();
|
|
|
- if (sb.length() != 0) {
|
|
|
- sb.append(",");
|
|
|
+
|
|
|
+ @Before
|
|
|
+ public void setup() throws Exception {
|
|
|
+ LOG.info("Initialize the Mock Namenodes to monitor");
|
|
|
+ for (String nsId : nsIds) {
|
|
|
+ nns.put(nsId, new HashMap<>());
|
|
|
+ for (String nnId : asList("nn0", "nn1")) {
|
|
|
+ nns.get(nsId).put(nnId, new MockNamenode());
|
|
|
}
|
|
|
- sb.append(suffix);
|
|
|
}
|
|
|
- // override with the namenodes: ns1.nn0,ns1.nn1
|
|
|
- routerConf.set(DFS_ROUTER_MONITOR_NAMENODE, sb.toString());
|
|
|
|
|
|
- cluster.addRouterOverrides(routerConf);
|
|
|
- cluster.startCluster();
|
|
|
- cluster.startRouters();
|
|
|
- cluster.waitClusterUp();
|
|
|
+ LOG.info("Set nn0 to active for all nameservices");
|
|
|
+ for (Map<String, MockNamenode> nnNS : nns.values()) {
|
|
|
+ nnNS.get("nn0").transitionToActive();
|
|
|
+ nnNS.get("nn1").transitionToStandby();
|
|
|
+ }
|
|
|
|
|
|
- routerContext = cluster.getRandomRouter();
|
|
|
- resolver = (MembershipNamenodeResolver) routerContext.getRouter()
|
|
|
- .getNamenodeResolver();
|
|
|
initializedTime = Time.now();
|
|
|
}
|
|
|
|
|
|
@After
|
|
|
- public void tearDown() {
|
|
|
- if (cluster != null) {
|
|
|
- cluster.stopRouter(routerContext);
|
|
|
- cluster.shutdown();
|
|
|
- cluster = null;
|
|
|
+ public void cleanup() throws Exception {
|
|
|
+ for (Map<String, MockNamenode> nnNS : nns.values()) {
|
|
|
+ for (MockNamenode nn : nnNS.values()) {
|
|
|
+ nn.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ nns.clear();
|
|
|
+
|
|
|
+ if (router != null) {
|
|
|
+ router.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the configuration of the cluster which contains all the Namenodes and
|
|
|
+ * their addresses.
|
|
|
+ * @return Configuration containing all the Namenodes.
|
|
|
+ */
|
|
|
+ private Configuration getNamenodesConfig() {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ conf.set(DFSConfigKeys.DFS_NAMESERVICES,
|
|
|
+ StringUtils.join(",", nns.keySet()));
|
|
|
+ for (String nsId : nns.keySet()) {
|
|
|
+ Set<String> nnIds = nns.get(nsId).keySet();
|
|
|
+
|
|
|
+ StringBuilder sb = new StringBuilder();
|
|
|
+ sb.append(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX);
|
|
|
+ sb.append(".").append(nsId);
|
|
|
+ conf.set(sb.toString(), StringUtils.join(",", nnIds));
|
|
|
+
|
|
|
+ for (String nnId : nnIds) {
|
|
|
+ final MockNamenode nn = nns.get(nsId).get(nnId);
|
|
|
+
|
|
|
+ sb = new StringBuilder();
|
|
|
+ sb.append(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
|
|
|
+ sb.append(".").append(nsId);
|
|
|
+ sb.append(".").append(nnId);
|
|
|
+ conf.set(sb.toString(), "localhost:" + nn.getRPCPort());
|
|
|
+
|
|
|
+ sb = new StringBuilder();
|
|
|
+ sb.append(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
|
|
|
+ sb.append(".").append(nsId);
|
|
|
+ sb.append(".").append(nnId);
|
|
|
+ conf.set(sb.toString(), "localhost:" + nn.getHTTPPort());
|
|
|
+ }
|
|
|
}
|
|
|
+ return conf;
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testNamenodeMonitoring() throws Exception {
|
|
|
- // Set nn0 to active for all nameservices
|
|
|
- for (String ns : cluster.getNameservices()) {
|
|
|
- cluster.switchToActive(ns, "nn0");
|
|
|
- cluster.switchToStandby(ns, "nn1");
|
|
|
- }
|
|
|
+ Configuration nsConf = getNamenodesConfig();
|
|
|
|
|
|
- Collection<NamenodeHeartbeatService> heartbeatServices = routerContext
|
|
|
- .getRouter().getNamenodeHeartbeatServices();
|
|
|
- // manually trigger the heartbeat
|
|
|
+ // Setup the State Store for the Router to use
|
|
|
+ Configuration stateStoreConfig = getStateStoreConfiguration();
|
|
|
+ stateStoreConfig.setClass(
|
|
|
+ RBFConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
|
|
|
+ MembershipNamenodeResolver.class, ActiveNamenodeResolver.class);
|
|
|
+ stateStoreConfig.setClass(
|
|
|
+ RBFConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
|
|
|
+ MountTableResolver.class, FileSubclusterResolver.class);
|
|
|
+
|
|
|
+ Configuration routerConf = new RouterConfigBuilder(nsConf)
|
|
|
+ .enableLocalHeartbeat(true)
|
|
|
+ .heartbeat()
|
|
|
+ .stateStore()
|
|
|
+ .rpc()
|
|
|
+ .build();
|
|
|
+
|
|
|
+ // Specify namenodes (ns1.nn0,ns1.nn1) to monitor
|
|
|
+ routerConf.set(RBFConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY, "0.0.0.0:0");
|
|
|
+ routerConf.set(RBFConfigKeys.DFS_ROUTER_MONITOR_NAMENODE,
|
|
|
+ "ns1.nn0,ns1.nn1");
|
|
|
+ routerConf.addResource(stateStoreConfig);
|
|
|
+
|
|
|
+ // Specify local node (ns0.nn1) to monitor
|
|
|
+ routerConf.set(DFSConfigKeys.DFS_NAMESERVICE_ID, "ns0");
|
|
|
+ routerConf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
|
|
|
+
|
|
|
+ // Start the Router with the namenodes to monitor
|
|
|
+ router = new Router();
|
|
|
+ router.init(routerConf);
|
|
|
+ router.start();
|
|
|
+
|
|
|
+ // Manually trigger the heartbeat and update the values
|
|
|
+ Collection<NamenodeHeartbeatService> heartbeatServices =
|
|
|
+ router.getNamenodeHeartbeatServices();
|
|
|
for (NamenodeHeartbeatService service : heartbeatServices) {
|
|
|
service.periodicInvoke();
|
|
|
}
|
|
|
-
|
|
|
+ MembershipNamenodeResolver resolver =
|
|
|
+ (MembershipNamenodeResolver) router.getNamenodeResolver();
|
|
|
resolver.loadCache(true);
|
|
|
- List<? extends FederationNamenodeContext> namespaceInfo0 =
|
|
|
- resolver.getNamenodesForNameserviceId(ns0);
|
|
|
- List<? extends FederationNamenodeContext> namespaceInfo1 =
|
|
|
- resolver.getNamenodesForNameserviceId(ns1);
|
|
|
|
|
|
- // The modified date won't be updated in ns0.nn0 since it isn't
|
|
|
- // monitored by the Router.
|
|
|
- assertEquals("nn0", namespaceInfo0.get(1).getNamenodeId());
|
|
|
- assertTrue(namespaceInfo0.get(1).getDateModified() < initializedTime);
|
|
|
+ // Check that the monitored values are expected
|
|
|
+ final List<FederationNamenodeContext> namespaceInfo = new ArrayList<>();
|
|
|
+ for (String nsId : nns.keySet()) {
|
|
|
+ List<? extends FederationNamenodeContext> nnReports =
|
|
|
+ resolver.getNamenodesForNameserviceId(nsId);
|
|
|
+ namespaceInfo.addAll(nnReports);
|
|
|
+ }
|
|
|
+ for (FederationNamenodeContext nnInfo : namespaceInfo) {
|
|
|
+ long modTime = nnInfo.getDateModified();
|
|
|
+ long diff = modTime - initializedTime;
|
|
|
+ if ("ns0".equals(nnInfo.getNameserviceId()) &&
|
|
|
+ "nn0".equals(nnInfo.getNamenodeId())) {
|
|
|
+ // The modified date won't be updated in ns0.nn0
|
|
|
+ // since it isn't monitored by the Router.
|
|
|
+ assertTrue(nnInfo + " shouldn't be updated: " + diff,
|
|
|
+ modTime < initializedTime);
|
|
|
+ } else {
|
|
|
+ // other namnodes should be updated as expected
|
|
|
+ assertTrue(nnInfo + " should be updated: " + diff,
|
|
|
+ modTime > initializedTime);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testNamenodeMonitoringConfig() throws Exception {
|
|
|
+ testConfig(asList(), "");
|
|
|
+ testConfig(asList("ns1.nn0"), "ns1.nn0");
|
|
|
+ testConfig(asList("ns1.nn0", "ns1.nn1"), "ns1.nn0,ns1.nn1");
|
|
|
+ testConfig(asList("ns1.nn0", "ns1.nn1"), "ns1.nn0, ns1.nn1");
|
|
|
+ testConfig(asList("ns1.nn0", "ns1.nn1"), " ns1.nn0,ns1.nn1");
|
|
|
+ testConfig(asList("ns1.nn0", "ns1.nn1"), "ns1.nn0,ns1.nn1,");
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test if configuring a Router to monitor particular Namenodes actually
|
|
|
+ * takes effect.
|
|
|
+ * @param expectedNNs Namenodes that should be monitored.
|
|
|
+ * @param confNsIds Router configuration setting for Namenodes to monitor.
|
|
|
+ */
|
|
|
+ private void testConfig(
|
|
|
+ Collection<String> expectedNNs, String confNsIds) {
|
|
|
+
|
|
|
+ // Setup and start the Router
|
|
|
+ Configuration conf = getNamenodesConfig();
|
|
|
+ Configuration routerConf = new RouterConfigBuilder(conf)
|
|
|
+ .heartbeat(true)
|
|
|
+ .build();
|
|
|
+ routerConf.set(RBFConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY, "0.0.0.0:0");
|
|
|
+ routerConf.set(RBFConfigKeys.DFS_ROUTER_MONITOR_NAMENODE, confNsIds);
|
|
|
+ router = new Router();
|
|
|
+ router.init(routerConf);
|
|
|
|
|
|
- // other namnodes should be updated as expected
|
|
|
- assertEquals("nn1", namespaceInfo0.get(0).getNamenodeId());
|
|
|
- assertTrue(namespaceInfo0.get(0).getDateModified() > initializedTime);
|
|
|
+ // Test the heartbeat services of the Router
|
|
|
+ Collection<NamenodeHeartbeatService> heartbeatServices =
|
|
|
+ router.getNamenodeHeartbeatServices();
|
|
|
+ assertNamenodeHeartbeatService(expectedNNs, heartbeatServices);
|
|
|
+ }
|
|
|
|
|
|
- assertEquals("nn0", namespaceInfo1.get(0).getNamenodeId());
|
|
|
- assertTrue(namespaceInfo1.get(0).getDateModified() > initializedTime);
|
|
|
+ /**
|
|
|
+ * Assert that the namenodes monitored by the Router are the expected.
|
|
|
+ * @param expected Expected namenodes.
|
|
|
+ * @param actual Actual heartbeat services for the Router
|
|
|
+ */
|
|
|
+ private static void assertNamenodeHeartbeatService(
|
|
|
+ Collection<String> expected,
|
|
|
+ Collection<NamenodeHeartbeatService> actual) {
|
|
|
|
|
|
- assertEquals("nn1", namespaceInfo1.get(1).getNamenodeId());
|
|
|
- assertTrue(namespaceInfo1.get(1).getDateModified() > initializedTime);
|
|
|
+ final Set<String> actualSet = new TreeSet<>();
|
|
|
+ for (NamenodeHeartbeatService heartbeatService : actual) {
|
|
|
+ NamenodeStatusReport report = heartbeatService.getNamenodeStatusReport();
|
|
|
+ StringBuilder sb = new StringBuilder();
|
|
|
+ sb.append(report.getNameserviceId());
|
|
|
+ sb.append(".");
|
|
|
+ sb.append(report.getNamenodeId());
|
|
|
+ actualSet.add(sb.toString());
|
|
|
+ }
|
|
|
+ assertTrue(expected + " does not contain all " + actualSet,
|
|
|
+ expected.containsAll(actualSet));
|
|
|
+ assertTrue(actualSet + " does not contain all " + expected,
|
|
|
+ actualSet.containsAll(expected));
|
|
|
}
|
|
|
}
|