You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2015/06/04 06:52:55 UTC

hbase git commit: HBASE-13817 ByteBufferOuputStream - add writeInt support.

Repository: hbase
Updated Branches:
  refs/heads/master e8e5a9f63 -> bb62d5b2e


HBASE-13817 ByteBufferOuputStream - add writeInt support.


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

Branch: refs/heads/master
Commit: bb62d5b2e8db1677568723bb8d0d8ea4fd2c6f46
Parents: e8e5a9f
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Jun 4 10:22:33 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu Jun 4 10:22:33 2015 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/KeyValue.java     | 15 +++++++++++++--
 .../java/org/apache/hadoop/hbase/KeyValueUtil.java |  6 +++---
 .../org/apache/hadoop/hbase/NoTagsKeyValue.java    |  4 +---
 .../hadoop/hbase/io/ByteBufferOutputStream.java    | 13 +++++++++++++
 .../io/encoding/BufferedDataBlockEncoder.java      | 17 ++++++++++++++---
 5 files changed, 44 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bb62d5b2/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index b2343f1..315e9a3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -37,6 +37,7 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -2532,12 +2533,22 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (!withTags) {
       length = this.getKeyLength() + this.getValueLength() + KEYVALUE_INFRASTRUCTURE_SIZE;
     }
-    // This does same as DataOuput#writeInt (big-endian, etc.)
-    StreamUtils.writeInt(out, length);
+    writeInt(out, length);
     out.write(this.bytes, this.offset, length);
     return length + Bytes.SIZEOF_INT;
   }
 
+  // This does same as DataOuput#writeInt (big-endian, etc.)
+  public static void writeInt(OutputStream out, int v) throws IOException {
+    // We have writeInt in ByteBufferOutputStream so that it can directly write int to underlying
+    // ByteBuffer in one step.
+    if (out instanceof ByteBufferOutputStream) {
+      ((ByteBufferOutputStream) out).writeInt(v);
+    } else {
+      StreamUtils.writeInt(out, v);
+    }
+  }
+
   /**
    * Comparator that compares row component only of a KeyValue.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb62d5b2/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
----------------------------------------------------------------------
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 3b0c05c..5035666 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
@@ -679,11 +679,11 @@ public class KeyValueUtil {
       int tlen = cell.getTagsLength();
 
       // write total length
-      StreamUtils.writeInt(out, length(rlen, flen, qlen, vlen, tlen, withTags));
+      KeyValue.writeInt(out, length(rlen, flen, qlen, vlen, tlen, withTags));
       // write key length
-      StreamUtils.writeInt(out, keyLength(rlen, flen, qlen));
+      KeyValue.writeInt(out, keyLength(rlen, flen, qlen));
       // write value length
-      StreamUtils.writeInt(out, vlen);
+      KeyValue.writeInt(out, vlen);
       // Write rowkey - 2 bytes rk length followed by rowkey bytes
       StreamUtils.writeShort(out, rlen);
       out.write(cell.getRowArray(), cell.getRowOffset(), rlen);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb62d5b2/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
index 6de6653..a571580 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -44,8 +43,7 @@ public class NoTagsKeyValue extends KeyValue {
   public int write(OutputStream out, boolean withTags) throws IOException {
     // In KeyValueUtil#oswrite we do a Cell serialization as KeyValue. Any changes doing here, pls
     // check KeyValueUtil#oswrite also and do necessary changes.
-    // This does same as DataOuput#writeInt (big-endian, etc.)
-    StreamUtils.writeInt(out, this.length);
+    writeInt(out, this.length);
     out.write(this.bytes, this.offset, this.length);
     return this.length + Bytes.SIZEOF_INT;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb62d5b2/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
index af12113..25b5d51 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.io;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.nio.channels.Channels;
 import java.nio.channels.WritableByteChannel;
 
@@ -57,6 +58,7 @@ public class ByteBufferOutputStream extends OutputStream {
    * @see #getByteBuffer()
    */
   public ByteBufferOutputStream(final ByteBuffer bb) {
+    assert bb.order() == ByteOrder.BIG_ENDIAN;
     this.buf = bb;
     this.buf.clear();
   }
@@ -128,6 +130,17 @@ public class ByteBufferOutputStream extends OutputStream {
     buf.put(b, off, len);
   }
 
+  /**
+   * Writes an <code>int</code> to the underlying output stream as four
+   * bytes, high byte first.
+   * @param i the <code>int</code> to write
+   * @throws IOException if an I/O error occurs.
+   */
+  public void writeInt(int i) throws IOException {
+    checkSizeAndGrow(Bytes.SIZEOF_INT);
+    this.buf.putInt(i);
+  }
+
   @Override
   public void flush() throws IOException {
     // noop

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb62d5b2/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index 62e81ab..8406a1c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Streamable;
 import org.apache.hadoop.hbase.SettableSequenceId;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
@@ -548,9 +549,9 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
     public int write(OutputStream out, boolean withTags) throws IOException {
       int lenToWrite = KeyValueUtil.length(rowLength, familyLength, qualifierLength, valueLength,
           tagsLength, withTags);
-      StreamUtils.writeInt(out, lenToWrite);
-      StreamUtils.writeInt(out, keyOnlyBuffer.length);
-      StreamUtils.writeInt(out, valueLength);
+      writeInt(out, lenToWrite);
+      writeInt(out, keyOnlyBuffer.length);
+      writeInt(out, valueLength);
       // Write key
       out.write(keyOnlyBuffer);
       // Write value
@@ -574,6 +575,16 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
     }
   }
 
+  private static void writeInt(OutputStream out, int v) throws IOException {
+    // We have writeInt in ByteBufferOutputStream so that it can directly write int to underlying
+    // ByteBuffer in one step.
+    if (out instanceof ByteBufferOutputStream) {
+      ((ByteBufferOutputStream) out).writeInt(v);
+    } else {
+      StreamUtils.writeInt(out, v);
+    }
+  }
+
   protected abstract static class
       BufferedEncodedSeeker<STATE extends SeekerState>
       implements EncodedSeeker {