|
@@ -17,6 +17,8 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.federation.router;
|
|
|
|
|
|
+import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.ACTIVE;
|
|
|
+import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.STANDBY;
|
|
|
import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.simulateSlowNamenode;
|
|
|
import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.simulateThrowExceptionRouterRpcServer;
|
|
|
import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.transitionClusterNSToStandby;
|
|
@@ -42,16 +44,19 @@ import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
+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;
|
|
|
import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics;
|
|
|
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
|
|
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
|
+import org.apache.hadoop.util.Time;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
@@ -357,6 +362,82 @@ public class TestRouterClientRejectOverload {
|
|
|
assertEquals(originalRouter0Failures, rpcMetrics0.getProxyOpNoNamenodes());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * When failover occurs, the router may record that the ns has no active namenode.
|
|
|
+ * Only when the router updates the cache next time can the memory status be updated,
|
|
|
+ * causing the router to report NoNamenodesAvailableException for a long time.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testNoNamenodesAvailableLongTimeWhenNsFailover() throws Exception {
|
|
|
+ setupCluster(false, true);
|
|
|
+ transitionClusterNSToStandby(cluster);
|
|
|
+ for (RouterContext routerContext : cluster.getRouters()) {
|
|
|
+ // Manually trigger the heartbeat
|
|
|
+ Collection<NamenodeHeartbeatService> heartbeatServices = routerContext
|
|
|
+ .getRouter().getNamenodeHeartbeatServices();
|
|
|
+ for (NamenodeHeartbeatService service : heartbeatServices) {
|
|
|
+ service.periodicInvoke();
|
|
|
+ }
|
|
|
+ // Update service cache
|
|
|
+ routerContext.getRouter().getStateStore().refreshCaches(true);
|
|
|
+ }
|
|
|
+ // Record the time after the router first updated the cache
|
|
|
+ long firstLoadTime = Time.now();
|
|
|
+ List<MiniRouterDFSCluster.NamenodeContext> namenodes = cluster.getNamenodes();
|
|
|
+
|
|
|
+ // Make sure all namenodes are in standby state
|
|
|
+ for (MiniRouterDFSCluster.NamenodeContext namenodeContext : namenodes) {
|
|
|
+ assertEquals(STANDBY.ordinal(), namenodeContext.getNamenode().getNameNodeState());
|
|
|
+ }
|
|
|
+
|
|
|
+ Configuration conf = cluster.getRouterClientConf();
|
|
|
+ // Set dfs.client.failover.random.order false, to pick 1st router at first
|
|
|
+ conf.setBoolean("dfs.client.failover.random.order", false);
|
|
|
+
|
|
|
+ DFSClient routerClient = new DFSClient(new URI("hdfs://fed"), conf);
|
|
|
+
|
|
|
+ for (RouterContext routerContext : cluster.getRouters()) {
|
|
|
+ // Get the second namenode in the router cache and make it active
|
|
|
+ List<? extends FederationNamenodeContext> ns0 = routerContext.getRouter()
|
|
|
+ .getNamenodeResolver()
|
|
|
+ .getNamenodesForNameserviceId("ns0", false);
|
|
|
+
|
|
|
+ String nsId = ns0.get(1).getNamenodeId();
|
|
|
+ cluster.switchToActive("ns0", nsId);
|
|
|
+ // Manually trigger the heartbeat, but the router does not manually load the cache
|
|
|
+ Collection<NamenodeHeartbeatService> heartbeatServices = routerContext
|
|
|
+ .getRouter().getNamenodeHeartbeatServices();
|
|
|
+ for (NamenodeHeartbeatService service : heartbeatServices) {
|
|
|
+ service.periodicInvoke();
|
|
|
+ }
|
|
|
+ assertEquals(ACTIVE.ordinal(),
|
|
|
+ cluster.getNamenode("ns0", nsId).getNamenode().getNameNodeState());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Get router0 metrics
|
|
|
+ FederationRPCMetrics rpcMetrics0 = cluster.getRouters().get(0)
|
|
|
+ .getRouter().getRpcServer().getRPCMetrics();
|
|
|
+ // Original failures
|
|
|
+ long originalRouter0Failures = rpcMetrics0.getProxyOpNoNamenodes();
|
|
|
+
|
|
|
+ /*
|
|
|
+ * At this time, the router has recorded 2 standby namenodes in memory,
|
|
|
+ * and the first accessed namenode is indeed standby,
|
|
|
+ * then an NoNamenodesAvailableException will be reported for the first access,
|
|
|
+ * and the next access will be successful.
|
|
|
+ */
|
|
|
+ routerClient.getFileInfo("/");
|
|
|
+ long successReadTime = Time.now();
|
|
|
+ assertEquals(originalRouter0Failures + 1, rpcMetrics0.getProxyOpNoNamenodes());
|
|
|
+
|
|
|
+ /*
|
|
|
+ * access the active namenode without waiting for the router to update the cache,
|
|
|
+ * even if there are 2 standby states recorded in the router memory.
|
|
|
+ */
|
|
|
+ assertTrue(successReadTime - firstLoadTime < cluster.getCacheFlushInterval());
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
@Test
|
|
|
public void testAsyncCallerPoolMetrics() throws Exception {
|
|
|
setupCluster(true, false);
|