You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2013/11/13 00:31:46 UTC

svn commit: r1541314 - in /hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase: KeyValue.java io/hfile/HFileReaderV2.java regionserver/ScanQueryMatcher.java

Author: larsh
Date: Tue Nov 12 23:31:46 2013
New Revision: 1541314

URL: http://svn.apache.org/r1541314
Log:
HBASE-9956 Remove keyLength cache from KeyValue

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/KeyValue.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1541314&r1=1541313&r2=1541314&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/KeyValue.java Tue Nov 12 23:31:46 2013
@@ -64,7 +64,7 @@ import com.google.common.primitives.Long
  * be < <code>Integer.MAX_SIZE</code>.
  * The column does not contain the family/qualifier delimiter, {@link #COLUMN_FAMILY_DELIMITER}
  */
-public class KeyValue implements Writable, HeapSize {
+public class KeyValue implements Writable, HeapSize, Cloneable {
   static final Log LOG = LogFactory.getLog(KeyValue.class);
   // TODO: Group Key-only comparators and operations into a Key class, just
   // for neatness sake, if can figure what to call it.
@@ -278,21 +278,6 @@ public class KeyValue implements Writabl
     this.length = length;
   }
 
-  /**
-   * Creates a KeyValue from the specified byte array, starting at offset,
-   * for length <code>length</code>, and a known <code>keyLength</code>.
-   * @param bytes byte array
-   * @param offset offset to start of the KeyValue
-   * @param length length of the KeyValue
-   * @param keyLength length of the key portion of the KeyValue
-   */
-  public KeyValue(final byte [] bytes, final int offset, final int length, final int keyLength) {
-    this.bytes = bytes;
-    this.offset = offset;
-    this.length = length;
-    this.keyLength = keyLength;
-  }
-
   /** Constructors that build a new backing byte array from fields */
 
   /**
@@ -632,6 +617,7 @@ public class KeyValue implements Writabl
 
   // Needed doing 'contains' on List.  Only compares the key portion, not the
   // value.
+  @Override
   public boolean equals(Object other) {
     if (!(other instanceof KeyValue)) {
       return false;
@@ -643,6 +629,7 @@ public class KeyValue implements Writabl
       kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
   }
 
+  @Override
   public int hashCode() {
     byte[] b = getBuffer();
     int start = getOffset(), end = getOffset() + getLength();
@@ -834,13 +821,8 @@ public class KeyValue implements Writabl
   /**
    * @return Length of key portion.
    */
-  private int keyLength = 0;
-
   public int getKeyLength() {
-    if (keyLength == 0) {
-      keyLength = Bytes.toInt(this.bytes, this.offset);
-    }
-    return keyLength;
+    return Bytes.toInt(this.bytes, this.offset);
   }
 
   /**
@@ -2265,7 +2247,7 @@ public class KeyValue implements Writabl
   public long heapSize() {
     return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE
         + ClassSize.align(ClassSize.ARRAY) + ClassSize.align(length)
-        + (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_LONG);
+        + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_LONG);
   }
 
   // this overload assumes that the length bytes have already been read,
@@ -2274,7 +2256,6 @@ public class KeyValue implements Writabl
   public void readFields(int length, final DataInput in) throws IOException {
     this.length = length;
     this.offset = 0;
-    this.keyLength = 0;
     this.bytes = new byte[this.length];
     in.readFully(this.bytes, 0, this.length);
   }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java?rev=1541314&r1=1541313&r2=1541314&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java Tue Nov 12 23:31:46 2013
@@ -638,8 +638,7 @@ public class HFileReaderV2 extends Abstr
 
       KeyValue ret = new KeyValue(blockBuffer.array(),
           blockBuffer.arrayOffset() + blockBuffer.position(),
-          KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen,
-          currKeyLen);
+          KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen);
       if (this.reader.shouldIncludeMemstoreTS()) {
         ret.setMemstoreTS(currMemstoreTS);
       }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java?rev=1541314&r1=1541313&r2=1541314&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java Tue Nov 12 23:31:46 2013
@@ -219,11 +219,12 @@ public class ScanQueryMatcher {
 
     byte [] bytes = kv.getBuffer();
     int offset = kv.getOffset();
-    int initialOffset = offset;
 
     int keyLength = Bytes.toInt(bytes, offset, Bytes.SIZEOF_INT);
     offset += KeyValue.ROW_OFFSET;
 
+    int initialOffset = offset;
+
     short rowLength = Bytes.toShort(bytes, offset, Bytes.SIZEOF_SHORT);
     offset += Bytes.SIZEOF_SHORT;
 
@@ -254,10 +255,10 @@ public class ScanQueryMatcher {
     byte familyLength = bytes [offset];
     offset += familyLength + 1;
 
-    int qualLength = keyLength + KeyValue.ROW_OFFSET -
+    int qualLength = keyLength -
       (offset - initialOffset) - KeyValue.TIMESTAMP_TYPE_SIZE;
 
-    long timestamp = kv.getTimestamp();
+    long timestamp = Bytes.toLong(bytes, initialOffset + keyLength - KeyValue.TIMESTAMP_TYPE_SIZE);
     // check for early out based on timestamp alone
     if (columns.isDone(timestamp)) {
         return columns.getNextRowOrNextColumn(bytes, offset, qualLength);
@@ -276,7 +277,7 @@ public class ScanQueryMatcher {
      * 7. Delete marker need to be version counted together with puts
      *    they affect
      */
-    byte type = kv.getType();
+    byte type = bytes[initialOffset + keyLength - 1];
     if (kv.isDelete()) {
       if (!keepDeletedCells) {
         // first ignore delete markers if the scanner can do so, and the