hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject hbase git commit: HBASE-17445 Count size of serialized exceptions in checking max result size quota
Date Tue, 17 Jan 2017 16:46:26 GMT
Repository: hbase
Updated Branches:
  refs/heads/branch-1.3 eb19a6748 -> dc72e0d4f


HBASE-17445 Count size of serialized exceptions in checking max result size quota


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

Branch: refs/heads/branch-1.3
Commit: dc72e0d4f802fd0f4902e4552e3c0eada1c49d51
Parents: eb19a67
Author: tedyu <yuzhihong@gmail.com>
Authored: Tue Jan 17 08:46:33 2017 -0800
Committer: tedyu <yuzhihong@gmail.com>
Committed: Tue Jan 17 08:46:33 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcCallContext.java |  3 +++
 .../java/org/apache/hadoop/hbase/ipc/RpcServer.java | 11 +++++++++++
 .../hadoop/hbase/regionserver/RSRpcServices.java    | 16 ++++++++++++----
 3 files changed, 26 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dc72e0d4/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
index b08afc2..4c66072 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
@@ -74,6 +74,9 @@ public interface RpcCallContext {
    */
   long getResponseCellSize();
 
+  long getResponseExceptionSize();
+  void incrementResponseExceptionSize(long exceptionSize);
+
   /**
    * Add on the given amount to the retained cell size.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/dc72e0d4/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 8b6ee75..9905638 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -348,6 +348,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver
{
 
     private long responseCellSize = 0;
     private long responseBlockSize = 0;
+    // cumulative size of serialized exceptions
+    private long exceptionSize = 0;
     private boolean retryImmediatelySupported;
 
     @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
@@ -562,6 +564,15 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver
{
       }
     }
 
+    @Override
+    public long getResponseExceptionSize() {
+      return exceptionSize;
+    }
+    @Override
+    public void incrementResponseExceptionSize(long exSize) {
+      exceptionSize += exSize;
+    }
+
     public long getSize() {
       return this.size;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dc72e0d4/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index acc0684..704b6ef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -154,6 +154,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
@@ -655,7 +656,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         if (context != null
             && context.isRetryImmediatelySupported()
             && (context.getResponseCellSize() > maxQuotaResultSize
-              || context.getResponseBlockSize() > maxQuotaResultSize)) {
+              || context.getResponseBlockSize() + context.getResponseExceptionSize()
+              > maxQuotaResultSize)) {
 
           // We're storing the exception since the exception and reason string won't
           // change after the response size limit is reached.
@@ -678,7 +680,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           //
           // This will create a copy in the builder.
           hasResultOrException = true;
-          resultOrExceptionBuilder.setException(ResponseConverter.buildException(sizeIOE));
+          NameBytesPair pair = ResponseConverter.buildException(sizeIOE);
+          resultOrExceptionBuilder.setException(pair);
+          context.incrementResponseExceptionSize(pair.getSerializedSize());
           resultOrExceptionBuilder.setIndex(action.getIndex());
           builder.addResultOrException(resultOrExceptionBuilder.build());
           if (cellScanner != null) {
@@ -710,7 +714,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                     .setValue(result.toByteString())));
           } catch (IOException ioe) {
             rpcServer.getMetrics().exception(ioe);
-            resultOrExceptionBuilder.setException(ResponseConverter.buildException(ioe));
+            NameBytesPair pair = ResponseConverter.buildException(ioe);
+            resultOrExceptionBuilder.setException(pair);
+            context.incrementResponseExceptionSize(pair.getSerializedSize());
           }
         } else if (action.hasMutation()) {
           MutationType type = action.getMutation().getMutateType();
@@ -764,7 +770,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       } catch (IOException ie) {
         rpcServer.getMetrics().exception(ie);
         hasResultOrException = true;
-        resultOrExceptionBuilder.setException(ResponseConverter.buildException(ie));
+        NameBytesPair pair = ResponseConverter.buildException(ie);
+        resultOrExceptionBuilder.setException(pair);
+        context.incrementResponseExceptionSize(pair.getSerializedSize());
       }
       if (hasResultOrException) {
         // Propagate index.


Mime
View raw message