Repository: hbase
Updated Branches:
  refs/heads/branch-1 a8909f5cb -> e7cd7fffb


HBASE-16444 CellUtil#estimatedSerializedSizeOfKey() should consider
KEY_INFRASTRUCTURE_SIZ (Ram)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e7cd7fff
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e7cd7fff
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e7cd7fff

Branch: refs/heads/branch-1
Commit: e7cd7fffbf4b297e54438c1378f0b9aa2c98523b
Parents: a8909f5
Author: Ramkrishna <ramkrishna.s.vasude...@intel.com>
Authored: Tue Aug 23 10:53:33 2016 +0530
Committer: Ramkrishna <ramkrishna.s.vasude...@intel.com>
Committed: Tue Aug 23 10:58:50 2016 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/CellUtil.java   | 14 +++++++++++---
 1 file changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e7cd7fff/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.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 ff6ad90..be50069 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
@@ -526,6 +526,7 @@ public final class CellUtil {
   }
 
   /**
+   * Estimate based on keyvalue's serialization format. 
    * @param cell
    * @return Estimate of the <code>cell</code> size in bytes.
    * @deprecated please use estimatedSerializedSizeOf(Cell)
@@ -549,7 +550,7 @@ public final class CellUtil {
     return getSumOfCellElementLengths(cell) +
       // Use the KeyValue's infrastructure size presuming that another 
implementation would have
       // same basic cost.
-      KeyValue.KEY_INFRASTRUCTURE_SIZE +
+      KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE +
       // Serialization is probably preceded by a length (it is in the 
KeyValueCodec at least).
       Bytes.SIZEOF_INT;
   }
@@ -573,10 +574,17 @@ public final class CellUtil {
     KeyValue.TIMESTAMP_TYPE_SIZE;
   }
 
+  /**
+   * Calculates the serialized key size. We always serialize in the KeyValue's 
serialization
+   * format.
+   * @param cell the cell for which the key size has to be calculated.
+   * @return the key size
+   */
   public static int estimatedSerializedSizeOfKey(final Cell cell) {
     if (cell instanceof KeyValue) return ((KeyValue)cell).getKeyLength();
-    // This will be a low estimate.  Will do for now.
-    return getSumOfCellKeyElementLengths(cell);
+    return cell.getRowLength() + cell.getFamilyLength() +
+        cell.getQualifierLength() +
+        KeyValue.KEY_INFRASTRUCTURE_SIZE;
   }
 
   /**

Reply via email to