You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/08/09 04:36:30 UTC

[4/9] git commit: HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)

HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c2bfc647
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c2bfc647
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c2bfc647

Branch: refs/heads/master
Commit: c2bfc64719718a8d2743cad91a25a842389062df
Parents: 3c03dcb
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 18:41:18 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 18:41:18 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c2bfc647/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 33474d4..c364786 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -110,7 +110,6 @@ public class CallRunner {
         RequestContext.clear();
       }
       RpcServer.CurCall.set(null);
-      this.rpcServer.addCallSize(call.getSize() * -1);
       // Set the response for undelayed calls and delayed calls with
       // undelayed responses.
       if (!call.isDelayed() || !call.isReturnValueDelayed()) {
@@ -139,6 +138,9 @@ public class CallRunner {
     } catch (Exception e) {
       RpcServer.LOG.warn(Thread.currentThread().getName()
           + ": caught: " + StringUtils.stringifyException(e));
+    } finally {
+      // regardless if succesful or not we need to reset the callQueueSize
+      this.rpcServer.addCallSize(call.getSize() * -1);
     }
   }