You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2016/08/23 05:26:02 UTC

hbase git commit: HBASE-16444 CellUtil#estimatedSerializedSizeOfKey() should consider KEY_INFRASTRUCTURE_SIZ (Ram)

Repository: hbase
Updated Branches:
  refs/heads/master abc64fa69 -> 2aae923c3


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/2aae923c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2aae923c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2aae923c

Branch: refs/heads/master
Commit: 2aae923c32528667644c57085a0180bede1ad262
Parents: abc64fa
Author: Ramkrishna <ra...@intel.com>
Authored: Tue Aug 23 10:53:33 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Tue Aug 23 10:53:33 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/2aae923c/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 c6a0a93..ac55112 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
@@ -902,6 +902,7 @@ public final class CellUtil {
   }
 
   /**
+   * Estimate based on keyvalue's serialization format. 
    * @param cell
    * @return Estimate of the <code>cell</code> size in bytes.
    */
@@ -915,7 +916,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;
   }
@@ -939,10 +940,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;
   }
 
   /**