|
24 | 24 | import java.net.BindException; |
25 | 25 | import java.net.InetAddress; |
26 | 26 | import java.net.InetSocketAddress; |
27 | | -import java.nio.ByteBuffer; |
28 | 27 | import java.util.ArrayList; |
29 | 28 | import java.util.Arrays; |
30 | 29 | import java.util.Collections; |
|
45 | 44 | import org.apache.hadoop.conf.Configuration; |
46 | 45 | import org.apache.hadoop.fs.FileSystem; |
47 | 46 | import org.apache.hadoop.fs.Path; |
48 | | -import org.apache.hadoop.hbase.ByteBufferExtendedCell; |
49 | 47 | import org.apache.hadoop.hbase.CacheEvictionStats; |
50 | 48 | import org.apache.hadoop.hbase.CacheEvictionStatsBuilder; |
51 | 49 | import org.apache.hadoop.hbase.Cell; |
@@ -710,7 +708,6 @@ private List<CellScannable> doNonAtomicRegionMutation(final HRegion region, |
710 | 708 | List<ClientProtos.Action> mutations = null; |
711 | 709 | long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getReadAvailable()); |
712 | 710 | IOException sizeIOE = null; |
713 | | - Object lastBlock = null; |
714 | 711 | ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = |
715 | 712 | ResultOrException.newBuilder(); |
716 | 713 | boolean hasResultOrException = false; |
@@ -835,7 +832,7 @@ private List<CellScannable> doNonAtomicRegionMutation(final HRegion region, |
835 | 832 | } else { |
836 | 833 | pbResult = ProtobufUtil.toResult(r); |
837 | 834 | } |
838 | | - lastBlock = addSize(context, r, lastBlock); |
| 835 | + addSize(context, r); |
839 | 836 | hasResultOrException = true; |
840 | 837 | resultOrExceptionBuilder.setResult(pbResult); |
841 | 838 | } |
@@ -1293,44 +1290,17 @@ long getScannerVirtualTime(long scannerId) { |
1293 | 1290 | } |
1294 | 1291 |
|
1295 | 1292 | /** |
1296 | | - * Method to account for the size of retained cells and retained data blocks. |
1297 | | - * @param context rpc call context |
1298 | | - * @param r result to add size. |
1299 | | - * @param lastBlock last block to check whether we need to add the block size in context. |
| 1293 | + * Method to account for the size of retained cells. |
| 1294 | + * @param context rpc call context |
| 1295 | + * @param r result to add size. |
1300 | 1296 | * @return an object that represents the last referenced block from this response. |
1301 | 1297 | */ |
1302 | | - Object addSize(RpcCallContext context, Result r, Object lastBlock) { |
| 1298 | + void addSize(RpcCallContext context, Result r) { |
1303 | 1299 | if (context != null && r != null && !r.isEmpty()) { |
1304 | 1300 | for (Cell c : r.rawCells()) { |
1305 | 1301 | context.incrementResponseCellSize(PrivateCellUtil.estimatedSerializedSizeOf(c)); |
1306 | | - |
1307 | | - // Since byte buffers can point all kinds of crazy places it's harder to keep track |
1308 | | - // of which blocks are kept alive by what byte buffer. |
1309 | | - // So we make a guess. |
1310 | | - if (c instanceof ByteBufferExtendedCell) { |
1311 | | - ByteBufferExtendedCell bbCell = (ByteBufferExtendedCell) c; |
1312 | | - ByteBuffer bb = bbCell.getValueByteBuffer(); |
1313 | | - if (bb != lastBlock) { |
1314 | | - context.incrementResponseBlockSize(bb.capacity()); |
1315 | | - lastBlock = bb; |
1316 | | - } |
1317 | | - } else { |
1318 | | - // We're using the last block being the same as the current block as |
1319 | | - // a proxy for pointing to a new block. This won't be exact. |
1320 | | - // If there are multiple gets that bounce back and forth |
1321 | | - // Then it's possible that this will over count the size of |
1322 | | - // referenced blocks. However it's better to over count and |
1323 | | - // use two rpcs than to OOME the regionserver. |
1324 | | - byte[] valueArray = c.getValueArray(); |
1325 | | - if (valueArray != lastBlock) { |
1326 | | - context.incrementResponseBlockSize(valueArray.length); |
1327 | | - lastBlock = valueArray; |
1328 | | - } |
1329 | | - } |
1330 | | - |
1331 | 1302 | } |
1332 | 1303 | } |
1333 | | - return lastBlock; |
1334 | 1304 | } |
1335 | 1305 |
|
1336 | 1306 | /** Returns Remote client's ip and port else null if can't be determined. */ |
@@ -2515,7 +2485,7 @@ public GetResponse get(final RpcController controller, final GetRequest request) |
2515 | 2485 | pbr = ProtobufUtil.toResultNoData(r); |
2516 | 2486 | ((HBaseRpcController) controller) |
2517 | 2487 | .setCellScanner(CellUtil.createCellScanner(r.rawCells())); |
2518 | | - addSize(context, r, null); |
| 2488 | + addSize(context, r); |
2519 | 2489 | } else { |
2520 | 2490 | pbr = ProtobufUtil.toResult(r); |
2521 | 2491 | } |
@@ -2957,7 +2927,7 @@ public MutateResponse mutate(final RpcController rpcc, final MutateRequest reque |
2957 | 2927 | boolean clientCellBlockSupported = isClientCellBlockSupport(context); |
2958 | 2928 | addResult(builder, result.getResult(), controller, clientCellBlockSupported); |
2959 | 2929 | if (clientCellBlockSupported) { |
2960 | | - addSize(context, result.getResult(), null); |
| 2930 | + addSize(context, result.getResult()); |
2961 | 2931 | } |
2962 | 2932 | } else { |
2963 | 2933 | Result r = null; |
@@ -2989,7 +2959,7 @@ public MutateResponse mutate(final RpcController rpcc, final MutateRequest reque |
2989 | 2959 | boolean clientCellBlockSupported = isClientCellBlockSupport(context); |
2990 | 2960 | addResult(builder, r, controller, clientCellBlockSupported); |
2991 | 2961 | if (clientCellBlockSupported) { |
2992 | | - addSize(context, r, null); |
| 2962 | + addSize(context, r); |
2993 | 2963 | } |
2994 | 2964 | } |
2995 | 2965 | return builder.build(); |
@@ -3343,9 +3313,10 @@ private void scan(HBaseRpcController controller, ScanRequest request, RegionScan |
3343 | 3313 | // of heap size occupied by cells(being read). Cell data means its key and value parts. |
3344 | 3314 | // maxQuotaResultSize - max results just from server side configuration and quotas, without |
3345 | 3315 | // user's specified max. We use this for evaluating limits based on blocks (not cells). |
3346 | | - // We may have accumulated some results in coprocessor preScannerNext call. We estimate |
3347 | | - // block and cell size of those using call to addSize. Update our maximums for scanner |
3348 | | - // context so we can account for them in the real scan. |
| 3316 | + // We may have accumulated some results in coprocessor preScannerNext call. Subtract any |
| 3317 | + // cell or block size from maximum here so we adhere to total limits of request. |
| 3318 | + // Note: we track block size in StoreScanner. If the CP hook got cells from hbase, it will |
| 3319 | + // have accumulated block bytes. If not, this will be 0 for block size. |
3349 | 3320 | long maxCellSize = maxResultSize; |
3350 | 3321 | long maxBlockSize = maxQuotaResultSize; |
3351 | 3322 | if (rpcCall != null) { |
@@ -3461,7 +3432,6 @@ private void scan(HBaseRpcController controller, ScanRequest request, RegionScan |
3461 | 3432 | values.clear(); |
3462 | 3433 | } |
3463 | 3434 | if (rpcCall != null) { |
3464 | | - rpcCall.incrementResponseBlockSize(scannerContext.getBlockSizeProgress()); |
3465 | 3435 | rpcCall.incrementResponseCellSize(scannerContext.getHeapSizeProgress()); |
3466 | 3436 | } |
3467 | 3437 | builder.setMoreResultsInRegion(moreRows); |
@@ -3634,18 +3604,11 @@ public ScanResponse scan(final RpcController controller, final ScanRequest reque |
3634 | 3604 | if (region.getCoprocessorHost() != null) { |
3635 | 3605 | Boolean bypass = region.getCoprocessorHost().preScannerNext(scanner, results, rows); |
3636 | 3606 | if (!results.isEmpty()) { |
3637 | | - // If scanner CP added results to list, we want to account for cell and block size of |
3638 | | - // that work. We estimate this using addSize, since CP does not get ScannerContext. If |
3639 | | - // !done, the actual scan call below will use more accurate ScannerContext block and |
3640 | | - // cell size tracking for the rest of the request. The two result sets will be added |
3641 | | - // together in the RpcCall accounting. |
3642 | | - // This here is just an estimate (see addSize for more details on estimation). We don't |
3643 | | - // pass lastBlock to the scan call below because the real scan uses ScannerContext, |
3644 | | - // which does not use lastBlock tracking. This may result in over counting by 1 block, |
3645 | | - // but that is unlikely since addSize is already a rough estimate. |
3646 | | - Object lastBlock = null; |
3647 | 3607 | for (Result r : results) { |
3648 | | - lastBlock = addSize(rpcCall, r, lastBlock); |
| 3608 | + // add cell size from CP results so we can track response size and update limits |
| 3609 | + // when calling scan below if !done. We'll also have tracked block size if the CP |
| 3610 | + // got results from hbase, since StoreScanner tracks that for all calls automatically. |
| 3611 | + addSize(rpcCall, r); |
3649 | 3612 | } |
3650 | 3613 | } |
3651 | 3614 | if (bypass != null && bypass.booleanValue()) { |
|
0 commit comments