浏览代码

HDFS-16302. RBF: RouterRpcFairnessPolicyController record requests accepted by each nameservice (#3621)

Symious 3 年之前
父节点
当前提交
d2b8d6b534

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java

@@ -132,4 +132,10 @@ public interface FederationRPCMBean {
    * @return Number of operations rejected due to lack of permits of each namespace.
    */
   String getProxyOpPermitRejectedPerNs();
+
+  /**
+   * Get the number of operations accepted of each namespace.
+   * @return Number of operations accepted of each namespace.
+   */
+  String getProxyOpPermitAcceptedPerNs();
 }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java

@@ -297,4 +297,9 @@ public class FederationRPCMetrics implements FederationRPCMBean {
   public String getProxyOpPermitRejectedPerNs() {
     return rpcServer.getRPCClient().getRejectedPermitsPerNsJSON();
   }
+
+  @Override
+  public String getProxyOpPermitAcceptedPerNs() {
+    return rpcServer.getRPCClient().getAcceptedPermitsPerNsJSON();
+  }
 }

+ 33 - 13
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java

@@ -136,6 +136,7 @@ public class RouterRpcClient {
   /** Fairness manager to control handlers assigned per NS. */
   private RouterRpcFairnessPolicyController routerRpcFairnessPolicyController;
   private Map<String, LongAdder> rejectedPermitsPerNs = new ConcurrentHashMap<>();
+  private Map<String, LongAdder> acceptedPermitsPerNs = new ConcurrentHashMap<>();
 
   /**
    * Create a router RPC client to manage remote procedure calls to NNs.
@@ -330,6 +331,14 @@ public class RouterRpcClient {
     return JSON.toString(rejectedPermitsPerNs);
   }
 
+  /**
+   * JSON representation of the accepted permits for each nameservice.
+   *
+   * @return String representation of the accepted permits for each nameservice.
+   */
+  public String getAcceptedPermitsPerNsJSON() {
+    return JSON.toString(acceptedPermitsPerNs);
+  }
   /**
    * Get ClientProtocol proxy client for a NameNode. Each combination of user +
    * NN must use a unique proxy client. Previously created clients are cached
@@ -1548,20 +1557,22 @@ public class RouterRpcClient {
   private void acquirePermit(
       final String nsId, final UserGroupInformation ugi, final RemoteMethod m)
       throws IOException {
-    if (routerRpcFairnessPolicyController != null
-        && !routerRpcFairnessPolicyController.acquirePermit(nsId)) {
-      // Throw StandByException,
-      // Clients could fail over and try another router.
-      if (rpcMonitor != null) {
-        rpcMonitor.getRPCMetrics().incrProxyOpPermitRejected();
+    if (routerRpcFairnessPolicyController != null) {
+      if (!routerRpcFairnessPolicyController.acquirePermit(nsId)) {
+        // Throw StandByException,
+        // Clients could fail over and try another router.
+        if (rpcMonitor != null) {
+          rpcMonitor.getRPCMetrics().incrProxyOpPermitRejected();
+        }
+        incrRejectedPermitForNs(nsId);
+        LOG.debug("Permit denied for ugi: {} for method: {}",
+            ugi, m.getMethodName());
+        String msg =
+            "Router " + router.getRouterId() +
+                " is overloaded for NS: " + nsId;
+        throw new StandbyException(msg);
       }
-      incrRejectedPermitForNs(nsId);
-      LOG.debug("Permit denied for ugi: {} for method: {}",
-          ugi, m.getMethodName());
-      String msg =
-          "Router " + router.getRouterId() +
-              " is overloaded for NS: " + nsId;
-      throw new StandbyException(msg);
+      incrAcceptedPermitForNs(nsId);
     }
   }
 
@@ -1596,4 +1607,13 @@ public class RouterRpcClient {
     return rejectedPermitsPerNs.containsKey(ns) ?
         rejectedPermitsPerNs.get(ns).longValue() : 0L;
   }
+
+  private void incrAcceptedPermitForNs(String ns) {
+    acceptedPermitsPerNs.computeIfAbsent(ns, k -> new LongAdder()).increment();
+  }
+
+  public Long getAcceptedPermitForNs(String ns) {
+    return acceptedPermitsPerNs.containsKey(ns) ?
+        acceptedPermitsPerNs.get(ns).longValue() : 0L;
+  }
 }

+ 66 - 36
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/fairness/TestRouterHandlersFairness.java

@@ -41,8 +41,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Test the Router handlers fairness control rejects
- * requests when the handlers are overloaded.
+ * Test the Router handlers fairness control rejects and accepts requests.
  */
 public class TestRouterHandlersFairness {
 
@@ -126,6 +125,12 @@ public class TestRouterHandlersFairness {
       throws Exception {
 
     RouterContext routerContext = cluster.getRandomRouter();
+    URI address = routerContext.getFileSystemURI();
+    Configuration conf = new HdfsConfiguration();
+    final int numOps = 10;
+    AtomicInteger overloadException = new AtomicInteger();
+
+    // Test when handlers are overloaded
     if (fairness) {
       if (isConcurrent) {
         LOG.info("Taking fanout lock first");
@@ -142,42 +147,11 @@ public class TestRouterHandlersFairness {
         }
       }
     }
-    URI address = routerContext.getFileSystemURI();
-    Configuration conf = new HdfsConfiguration();
-    final int numOps = 10;
-    final AtomicInteger overloadException = new AtomicInteger();
     int originalRejectedPermits = getTotalRejectedPermits(routerContext);
 
-    for (int i = 0; i < numOps; i++) {
-      DFSClient routerClient = null;
-      try {
-        routerClient = new DFSClient(address, conf);
-        String clientName = routerClient.getClientName();
-        ClientProtocol routerProto = routerClient.getNamenode();
-        if (isConcurrent) {
-          invokeConcurrent(routerProto, clientName);
-        } else {
-          invokeSequential(routerProto);
-        }
-      } catch (RemoteException re) {
-        IOException ioe = re.unwrapRemoteException();
-        assertTrue("Wrong exception: " + ioe,
-            ioe instanceof StandbyException);
-        assertExceptionContains("is overloaded for NS", ioe);
-        overloadException.incrementAndGet();
-      } catch (Throwable e) {
-        throw e;
-      } finally {
-        if (routerClient != null) {
-          try {
-            routerClient.close();
-          } catch (IOException e) {
-            LOG.error("Cannot close the client");
-          }
-        }
-      }
-      overloadException.get();
-    }
+    // |- All calls should fail since permits not released
+    innerCalls(address, numOps, isConcurrent, conf, overloadException);
+
     int latestRejectedPermits = getTotalRejectedPermits(routerContext);
     assertEquals(latestRejectedPermits - originalRejectedPermits,
         overloadException.get());
@@ -201,6 +175,17 @@ public class TestRouterHandlersFairness {
       assertEquals("Number of failed RPCs without fairness configured",
           0, overloadException.get());
     }
+
+    // Test when handlers are not overloaded
+    int originalAcceptedPermits = getTotalAcceptedPermits(routerContext);
+    overloadException = new AtomicInteger();
+
+    // |- All calls should succeed since permits not acquired
+    innerCalls(address, numOps, isConcurrent, conf, overloadException);
+
+    int latestAcceptedPermits = getTotalAcceptedPermits(routerContext);
+    assertEquals(latestAcceptedPermits - originalAcceptedPermits, numOps);
+    assertEquals(overloadException.get(), 0);
   }
 
   private void invokeSequential(ClientProtocol routerProto) throws IOException {
@@ -222,4 +207,49 @@ public class TestRouterHandlersFairness {
         .getRejectedPermitForNs(RouterRpcFairnessConstants.CONCURRENT_NS);
     return totalRejectedPermits;
   }
+
+  private int getTotalAcceptedPermits(RouterContext routerContext) {
+    int totalAcceptedPermits = 0;
+    for (String ns : cluster.getNameservices()) {
+      totalAcceptedPermits += routerContext.getRouterRpcClient()
+          .getAcceptedPermitForNs(ns);
+    }
+    totalAcceptedPermits += routerContext.getRouterRpcClient()
+        .getAcceptedPermitForNs(RouterRpcFairnessConstants.CONCURRENT_NS);
+    return totalAcceptedPermits;
+  }
+
+  private void innerCalls(URI address, int numOps, boolean isConcurrent,
+      Configuration conf, AtomicInteger overloadException) throws IOException {
+    for (int i = 0; i < numOps; i++) {
+      DFSClient routerClient = null;
+      try {
+        routerClient = new DFSClient(address, conf);
+        String clientName = routerClient.getClientName();
+        ClientProtocol routerProto = routerClient.getNamenode();
+        if (isConcurrent) {
+          invokeConcurrent(routerProto, clientName);
+        } else {
+          invokeSequential(routerProto);
+        }
+      } catch (RemoteException re) {
+        IOException ioe = re.unwrapRemoteException();
+        assertTrue("Wrong exception: " + ioe,
+            ioe instanceof StandbyException);
+        assertExceptionContains("is overloaded for NS", ioe);
+        overloadException.incrementAndGet();
+      } catch (Throwable e) {
+        throw e;
+      } finally {
+        if (routerClient != null) {
+          try {
+            routerClient.close();
+          } catch (IOException e) {
+            LOG.error("Cannot close the client");
+          }
+        }
+      }
+      overloadException.get();
+    }
+  }
 }