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/10/05 08:30:42 UTC

hbase git commit: HBASE-14480 Small optimization in SingleByteBuff.

Repository: hbase
Updated Branches:
  refs/heads/master dd955fa41 -> b014ba124


HBASE-14480 Small optimization in SingleByteBuff.


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

Branch: refs/heads/master
Commit: b014ba12431dc87903cd5065b14908f6a4ad458d
Parents: dd955fa
Author: anoopsjohn <an...@gmail.com>
Authored: Mon Oct 5 12:00:19 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Mon Oct 5 12:00:19 2015 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/nio/SingleByteBuff.java | 43 +++++++++++---
 .../apache/hadoop/hbase/util/UnsafeAccess.java  | 54 ++++++++++++++++-
 .../hadoop/hbase/nio/TestSingleByteBuff.java    | 62 ++++++++++++++++++++
 3 files changed, 150 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b014ba12/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
index dbda678..6d71eb2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
@@ -22,6 +22,9 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.ObjectIntPair;
+import org.apache.hadoop.hbase.util.UnsafeAccess;
+
+import sun.nio.ch.DirectBuffer;
 
 /**
  * An implementation of ByteBuff where a single BB backs the BBI. This just acts
@@ -30,11 +33,23 @@ import org.apache.hadoop.hbase.util.ObjectIntPair;
 @InterfaceAudience.Private
 public class SingleByteBuff extends ByteBuff {
 
+  private static final boolean UNSAFE_AVAIL = UnsafeAccess.isAvailable();
+
   // Underlying BB
   private final ByteBuffer buf;
 
+  // To access primitive values from underlying ByteBuffer using Unsafe
+  private long unsafeOffset;
+  private Object unsafeRef = null;
+
   public SingleByteBuff(ByteBuffer buf) {
     this.buf = buf;
+    if (buf.hasArray()) {
+      this.unsafeOffset = UnsafeAccess.BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset();
+      this.unsafeRef = buf.array();
+    } else {
+      this.unsafeOffset = ((DirectBuffer) buf).address();
+    }
   }
 
   @Override
@@ -134,12 +149,15 @@ public class SingleByteBuff extends ByteBuff {
 
   @Override
   public byte get(int index) {
-    return ByteBufferUtils.toByte(this.buf, index);
+    if (UNSAFE_AVAIL) {
+      return UnsafeAccess.toByte(this.unsafeRef, this.unsafeOffset + index);
+    }
+    return this.buf.get(index);
   }
 
   @Override
   public byte getByteAfterPosition(int offset) {
-    return ByteBufferUtils.toByte(this.buf, this.buf.position() + offset);
+    return get(this.buf.position() + offset);
   }
 
   @Override
@@ -219,12 +237,15 @@ public class SingleByteBuff extends ByteBuff {
 
   @Override
   public short getShort(int index) {
-    return ByteBufferUtils.toShort(this.buf, index);
+    if (UNSAFE_AVAIL) {
+      return UnsafeAccess.toShort(unsafeRef, unsafeOffset + index);
+    }
+    return this.buf.getShort(index);
   }
 
   @Override
   public short getShortAfterPosition(int offset) {
-    return ByteBufferUtils.toShort(this.buf, this.buf.position() + offset);
+    return getShort(this.buf.position() + offset);
   }
 
   @Override
@@ -240,12 +261,15 @@ public class SingleByteBuff extends ByteBuff {
 
   @Override
   public int getInt(int index) {
-    return ByteBufferUtils.toInt(this.buf, index);
+    if (UNSAFE_AVAIL) {
+      return UnsafeAccess.toInt(unsafeRef, unsafeOffset + index);
+    }
+    return this.buf.getInt(index);
   }
 
   @Override
   public int getIntAfterPosition(int offset) {
-    return ByteBufferUtils.toInt(this.buf, this.buf.position() + offset);
+    return getInt(this.buf.position() + offset);
   }
 
   @Override
@@ -261,12 +285,15 @@ public class SingleByteBuff extends ByteBuff {
 
   @Override
   public long getLong(int index) {
-    return ByteBufferUtils.toLong(this.buf, index);
+    if (UNSAFE_AVAIL) {
+      return UnsafeAccess.toLong(unsafeRef, unsafeOffset + index);
+    }
+    return this.buf.getLong(index);
   }
 
   @Override
   public long getLongAfterPosition(int offset) {
-    return ByteBufferUtils.toLong(this.buf, this.buf.position() + offset);
+    return getLong(this.buf.position() + offset);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/b014ba12/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
index 0cccee6..fd79b80 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
@@ -40,7 +40,7 @@ public final class UnsafeAccess {
   static final Unsafe theUnsafe;
 
   /** The offset to the first element in a byte array. */
-  static final long BYTE_ARRAY_BASE_OFFSET;
+  public static final long BYTE_ARRAY_BASE_OFFSET;
 
   static final boolean littleEndian = ByteOrder.nativeOrder()
       .equals(ByteOrder.LITTLE_ENDIAN);
@@ -182,6 +182,20 @@ public final class UnsafeAccess {
   }
 
   /**
+   * Reads a short value at the given Object's offset considering it was written in big-endian
+   * format.
+   * @param ref
+   * @param offset
+   * @return short value at offset
+   */
+  public static short toShort(Object ref, long offset) {
+    if (littleEndian) {
+      return Short.reverseBytes(theUnsafe.getShort(ref, offset));
+    }
+    return theUnsafe.getShort(ref, offset);
+  }
+
+  /**
    * Reads bytes at the given offset as a short value.
    * @param buf
    * @param offset
@@ -210,6 +224,20 @@ public final class UnsafeAccess {
   }
 
   /**
+   * Reads a int value at the given Object's offset considering it was written in big-endian
+   * format.
+   * @param ref
+   * @param offset
+   * @return int value at offset
+   */
+  public static int toInt(Object ref, long offset) {
+    if (littleEndian) {
+      return Integer.reverseBytes(theUnsafe.getInt(ref, offset));
+    }
+    return theUnsafe.getInt(ref, offset);
+  }
+
+  /**
    * Reads bytes at the given offset as an int value.
    * @param buf
    * @param offset
@@ -238,6 +266,20 @@ public final class UnsafeAccess {
   }
 
   /**
+   * Reads a long value at the given Object's offset considering it was written in big-endian
+   * format.
+   * @param ref
+   * @param offset
+   * @return long value at offset
+   */
+  public static long toLong(Object ref, long offset) {
+    if (littleEndian) {
+      return Long.reverseBytes(theUnsafe.getLong(ref, offset));
+    }
+    return theUnsafe.getLong(ref, offset);
+  }
+
+  /**
    * Reads bytes at the given offset as a long value.
    * @param buf
    * @param offset
@@ -411,4 +453,14 @@ public final class UnsafeAccess {
       return theUnsafe.getByte(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset);
     }
   }
+
+  /**
+   * Returns the byte at the given offset of the object
+   * @param ref
+   * @param offset
+   * @return the byte at the given offset
+   */
+  public static byte toByte(Object ref, long offset) {
+    return theUnsafe.getByte(ref, offset);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b014ba12/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java
new file mode 100644
index 0000000..98a1cc0
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestSingleByteBuff.java
@@ -0,0 +1,62 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.nio;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestSingleByteBuff {
+
+  @Test
+  public void testPositionalReads() {
+    // Off heap buffer
+    testPositionalReads(ByteBuffer.allocateDirect(15));
+    // On heap buffer
+    testPositionalReads(ByteBuffer.allocate(15));
+  }
+
+  private void testPositionalReads(ByteBuffer bb) {
+    int i = 9;
+    short s = 5;
+    byte b = 2;
+    long l = 1234L;
+    bb.putInt(i);
+    bb.putLong(l);
+    bb.put(b);
+    bb.putShort(s);
+    SingleByteBuff sbb = new SingleByteBuff(bb);
+    assertEquals(i, sbb.getInt(0));
+    assertEquals(l, sbb.getLong(4));
+    assertEquals(b, sbb.get(12));
+    assertEquals(s, sbb.getShort(13));
+    sbb.rewind();
+    assertEquals(i, sbb.getIntAfterPosition(0));
+    assertEquals(l, sbb.getLongAfterPosition(4));
+    assertEquals(b, sbb.getByteAfterPosition(12));
+    assertEquals(s, sbb.getShortAfterPosition(13));
+  }
+}