From f84999f40cc206166c3509f740ffe9db23cbdddd Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Fri, 20 Mar 2015 10:50:54 -0700 Subject: [PATCH] HBASE-13303 Fix size calculation of results on the region server --- .../java/org/apache/hadoop/hbase/CellUtil.java | 32 +++++++- .../hadoop/hbase/regionserver/HRegionServer.java | 4 +- .../hadoop/hbase/client/TestMaxResultSize.java | 85 ++++++++++++++++++++++ 3 files changed, 117 insertions(+), 4 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMaxResultSize.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java index 7c82c6c..1bf4f31 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -397,8 +397,36 @@ public final class CellUtil { // Serialization is probably preceded by a length (it is in the KeyValueCodec at least). Bytes.SIZEOF_INT; } - - + + /** + * This is a hack that should be removed once we don't care about matching + * up client- and server-side estimations of cell size. It needed to be + * backwards compatible with estimations done by older clients. We need to + * pretend that tags never exist and KeyValues aren't serialized with tag + * length included. See HBASE-13262 and HBASE-13303 + * @param cell + * @return Estimate of the cell size in bytes, for RPC. + */ + public static int estimatedSerializedSizeOfForRPC(final Cell cell) { + // If a KeyValue, we can give a good estimate of size. + if (cell instanceof KeyValue) { + // We don't care about tags when making the estimate. Normal clients + // should never get tags because the default RPC codec strips them + // out. The special case is replication. However, it's fine to + // _under_estimate on the server side in all cases. + return KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + ((KeyValue)cell).getKeyLength() + + cell.getValueLength(); + } + // TODO: Should we add to Cell a sizeOf? Would it help? Does it make sense if Cell is + // prefix encoded or compressed? + return cell.getRowLength() + cell.getFamilyLength() + + cell.getQualifierLength() + + cell.getValueLength() + + // Use the KeyValue's infrastructure size presuming that another implementation would have + // same basic cost. + KeyValue.KEY_INFRASTRUCTURE_SIZE; + } + /********************* tags *************************************/ /** * Util method to iterate through the tags diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 2e088df..291cfd7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -3254,7 +3254,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa if (maxScannerResultSize < Long.MAX_VALUE){ for (Cell cell : r.rawCells()) { KeyValue kv = KeyValueUtil.ensureKeyValue(cell); - currentScanResultSize += kv.heapSize(); + currentScanResultSize += CellUtil.estimatedSerializedSizeOfForRPC(kv); totalKvSize += kv.getLength(); } } @@ -3286,7 +3286,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa if (!values.isEmpty()) { for (Cell cell : values) { KeyValue kv = KeyValueUtil.ensureKeyValue(cell); - currentScanResultSize += kv.heapSize(); + currentScanResultSize += CellUtil.estimatedSerializedSizeOfForRPC(kv); totalKvSize += kv.getLength(); } results.add(Result.create(values)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMaxResultSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMaxResultSize.java new file mode 100644 index 0000000..8cfea54 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMaxResultSize.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import static org.junit.Assert.assertEquals; + +@Category(LargeTests.class) +public class TestMaxResultSize { + + final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + // set max result size to 1 byte + conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 1); + TEST_UTIL.startMiniCluster(1); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testMaxResultSize() throws Exception { + byte [] ROW1 = Bytes.toBytes("testRow1"); + byte [] ROW2 = Bytes.toBytes("testRow2"); + byte [] FAMILY = Bytes.toBytes("testFamily"); + byte [] QUALIFIER = Bytes.toBytes("testQualifier"); + byte [] VALUE = Bytes.toBytes("testValue"); + + TableName TABLE = TableName.valueOf("testMaxResultSize"); + byte[][] FAMILIES = new byte[][] { FAMILY }; + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, Long.MAX_VALUE); + HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf); + Put p = new Put(ROW1); + p.add(FAMILY, QUALIFIER, VALUE); + table.put(p); + p = new Put(ROW2); + p.add(FAMILY, QUALIFIER, VALUE); + table.put(p); + + Scan s = new Scan(); + ResultScanner rs = table.getScanner(s); + int count = 0; + while(rs.next() != null) { + count++; + } + assertEquals(2, count); + rs.close(); + table.close(); + } +} \ No newline at end of file -- 2.2.2