HBASE-15437 Response size calculated in RPCServer for warning tooLarge responses does NOT count CellScanner payload

This commit is contained in:
Jerry He 2016-12-07 14:47:10 -08:00
parent 6f25f838c0
commit 75567f828c
2 changed files with 18 additions and 16 deletions

View File

@ -2659,8 +2659,13 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
" processingTime: " + processingTime +
" totalTime: " + totalTime);
}
long requestSize = param.getSerializedSize();
// Use the raw request call size for now.
long requestSize = call.getSize();
long responseSize = result.getSerializedSize();
if (call.isClientCellBlockSupported()) {
// Include the payload size in HBaseRpcController
responseSize += call.getResponseCellSize();
}
metrics.dequeuedCall(qTime);
metrics.processedCall(processingTime);

View File

@ -460,22 +460,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
/**
* @return True if current call supports cellblocks
*/
private boolean isClientCellBlockSupport() {
RpcCallContext context = RpcServer.getCurrentCall();
return context != null && context.isClientCellBlockSupported();
}
private boolean isClientCellBlockSupport(RpcCallContext context) {
return context != null && context.isClientCellBlockSupported();
}
private void addResult(final MutateResponse.Builder builder, final Result result,
final HBaseRpcController rpcc) {
final HBaseRpcController rpcc, boolean clientCellBlockSupported) {
if (result == null) return;
if (isClientCellBlockSupport()) {
if (clientCellBlockSupported) {
builder.setResult(ProtobufUtil.toResultNoData(result));
rpcc.setCellScanner(result.cellScanner());
} else {
@ -1167,7 +1159,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @return an object that represents the last referenced block from this response.
*/
Object addSize(RpcCallContext context, Result r, Object lastBlock) {
if (context != null && !r.isEmpty()) {
if (context != null && r != null && !r.isEmpty()) {
for (Cell c : r.rawCells()) {
context.incrementResponseCellSize(CellUtil.estimatedHeapSizeOf(c));
@ -2299,12 +2291,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
builder.setResult(pbr);
} else if (r != null) {
ClientProtos.Result pbr;
RpcCallContext call = RpcServer.getCurrentCall();
if (isClientCellBlockSupport(call) && controller instanceof HBaseRpcController
&& VersionInfoUtil.hasMinimumVersion(call.getClientVersionInfo(), 1, 3)) {
if (isClientCellBlockSupport(context) && controller instanceof HBaseRpcController
&& VersionInfoUtil.hasMinimumVersion(context.getClientVersionInfo(), 1, 3)) {
pbr = ProtobufUtil.toResultNoData(r);
((HBaseRpcController) controller).setCellScanner(CellUtil.createCellScanner(r
.rawCells()));
addSize(context, r, null);
} else {
pbr = ProtobufUtil.toResult(r);
}
@ -2533,6 +2525,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
HBaseRpcController controller = (HBaseRpcController)rpcc;
CellScanner cellScanner = controller != null ? controller.cellScanner() : null;
OperationQuota quota = null;
RpcCallContext context = RpcServer.getCurrentCall();
// Clear scanner so we are not holding on to reference across call.
if (controller != null) {
controller.setCellScanner(null);
@ -2628,7 +2621,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (processed != null) {
builder.setProcessed(processed.booleanValue());
}
addResult(builder, r, controller);
boolean clientCellBlockSupported = isClientCellBlockSupport(context);
addResult(builder, r, controller, clientCellBlockSupported);
if (clientCellBlockSupported) {
addSize(context, r, null);
}
return builder.build();
} catch (IOException ie) {
regionServer.checkFileSystem();