From 9d881bfbad1a1243bbbb82d22c2fbedf16194f8f Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 21 Jan 2019 22:47:34 +0800 Subject: [PATCH] HBASE-21750 Most of KeyValueUtil#length can be replaced by cell#getSerializedSize for better performance because the latter one has been optimized --- .../apache/hadoop/hbase/client/HTable.java | 2 +- .../hadoop/hbase/ByteBufferKeyValue.java | 2 +- .../org/apache/hadoop/hbase/KeyValueUtil.java | 12 +---------- .../hbase/io/hfile/HFilePrettyPrinter.java | 2 +- .../hbase/mob/DefaultMobStoreCompactor.java | 2 +- .../hbase/mob/DefaultMobStoreFlusher.java | 3 +-- .../apache/hadoop/hbase/quotas/QuotaUtil.java | 8 ++++---- .../CellChunkImmutableSegment.java | 2 +- .../hadoop/hbase/regionserver/HRegion.java | 2 +- .../hbase/regionserver/StoreFlusher.java | 12 +++++------ .../regionserver/compactions/Compactor.java | 2 +- .../hbase/regionserver/TestCellFlatSet.java | 6 +++--- .../TestCompactingToCellFlatMapMemStore.java | 20 +++++++++---------- .../hbase/regionserver/TestHRegion.java | 4 ++-- .../regionserver/TestMemStoreChunkPool.java | 2 +- .../hbase/regionserver/TestMemStoreLAB.java | 4 ++-- .../TestMemstoreLABWithoutPool.java | 2 +- 17 files changed, 37 insertions(+), 50 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index fb69a2530b..fb49d6a985 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -933,7 +933,7 @@ public class HTable implements Table { if (maxKeyValueSize > 0) { for (List list : put.getFamilyCellMap().values()) { for (Cell cell : list) { - if (KeyValueUtil.length(cell) > maxKeyValueSize) { + if (cell.getSerializedSize() > maxKeyValueSize) { throw new IllegalArgumentException("KeyValue size too large"); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java index cafeb3e34e..8c6a2d5889 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java @@ -268,7 +268,7 @@ public class ByteBufferKeyValue extends ByteBufferExtendedCell { if (this.buf.hasArray()) { return ClassSize.align(FIXED_OVERHEAD + length); } - return ClassSize.align(FIXED_OVERHEAD) + KeyValueUtil.length(this); + return ClassSize.align(FIXED_OVERHEAD) + this.getSerializedSize(); } @Override diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java index 1230469aa9..581c6bb5e2 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java @@ -48,16 +48,6 @@ public class KeyValueUtil { /**************** length *********************/ - /** - * Returns number of bytes this cell would have been used if serialized as in {@link KeyValue} - * @param cell - * @return the length - */ - public static int length(final Cell cell) { - return length(cell.getRowLength(), cell.getFamilyLength(), cell.getQualifierLength(), - cell.getValueLength(), cell.getTagsLength(), true); - } - public static int length(short rlen, byte flen, int qlen, int vlen, int tlen, boolean withTags) { if (withTags) { return (int) (KeyValue.getKeyValueDataStructureSize(rlen, flen, qlen, vlen, tlen)); @@ -134,7 +124,7 @@ public class KeyValueUtil { } public static byte[] copyToNewByteArray(final Cell cell) { - int v1Length = length(cell); + int v1Length = cell.getSerializedSize(); byte[] backingBytes = new byte[v1Length]; appendToByteArray(cell, backingBytes, 0, true); return backingBytes; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 5a6f6c12e9..d7641604b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -622,7 +622,7 @@ public class HFilePrettyPrinter extends Configured implements Tool { // new row collectRow(); } - curRowBytes += KeyValueUtil.length(cell); + curRowBytes += cell.getSerializedSize(); curRowKeyLength = KeyValueUtil.keyLength(cell); curRowCols++; prevCell = cell; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 01c195a4d3..b04ca69b06 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -281,7 +281,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); } - int len = KeyValueUtil.length(c); + int len = c.getSerializedSize(); ++progress.currentCompactedKVs; progress.totalCompactedSize += len; bytesWrittenProgressForShippedCall += len; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index a932dad708..9cab142928 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -212,9 +212,8 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { this.mobStore.getRefCellTags()); writer.append(reference); } - int len = KeyValueUtil.length(c); if (control) { - throughputController.control(flushName, len); + throughputController.control(flushName, c.getSerializedSize()); } } cells.clear(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java index f6b5d95da3..7b51c549e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java @@ -290,9 +290,9 @@ public class QuotaUtil extends QuotaTableUtil { */ public static long calculateMutationSize(final Mutation mutation) { long size = 0; - for (Map.Entry> entry : mutation.getFamilyCellMap().entrySet()) { + for (Map.Entry> entry : mutation.getFamilyCellMap().entrySet()) { for (Cell cell : entry.getValue()) { - size += KeyValueUtil.length(cell); + size += cell.getSerializedSize(); } } return size; @@ -301,7 +301,7 @@ public class QuotaUtil extends QuotaTableUtil { public static long calculateResultSize(final Result result) { long size = 0; for (Cell cell : result.rawCells()) { - size += KeyValueUtil.length(cell); + size += cell.getSerializedSize(); } return size; } @@ -310,7 +310,7 @@ public class QuotaUtil extends QuotaTableUtil { long size = 0; for (Result result: results) { for (Cell cell : result.rawCells()) { - size += KeyValueUtil.length(cell); + size += cell.getSerializedSize(); } } return size; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java index 72cc4a22ea..c89975955d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java @@ -260,7 +260,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment { offset = ByteBufferUtils.putInt(idxBuffer, offset, dataChunkID); // write data chunk id offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getOffset()); // offset - offset = ByteBufferUtils.putInt(idxBuffer, offset, KeyValueUtil.length(cell)); // length + offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getSerializedSize()); // length offset = ByteBufferUtils.putLong(idxBuffer, offset, cell.getSequenceId()); // seqId return offset; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index afe2c0189b..0959245177 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -8491,7 +8491,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi int listSize = cells.size(); for (int i=0; i < listSize; i++) { Cell cell = cells.get(i); - mutationSize += KeyValueUtil.length(cell); + mutationSize += cell.getSerializedSize(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java index 442d47d3a1..0678f792b9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java @@ -110,7 +110,7 @@ abstract class StoreFlusher { protected void performFlush(InternalScanner scanner, CellSink sink, long smallestReadPoint, ThroughputController throughputController) throws IOException { int compactionKVMax = - conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT); + conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT); ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); @@ -119,7 +119,8 @@ abstract class StoreFlusher { boolean hasMore; String flushName = ThroughputControlUtil.getNameForThrottling(store, "flush"); // no control on system table (such as meta, namespace, etc) flush - boolean control = throughputController != null && !store.getRegionInfo().getTable().isSystemTable(); + boolean control = + throughputController != null && !store.getRegionInfo().getTable().isSystemTable(); if (control) { throughputController.start(flushName); } @@ -132,17 +133,16 @@ abstract class StoreFlusher { // set its memstoreTS to 0. This will help us save space when writing to // disk. sink.append(c); - int len = KeyValueUtil.length(c); if (control) { - throughputController.control(flushName, len); + throughputController.control(flushName, c.getSerializedSize()); } } kvs.clear(); } } while (hasMore); } catch (InterruptedException e) { - throw new InterruptedIOException("Interrupted while control throughput of flushing " - + flushName); + throw new InterruptedIOException( + "Interrupted while control throughput of flushing " + flushName); } finally { if (control) { throughputController.finish(flushName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 83690a95c7..8f6ead4feb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -401,7 +401,7 @@ public abstract class Compactor { lastCleanCellSeqId = 0; } writer.append(c); - int len = KeyValueUtil.length(c); + int len = c.getSerializedSize(); ++progress.currentCompactedKVs; progress.totalCompactedSize += len; bytesWrittenProgressForShippedCall += len; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java index 55d4947f37..c1f5262881 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java @@ -295,7 +295,7 @@ public class TestCellFlatSet { for (Cell kv: cellArray) { // do we have enough space to write the cell data on the data chunk? - if (dataOffset + KeyValueUtil.length(kv) > chunkCreator.getChunkSize()) { + if (dataOffset + kv.getSerializedSize() > chunkCreator.getChunkSize()) { // allocate more data chunks if needed dataChunk = chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP); dataBuffer = dataChunk.getData(); @@ -314,7 +314,7 @@ public class TestCellFlatSet { } idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataChunk.getId()); // write data chunk id idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataStartOfset); // offset - idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, KeyValueUtil.length(kv)); // length + idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, kv.getSerializedSize()); // length idxOffset = ByteBufferUtils.putLong(idxBuffer, idxOffset, kv.getSequenceId()); // seqId } @@ -357,7 +357,7 @@ public class TestCellFlatSet { // write data chunk id idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataJumboChunk.getId()); idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataStartOfset); // offset - idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, KeyValueUtil.length(kv)); // length + idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, kv.getSerializedSize()); // length idxOffset = ByteBufferUtils.putLong(idxBuffer, idxOffset, kv.getSequenceId()); // seqId // Jumbo chunks are working only with one cell per chunk, thus always allocate a new jumbo diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java index 19519157e2..960e9721b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java @@ -636,10 +636,8 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore // test 1 bucket int totalCellsLen = addRowsByKeys(memstore, keys1); - long oneCellOnCSLMHeapSize = - ClassSize.align( - ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + KeyValueUtil - .length(kv)); + long oneCellOnCSLMHeapSize = ClassSize.align( + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + kv.getSerializedSize()); long totalHeapSize = numOfCells * oneCellOnCSLMHeapSize + MutableSegment.DEEP_OVERHEAD; assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize()); @@ -648,7 +646,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and flatten assertEquals(0, memstore.getSnapshot().getCellsCount()); long oneCellOnCCMHeapSize = - ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv)); + ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize()); totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM + numOfCells * oneCellOnCCMHeapSize; @@ -721,7 +719,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore // One cell is duplicated, but it shouldn't be compacted because we are in BASIC mode. // totalCellsLen should remain the same long oneCellOnCCMHeapSize = - ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv)); + ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize()); totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM + numOfCells * oneCellOnCCMHeapSize; @@ -796,7 +794,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore // One cell is duplicated, but it shouldn't be compacted because we are in BASIC mode. // totalCellsLen should remain the same long oneCellOnCCMHeapSize = - (long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv)); + (long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize()); totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM + numOfCells * oneCellOnCCMHeapSize; @@ -876,7 +874,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore KeyValue kv = new KeyValue(Bytes.toBytes("A"), Bytes.toBytes("testfamily"), Bytes.toBytes("testqualifier"), System.currentTimeMillis(), val); long oneCellOnCCMHeapSize = - (long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv)); + (long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize()); long oneCellOnCSLMHeapSize = ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()); long totalHeapSize = MutableSegment.DEEP_OVERHEAD; @@ -932,7 +930,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore new KeyValue(row, Bytes.toBytes("testfamily"), Bytes.toBytes("testqualifier"), System.currentTimeMillis(), val); return ClassSize.align( - ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + KeyValueUtil.length(kv)); + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + kv.getSerializedSize()); } private long cellAfterFlushSize() { @@ -945,8 +943,8 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore return toCellChunkMap ? ClassSize.align( - ClassSize.CELL_CHUNK_MAP_ENTRY + KeyValueUtil.length(kv)) : + ClassSize.CELL_CHUNK_MAP_ENTRY + kv.getSerializedSize()) : ClassSize.align( - ClassSize.CELL_ARRAY_MAP_ENTRY + KeyValue.FIXED_OVERHEAD + KeyValueUtil.length(kv)); + ClassSize.CELL_ARRAY_MAP_ENTRY + KeyValue.FIXED_OVERHEAD + kv.getSerializedSize()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index b2d9a1bff2..09990cb413 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -2365,11 +2365,11 @@ public class TestHRegion { Cell originalCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1, System.currentTimeMillis(), KeyValue.Type.Put.getCode(), value1); - final long originalSize = KeyValueUtil.length(originalCell); + final long originalSize = originalCell.getSerializedSize(); Cell addCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1, System.currentTimeMillis(), KeyValue.Type.Put.getCode(), Bytes.toBytes("xxxxxxxxxx")); - final long addSize = KeyValueUtil.length(addCell); + final long addSize = addCell.getSerializedSize(); LOG.info("originalSize:" + originalSize + ", addSize:" + addSize); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java index 4f3de36a33..53a84d6395 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java @@ -92,7 +92,7 @@ public class TestMemStoreChunkPool { for (int i = 0; i < 100; i++) { int valSize = rand.nextInt(1000); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); - int size = KeyValueUtil.length(kv); + int size = kv.getSerializedSize(); ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv); if (newKv.getBuffer() != lastBuffer) { expectedOff = 4; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java index ef4ad69822..efa4baa11e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java @@ -94,7 +94,7 @@ public class TestMemStoreLAB { for (int i = 0; i < 100000; i++) { int valSize = rand.nextInt(3); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); - int size = KeyValueUtil.length(kv); + int size = kv.getSerializedSize(); ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv); if (newKv.getBuffer() != lastBuffer) { // since we add the chunkID at the 0th offset of the chunk and the @@ -145,7 +145,7 @@ public class TestMemStoreLAB { public void doAnAction() throws Exception { int valSize = r.nextInt(3); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); - int size = KeyValueUtil.length(kv); + int size = kv.getSerializedSize(); ByteBufferKeyValue newCell = (ByteBufferKeyValue) mslab.copyCellInto(kv); totalAllocated.addAndGet(size); allocsByThisThread.add(new AllocRecord(newCell.getBuffer(), newCell.getOffset(), size)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java index e2da5d03b5..c5d6f248f8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java @@ -80,7 +80,7 @@ public class TestMemstoreLABWithoutPool { for (int i = 0; i < 100000; i++) { int valSize = rand.nextInt(1000); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); - int size = KeyValueUtil.length(kv); + int size = kv.getSerializedSize(); ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv); if (newKv.getBuffer() != lastBuffer) { // since we add the chunkID at the 0th offset of the chunk and the -- 2.17.1