hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ramkris...@apache.org
Subject hbase git commit: HBASE-17484 Add non cached version of OffheapKV for write path (Ram)
Date Tue, 07 Feb 2017 16:04:02 GMT
Repository: hbase
Updated Branches:
  refs/heads/master 9d8de85fa -> 6c5eec249


HBASE-17484 Add non cached version of OffheapKV for write path (Ram)


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

Branch: refs/heads/master
Commit: 6c5eec249c6fcedd3d9f7fd810f89656647c1c67
Parents: 9d8de85
Author: Ramkrishna <ramkrishna.s.vasudevan@intel.com>
Authored: Tue Feb 7 21:27:33 2017 +0530
Committer: Ramkrishna <ramkrishna.s.vasudevan@intel.com>
Committed: Tue Feb 7 21:27:33 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  2 +-
 .../apache/hadoop/hbase/OffheapKeyValue.java    | 59 +++++++++-----------
 .../hadoop/hbase/codec/KeyValueCodec.java       |  2 +-
 .../hbase/io/encoding/RowIndexSeekerV1.java     |  3 +-
 .../hbase/util/test/RedundantKVGenerator.java   |  9 ++-
 .../hadoop/hbase/TestOffheapKeyValue.java       |  8 +--
 .../hbase/io/TestTagCompressionContext.java     |  2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  2 +-
 8 files changed, 39 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/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 7a9fe66..986a026 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
@@ -3202,6 +3202,6 @@ public final class CellUtil {
       newKv.setSequenceId(cell.getSequenceId());
       return newKv;
     }
-    return new OffheapKeyValue(buf, offset, len, tagsLen > 0, cell.getSequenceId());
+    return new OffheapKeyValue(buf, offset, len, cell.getSequenceId());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
index c08927c..1bc9549 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
@@ -36,9 +36,6 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
   protected final ByteBuffer buf;
   protected final int offset;
   protected final int length;
-  protected final boolean hasTags;
-  private final short rowLen;
-  private final int keyLen;
   private long seqId = 0;
   // TODO : See if famLen can be cached or not?
 
@@ -46,14 +43,11 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
       + (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT
       + Bytes.SIZEOF_BOOLEAN + Bytes.SIZEOF_LONG;
 
-  public OffheapKeyValue(ByteBuffer buf, int offset, int length, boolean hasTags, long seqId)
{
+  public OffheapKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
     assert buf.isDirect();
     this.buf = buf;
     this.offset = offset;
     this.length = length;
-    rowLen = ByteBufferUtils.toShort(this.buf, this.offset + KeyValue.ROW_OFFSET);
-    keyLen = ByteBufferUtils.toInt(this.buf, this.offset);
-    this.hasTags = hasTags;
     this.seqId = seqId;
   }
 
@@ -62,11 +56,6 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
     this.buf = buf;
     this.offset = offset;
     this.length = length;
-    rowLen = ByteBufferUtils.toShort(this.buf, this.offset + KeyValue.ROW_OFFSET);
-    keyLen = ByteBufferUtils.toInt(this.buf, this.offset);
-    int tagsLen = this.length
-        - (this.keyLen + getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
-    this.hasTags = tagsLen > 0;
   }
 
   @Override
@@ -81,7 +70,11 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
 
   @Override
   public short getRowLength() {
-    return this.rowLen;
+    return getRowLen();
+  }
+
+  private short getRowLen() {
+    return ByteBufferUtils.toShort(this.buf, this.offset + KeyValue.ROW_OFFSET);
   }
 
   @Override
@@ -100,7 +93,8 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
   }
 
   private int getFamilyLengthPosition() {
-    return this.offset + KeyValue.ROW_KEY_OFFSET + rowLen;
+    return this.offset + KeyValue.ROW_KEY_OFFSET
+        + getRowLen();
   }
 
   private byte getFamilyLength(int famLenPos) {
@@ -123,22 +117,28 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
   }
 
   private int getQualifierLength(int rlength, int flength) {
-    return this.keyLen - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0);
+    return getKeyLen()
+        - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0);
   }
 
   @Override
   public long getTimestamp() {
-    int offset = getTimestampOffset(this.keyLen);
+    int offset = getTimestampOffset(getKeyLen());
     return ByteBufferUtils.toLong(this.buf, offset);
   }
 
+  private int getKeyLen() {
+    return ByteBufferUtils.toInt(this.buf, this.offset);
+  }
+
   private int getTimestampOffset(int keyLen) {
     return this.offset + KeyValue.ROW_OFFSET + keyLen - KeyValue.TIMESTAMP_TYPE_SIZE;
   }
 
   @Override
   public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.keyLen - 1 + KeyValue.ROW_OFFSET);
+    return ByteBufferUtils.toByte(this.buf,
+      this.offset + getKeyLen() - 1 + KeyValue.ROW_OFFSET);
   }
 
   @Override
@@ -177,11 +177,8 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
 
   @Override
   public int getTagsLength() {
-    if(!hasTags) {
-      return 0;
-    }
-    int tagsLen = this.length
-        - (this.keyLen + getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
+    int tagsLen = this.length - (getKeyLen() + getValueLength()
+        + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
     if (tagsLen > 0) {
       // There are some Tag bytes in the byte[]. So reduce 2 bytes which is
       // added to denote the tags
@@ -228,7 +225,7 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
 
   @Override
   public int getValuePosition() {
-    return this.offset + KeyValue.ROW_OFFSET + this.keyLen;
+    return this.offset + KeyValue.ROW_OFFSET + getKeyLen();
   }
 
   @Override
@@ -262,7 +259,8 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
     if (withTags) {
       return this.length;
     }
-    return this.keyLen + this.getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
+    return getKeyLen() + this.getValueLength()
+        + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
   }
 
   @Override
@@ -278,12 +276,12 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
   @Override
   public void setTimestamp(long ts) throws IOException {
     ByteBufferUtils.copyFromArrayToBuffer(this.buf, this.getTimestampOffset(), Bytes.toBytes(ts),
0,
-        Bytes.SIZEOF_LONG);
+      Bytes.SIZEOF_LONG);
   }
 
   private int getTimestampOffset() {
-    return this.offset + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + this.keyLen
-        - KeyValue.TIMESTAMP_TYPE_SIZE;
+    return this.offset + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE
+        + getKeyLen() - KeyValue.TIMESTAMP_TYPE_SIZE;
   }
 
   @Override
@@ -301,12 +299,7 @@ public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell
{
   public Cell deepClone() {
     byte[] copy = new byte[this.length];
     ByteBufferUtils.copyFromBufferToArray(copy, this.buf, this.offset, 0, this.length);
-    KeyValue kv;
-    if (this.hasTags) {
-      kv = new KeyValue(copy, 0, copy.length);
-    } else {
-      kv = new NoTagsKeyValue(copy, 0, copy.length);
-    }
+    KeyValue kv = new KeyValue(copy, 0, copy.length);
     kv.setSequenceId(this.getSequenceId());
     return kv;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
index afe4e11..70ad3dd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
@@ -111,7 +111,7 @@ public class KeyValueCodec implements Codec {
 
     protected Cell createCell(ByteBuffer bb, int pos, int len) {
       // We know there is not going to be any tags.
-      return new OffheapKeyValue(bb, pos, len, false, 0);
+      return new OffheapKeyValue(bb, pos, len, 0);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
index 9ad098c..d202f48 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
@@ -382,8 +382,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
         currentBuffer.asSubByteBuffer(startOffset, cellBufSize, tmpPair);
         ByteBuffer buf = tmpPair.getFirst();
         if (buf.isDirect()) {
-          ret = new OffheapKeyValue(buf, tmpPair.getSecond(), cellBufSize,
-              tagsLength > 0, seqId);
+          ret = new OffheapKeyValue(buf, tmpPair.getSecond(), cellBufSize, seqId);
         } else {
           if (tagsLength > 0) {
             ret = new SizeCachedKeyValue(buf.array(), buf.arrayOffset()

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
index 7dc3d5a..0f9c0ce 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
@@ -371,7 +371,7 @@ public class RedundantKVGenerator {
         ByteBufferUtils.copyFromArrayToBuffer(offheapKVBB, keyValue.getBuffer(),
           keyValue.getOffset(), keyValue.getLength());
         OffheapKeyValue offheapKV =
-            new ExtendedOffheapKeyValue(offheapKVBB, 0, keyValue.getLength(), true, 0);
+            new ExtendedOffheapKeyValue(offheapKVBB, 0, keyValue.getLength(), 0);
         result.add(offheapKV);
       } else {
         KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp, value);
@@ -379,7 +379,7 @@ public class RedundantKVGenerator {
         ByteBufferUtils.copyFromArrayToBuffer(offheapKVBB, keyValue.getBuffer(),
           keyValue.getOffset(), keyValue.getLength());
         OffheapKeyValue offheapKV =
-            new ExtendedOffheapKeyValue(offheapKVBB, 0, keyValue.getLength(), false, 0);
+            new ExtendedOffheapKeyValue(offheapKVBB, 0, keyValue.getLength(), 0);
         result.add(offheapKV);
       }
     }
@@ -390,9 +390,8 @@ public class RedundantKVGenerator {
   }
 
   static class ExtendedOffheapKeyValue extends OffheapKeyValue {
-    public ExtendedOffheapKeyValue(ByteBuffer buf, int offset, int length, boolean hasTags,
-        long seqId) {
-      super(buf, offset, length, hasTags, seqId);
+    public ExtendedOffheapKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
+      super(buf, offset, length, seqId);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
index ec44408..dac7a56 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
@@ -56,7 +56,7 @@ public class TestOffheapKeyValue {
     KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1);
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l);
+    ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -99,7 +99,7 @@ public class TestOffheapKeyValue {
     kvCell = new KeyValue(row1, fam2, qual2, 0l, Type.Put, row1);
     buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l);
+    offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
     assertEquals(
       FAM2,
       ByteBufferUtils.toStringBinary(offheapKV.getFamilyByteBuffer(),
@@ -112,7 +112,7 @@ public class TestOffheapKeyValue {
     kvCell = new KeyValue(row1, fam1, nullQualifier, 0L, Type.Put, row1);
     buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l);
+    offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -138,7 +138,7 @@ public class TestOffheapKeyValue {
     KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1, tags);
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), true, 0l);
+    ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
index 0a14443..9e8ba8e 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
@@ -166,7 +166,7 @@ public class TestTagCompressionContext {
     KeyValue kv = new KeyValue(ROW, CF, Q, 1234L, V, tags);
     ByteBuffer dbb = ByteBuffer.allocateDirect(kv.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(dbb, kv.getBuffer(), 0, kv.getBuffer().length);
-    OffheapKeyValue offheapKV = new OffheapKeyValue(dbb, 0, kv.getBuffer().length, true,
0);
+    OffheapKeyValue offheapKV = new OffheapKeyValue(dbb, 0, kv.getBuffer().length, 0);
     return offheapKV;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c5eec24/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 4cf1bf2..1bb5274 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -957,7 +957,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
       } else {
         ByteBuffer buf = blockBuffer.asSubByteBuffer(cellBufSize);
         if (buf.isDirect()) {
-          ret = new OffheapKeyValue(buf, buf.position(), cellBufSize, currTagsLen > 0,
seqId);
+          ret = new OffheapKeyValue(buf, buf.position(), cellBufSize, seqId);
         } else {
           if (currTagsLen > 0) {
             ret = new SizeCachedKeyValue(buf.array(), buf.arrayOffset() + buf.position(),


Mime
View raw message