[ 
https://issues.apache.org/jira/browse/HDFS-17265?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17789847#comment-17789847
 ] 

ASF GitHub Bot commented on HDFS-17265:
---------------------------------------

goiri commented on code in PR #6298:
URL: https://github.com/apache/hadoop/pull/6298#discussion_r1405472006


##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/fairness/TestRouterHandlersFairness.java:
##########
@@ -100,6 +108,57 @@ public void testFairnessControlOn() throws Exception {
     startLoadTest(true);
   }
 
+  /**
+   * Ensure that the semaphore is not acquired,
+   * when invokeSequential or invokeConcurrent throws any exception.
+   */
+  @Test
+  public void testReleasedWhenExceptionOccurs() throws Exception{
+    setupCluster(true, false);
+    RouterContext routerContext = cluster.getRandomRouter();
+    RouterRpcClient rpcClient =
+        routerContext.getRouter().getRpcServer().getRPCClient();
+    // Mock an ActiveNamenodeResolver and inject it into RouterRpcClient,
+    // so RouterRpcClient's getOrderedNamenodes method will report an 
exception.
+    ActiveNamenodeResolver mockNamenodeResolver = 
mock(ActiveNamenodeResolver.class);
+    Field field = rpcClient.getClass().getDeclaredField("namenodeResolver");
+    field.setAccessible(true);
+    field.set(rpcClient, mockNamenodeResolver);
+
+    // Use getFileInfo test invokeSequential.
+    DFSClient client = routerContext.getClient();
+    int availablePermits =
+        
rpcClient.getRouterRpcFairnessPolicyController().getAvailablePermits("ns0");
+    try {
+      LOG.info("Use getFileInfo test invokeSequential.");
+      client.getFileInfo("/test.txt");

Review Comment:
   Do you always expect the exception?
   In that case, we should use LambdaTestUtils#intercept



##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/fairness/TestRouterHandlersFairness.java:
##########
@@ -100,6 +108,57 @@ public void testFairnessControlOn() throws Exception {
     startLoadTest(true);
   }
 
+  /**
+   * Ensure that the semaphore is not acquired,
+   * when invokeSequential or invokeConcurrent throws any exception.
+   */
+  @Test
+  public void testReleasedWhenExceptionOccurs() throws Exception{
+    setupCluster(true, false);
+    RouterContext routerContext = cluster.getRandomRouter();
+    RouterRpcClient rpcClient =
+        routerContext.getRouter().getRpcServer().getRPCClient();
+    // Mock an ActiveNamenodeResolver and inject it into RouterRpcClient,
+    // so RouterRpcClient's getOrderedNamenodes method will report an 
exception.
+    ActiveNamenodeResolver mockNamenodeResolver = 
mock(ActiveNamenodeResolver.class);
+    Field field = rpcClient.getClass().getDeclaredField("namenodeResolver");
+    field.setAccessible(true);
+    field.set(rpcClient, mockNamenodeResolver);
+
+    // Use getFileInfo test invokeSequential.
+    DFSClient client = routerContext.getClient();
+    int availablePermits =
+        
rpcClient.getRouterRpcFairnessPolicyController().getAvailablePermits("ns0");
+    try {
+      LOG.info("Use getFileInfo test invokeSequential.");
+      client.getFileInfo("/test.txt");
+    }catch (IOException ioe) {
+      assertExceptionContains("Cannot locate a registered namenode", ioe);
+    }
+    // Ensure that the semaphore is not acquired.
+    assertEquals(availablePermits,
+        
rpcClient.getRouterRpcFairnessPolicyController().getAvailablePermits("ns0"));
+
+    // Use renewLease test invokeConcurrent.
+    Collection<RemoteLocation> locations = new ArrayList<>();
+    locations.add(new RemoteLocation("ns0", "/", "/"));
+    RemoteMethod renewLease = new RemoteMethod(
+        "renewLease",
+        new Class[]{java.lang.String.class, java.util.List.class},
+        new Object[]{null, null});
+    availablePermits =
+        
rpcClient.getRouterRpcFairnessPolicyController().getAvailablePermits("ns0");
+    try {
+      LOG.info("Use renewLease test invokeConcurrent.");
+      rpcClient.invokeConcurrent(locations, renewLease);
+    }catch (IOException ioe) {

Review Comment:
   Spaces



##########
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/fairness/TestRouterHandlersFairness.java:
##########
@@ -100,6 +108,57 @@ public void testFairnessControlOn() throws Exception {
     startLoadTest(true);
   }
 
+  /**
+   * Ensure that the semaphore is not acquired,
+   * when invokeSequential or invokeConcurrent throws any exception.
+   */
+  @Test
+  public void testReleasedWhenExceptionOccurs() throws Exception{
+    setupCluster(true, false);
+    RouterContext routerContext = cluster.getRandomRouter();
+    RouterRpcClient rpcClient =
+        routerContext.getRouter().getRpcServer().getRPCClient();
+    // Mock an ActiveNamenodeResolver and inject it into RouterRpcClient,
+    // so RouterRpcClient's getOrderedNamenodes method will report an 
exception.
+    ActiveNamenodeResolver mockNamenodeResolver = 
mock(ActiveNamenodeResolver.class);
+    Field field = rpcClient.getClass().getDeclaredField("namenodeResolver");
+    field.setAccessible(true);
+    field.set(rpcClient, mockNamenodeResolver);
+
+    // Use getFileInfo test invokeSequential.
+    DFSClient client = routerContext.getClient();
+    int availablePermits =
+        
rpcClient.getRouterRpcFairnessPolicyController().getAvailablePermits("ns0");
+    try {
+      LOG.info("Use getFileInfo test invokeSequential.");
+      client.getFileInfo("/test.txt");
+    }catch (IOException ioe) {

Review Comment:
   Space





> RBF: Throwing an exception prevents the permit from being released when using 
> FairnessPolicyController
> ------------------------------------------------------------------------------------------------------
>
>                 Key: HDFS-17265
>                 URL: https://issues.apache.org/jira/browse/HDFS-17265
>             Project: Hadoop HDFS
>          Issue Type: Bug
>            Reporter: Jian Zhang
>            Assignee: Jian Zhang
>            Priority: Major
>              Labels: pull-request-available
>
> *Bug description*
> When the router uses FairnessPolicyController, each time a request is 
> processed,
> the permit of the ns corresponding to the request will be obtained first 
> {*}(method acquirePermit){*},
> and then the  information of namenodes corresponding to the ns will be 
> obtained{*}(method getOrderedNamenodes){*}.
> getOrderedNamenodes comes after acquirePermit, so if acquirePermit succeeds 
> but getOrderedNamenodes throws an exception, the permit cannot be released.
>  
> *How to reproduce*
> Use the original code to run the new unit test 
> testReleasedWhenExceptionOccurs in this PR
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to