This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 4eba6b3  HBASE-21750 Most of KeyValueUtil#length can be replaced by 
cell#getSerializedSize for better performance because the latter one has been 
optimized
4eba6b3 is described below

commit 4eba6b365638bddbf3f78047eaf0a4dc1a431fc0
Author: huzheng <[email protected]>
AuthorDate: Mon Jan 21 22:47:34 2019 +0800

    HBASE-21750 Most of KeyValueUtil#length can be replaced by 
cell#getSerializedSize for better performance because the latter one has been 
optimized
---
 .../java/org/apache/hadoop/hbase/client/HTable.java |  3 +--
 .../apache/hadoop/hbase/filter/KeyOnlyFilter.java   |  6 +++++-
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java      |  3 +++
 .../org/apache/hadoop/hbase/ByteBufferKeyValue.java |  2 +-
 .../java/org/apache/hadoop/hbase/KeyValueUtil.java  | 12 +-----------
 .../hadoop/hbase/io/hfile/HFilePrettyPrinter.java   |  2 +-
 .../hadoop/hbase/mob/DefaultMobStoreCompactor.java  |  3 +--
 .../hadoop/hbase/mob/DefaultMobStoreFlusher.java    |  4 +---
 .../org/apache/hadoop/hbase/quotas/QuotaUtil.java   |  9 ++++-----
 .../regionserver/CellChunkImmutableSegment.java     |  3 +--
 .../apache/hadoop/hbase/regionserver/HRegion.java   |  3 +--
 .../hadoop/hbase/regionserver/StoreFlusher.java     | 14 ++++++--------
 .../hbase/regionserver/compactions/Compactor.java   |  3 +--
 .../hadoop/hbase/regionserver/TestCellFlatSet.java  |  6 +++---
 .../TestCompactingToCellFlatMapMemStore.java        | 21 +++++++++------------
 .../hadoop/hbase/regionserver/TestHRegion.java      |  5 ++---
 .../hbase/regionserver/TestMemStoreChunkPool.java   |  3 +--
 .../hadoop/hbase/regionserver/TestMemStoreLAB.java  |  5 ++---
 .../regionserver/TestMemstoreLABWithoutPool.java    |  3 +--
 19 files changed, 45 insertions(+), 65 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 fb69a25..4b86e22 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -933,7 +932,7 @@ public class HTable implements Table {
     if (maxKeyValueSize > 0) {
       for (List<Cell> 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-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index 3368078..9603304 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -29,6 +29,8 @@ import java.util.Optional;
 import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -147,11 +149,13 @@ public class KeyOnlyFilter extends FilterBase {
 
   static class KeyOnlyCell implements Cell {
     private Cell cell;
+    private int keyLen;
     private boolean lenAsVal;
 
     public KeyOnlyCell(Cell c, boolean lenAsVal) {
       this.cell = c;
       this.lenAsVal = lenAsVal;
+      this.keyLen = KeyValueUtil.keyLength(c);
     }
 
     @Override
@@ -245,7 +249,7 @@ public class KeyOnlyFilter extends FilterBase {
 
     @Override
     public int getSerializedSize() {
-      return cell.getSerializedSize();
+      return KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + keyLen + getValueLength();
     }
 
     @Override
diff --git 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
index 4f92739..3e03a07 100644
--- 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
+++ 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
@@ -102,6 +103,8 @@ public class TestKeyOnlyFilter {
     assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue, keyOnlyCell));
     assertTrue(KeyOnlyKeyValue.getValueLength() == keyOnlyByteBufferedCell
         .getValueLength());
+    assertEquals(8 + keyLen + (lenAsVal ? 4 : 0), 
KeyOnlyKeyValue.getSerializedSize());
+    assertEquals(8 + keyLen + (lenAsVal ? 4 : 0), 
keyOnlyCell.getSerializedSize());
     if (keyOnlyByteBufferedCell.getValueLength() > 0) {
       assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue,
           keyOnlyByteBufferedCell));
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 cafeb3e..8c6a2d5 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 1230469..581c6bb 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 5a6f6c1..d764160 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 01c195a..062bec6 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
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.regionserver.CellSink;
 import org.apache.hadoop.hbase.regionserver.HMobStore;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -281,7 +280,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 a932dad..3de7992 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
@@ -29,7 +29,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
 import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
@@ -212,9 +211,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 f6b5d95..c183610 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
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -290,9 +289,9 @@ public class QuotaUtil extends QuotaTableUtil {
    */
   public static long calculateMutationSize(final Mutation mutation) {
     long size = 0;
-    for (Map.Entry<byte [], List<Cell>> entry : 
mutation.getFamilyCellMap().entrySet()) {
+    for (Map.Entry<byte[], List<Cell>> entry : 
mutation.getFamilyCellMap().entrySet()) {
       for (Cell cell : entry.getValue()) {
-        size += KeyValueUtil.length(cell);
+        size += cell.getSerializedSize();
       }
     }
     return size;
@@ -301,7 +300,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 +309,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 72cc4a2..236c7d3 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
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.ExtendedCell;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -260,7 +259,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 05d70a2..c9bbebe 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
@@ -92,7 +92,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.PrivateCellUtil;
@@ -8448,7 +8447,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 442d47d..4c539ce 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
@@ -22,13 +22,11 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.OptionalInt;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@@ -110,7 +108,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 +117,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 +131,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 83690a9..e5efb94 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
@@ -34,7 +34,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
@@ -401,7 +400,7 @@ public abstract class Compactor<T extends CellSink> {
             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 55d4947..c1f5262 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 1e67015..7c80eee 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
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -638,10 +637,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());
@@ -650,7 +647,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;
 
@@ -723,7 +720,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;
 
@@ -798,7 +795,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;
 
@@ -878,7 +875,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;
@@ -934,7 +931,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() {
@@ -947,8 +944,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 8ee3eae..150c650 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
@@ -86,7 +86,6 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.MultithreadedTestUtil;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
@@ -2364,11 +2363,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 4f3de36..8eea7c5 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
@@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -92,7 +91,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 ef4ad69..d74eb46 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.MultithreadedTestUtil;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
@@ -94,7 +93,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 +144,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 e2da5d0..e0ee862 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
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -80,7 +79,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

Reply via email to