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 2014/06/15 12:53:45 UTC

git commit: HBASE-10771 Primitive type put/get APIs in ByteRange (Anoop)

Repository: hbase
Updated Branches:
  refs/heads/master 63f0dffdb -> 14c2c0029


HBASE-10771 Primitive type put/get APIs in ByteRange (Anoop)


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

Branch: refs/heads/master
Commit: 14c2c002966872373312f871f29cee9c1538cd9e
Parents: 63f0dff
Author: anoopsjohn <an...@gmail.com>
Authored: Sun Jun 15 16:22:11 2014 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Sun Jun 15 16:22:11 2014 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/ByteRange.java |  60 ++++++++++
 .../hadoop/hbase/util/PositionedByteRange.java  |  58 ++++++++++
 .../hadoop/hbase/util/SimpleByteRange.java      | 114 +++++++++++++++++++
 .../hbase/util/SimplePositionedByteRange.java   |  75 ++++++++++++
 .../util/TestByteRangeWithKVSerialization.java  |  89 +++++++++++++++
 .../hbase/util/TestPositionedByteRange.java     |  53 +++++++++
 .../hadoop/hbase/util/TestSimpleByteRange.java  |  41 +++++++
 7 files changed, 490 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/14c2c002/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
index 36dca9b..90cfa09 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
@@ -145,6 +145,34 @@ public interface ByteRange extends Comparable<ByteRange> {
   public byte get(int index);
 
   /**
+   * Retrieve the short value at {@code index}
+   * @param index zero-based index into this range
+   * @return the short value at {@code index}
+   */
+  public short getShort(int index);
+
+  /**
+   * Retrieve the int value at {@code index}
+   * @param index zero-based index into this range
+   * @return the int value at {@code index}
+   */
+  public int getInt(int index);
+
+  /**
+   * Retrieve the long value at {@code index}
+   * @param index zero-based index into this range
+   * @return the long value at {@code index}
+   */
+  public long getLong(int index);
+
+  /**
+   * Retrieve the long value at {@code index} which is stored as VLong
+   * @param index zero-based index into this range
+   * @return the long value at {@code index} which is stored as VLong
+   */
+  public long getVLong(int index);
+
+  /**
    * Fill {@code dst} with bytes from the range, starting from {@code index}.
    * @param index zero-based index into this range.
    * @param dst the destination of the copy.
@@ -172,6 +200,38 @@ public interface ByteRange extends Comparable<ByteRange> {
   public ByteRange put(int index, byte val);
 
   /**
+   * Store the short value at {@code index}
+   * @param index the index in the range where {@code val} is stored
+   * @param val the value to store
+   * @return this
+   */
+  public ByteRange putShort(int index, short val);
+
+  /**
+   * Store the int value at {@code index}
+   * @param index the index in the range where {@code val} is stored
+   * @param val the value to store
+   * @return this
+   */
+  public ByteRange putInt(int index, int val);
+
+  /**
+   * Store the long value at {@code index}
+   * @param index the index in the range where {@code val} is stored
+   * @param val the value to store
+   * @return this
+   */
+  public ByteRange putLong(int index, long val);
+
+  /**
+   * Store the long value at {@code index} as a VLong
+   * @param index the index in the range where {@code val} is stored
+   * @param val the value to store
+   * @return number of bytes written
+   */
+  public int putVLong(int index, long val);
+
+  /**
    * Store {@code val} at {@code index}.
    * @param index the index in the range where {@code val} is stored.
    * @param val the value to store.

http://git-wip-us.apache.org/repos/asf/hbase/blob/14c2c002/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
index 7d49538..5aebf31 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
@@ -70,6 +70,27 @@ public interface PositionedByteRange extends ByteRange {
   public byte get();
 
   /**
+   * Retrieve the next short value from this range.
+   */
+  public short getShort();
+
+  /**
+   * Retrieve the next int value from this range.
+   */
+  public int getInt();
+
+  /**
+   * Retrieve the next long value from this range.
+   */
+  public long getLong();
+
+  /**
+   * Retrieve the next long value, which is stored as VLong, from this range
+   * @return the long value which is stored as VLong
+   */
+  public long getVLong();
+
+  /**
    * Fill {@code dst} with bytes from the range, starting from {@code position}.
    * This range's {@code position} is incremented by the length of {@code dst},
    * the number of bytes copied.
@@ -98,6 +119,34 @@ public interface PositionedByteRange extends ByteRange {
   public PositionedByteRange put(byte val);
 
   /**
+   * Store short {@code val} at the next position in this range.
+   * @param val the new value.
+   * @return this.
+   */
+  public PositionedByteRange putShort(short val);
+
+  /**
+   * Store int {@code val} at the next position in this range.
+   * @param val the new value.
+   * @return this.
+   */
+  public PositionedByteRange putInt(int val);
+
+  /**
+   * Store long {@code val} at the next position in this range.
+   * @param val the new value.
+   * @return this.
+   */
+  public PositionedByteRange putLong(long val);
+
+  /**
+   * Store the long {@code val} at the next position as a VLong
+   * @param val the value to store
+   * @return number of bytes written
+   */
+  public int putVLong(long val);
+
+  /**
    * Store the content of {@code val} in this range, starting at the next position.
    * @param val the new value.
    * @return this.
@@ -145,6 +194,15 @@ public interface PositionedByteRange extends ByteRange {
   public PositionedByteRange put(int index, byte val);
 
   @Override
+  public PositionedByteRange putShort(int index, short val);
+
+  @Override
+  public PositionedByteRange putInt(int index, int val);
+
+  @Override
+  public PositionedByteRange putLong(int index, long val);
+
+  @Override
   public PositionedByteRange put(int index, byte[] val);
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/14c2c002/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
index 6f71c66..3205854 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
@@ -181,6 +181,63 @@ public class SimpleByteRange implements ByteRange {
   }
 
   @Override
+  public short getShort(int index) {
+    int offset = this.offset + index;
+    short n = 0;
+    n ^= bytes[offset] & 0xFF;
+    n <<= 8;
+    n ^= bytes[offset + 1] & 0xFF;
+    return n;
+  }
+
+  @Override
+  public int getInt(int index) {
+    int offset = this.offset + index;
+    int n = 0;
+    for (int i = offset; i < (offset + Bytes.SIZEOF_INT); i++) {
+      n <<= 8;
+      n ^= bytes[i] & 0xFF;
+    }
+    return n;
+  }
+
+  @Override
+  public long getLong(int index) {
+    int offset = this.offset + index;
+    long l = 0;
+    for (int i = offset; i < offset + Bytes.SIZEOF_LONG; i++) {
+      l <<= 8;
+      l ^= bytes[i] & 0xFF;
+    }
+    return l;
+  }
+
+  // Copied from com.google.protobuf.CodedInputStream
+  @Override
+  public long getVLong(int index) {
+    int shift = 0;
+    long result = 0;
+    while (shift < 64) {
+      final byte b = get(index++);
+      result |= (long) (b & 0x7F) << shift;
+      if ((b & 0x80) == 0) {
+        break;
+      }
+      shift += 7;
+    }
+    return result;
+  }
+
+  public static int getVLongSize(long val) {
+    int rPos = 0;
+    while ((val & ~0x7F) != 0) {
+      val >>>= 7;
+      rPos++;
+    }
+    return rPos + 1;
+  }
+
+  @Override
   public ByteRange get(int index, byte[] dst) {
     if (0 == dst.length) return this;
     return get(index, dst, 0, dst.length);
@@ -196,10 +253,66 @@ public class SimpleByteRange implements ByteRange {
   @Override
   public ByteRange put(int index, byte val) {
     bytes[offset + index] = val;
+    clearHashCache();
     return this;
   }
 
   @Override
+  public ByteRange putShort(int index, short val) {
+    // This writing is same as BB's putShort. When byte[] is wrapped in a BB and call putShort(),
+    // one can get the same result.
+    bytes[offset + index + 1] = (byte) val;
+    val >>= 8;
+    bytes[offset + index] = (byte) val;
+    clearHashCache();
+    return this;
+  }
+
+  @Override
+  public ByteRange putInt(int index, int val) {
+    // This writing is same as BB's putInt. When byte[] is wrapped in a BB and call getInt(), one
+    // can get the same result.
+    for (int i = Bytes.SIZEOF_INT - 1; i > 0; i--) {
+      bytes[offset + index + i] = (byte) val;
+      val >>>= 8;
+    }
+    bytes[offset + index] = (byte) val;
+    clearHashCache();
+    return this;
+  }
+
+  @Override
+  public ByteRange putLong(int index, long val) {
+    // This writing is same as BB's putLong. When byte[] is wrapped in a BB and call putLong(), one
+    // can get the same result.
+    for (int i = Bytes.SIZEOF_LONG - 1; i > 0; i--) {
+      bytes[offset + index + i] = (byte) val;
+      val >>>= 8;
+    }
+    bytes[offset + index] = (byte) val;
+    clearHashCache();
+    return this;
+  }
+
+  // Copied from com.google.protobuf.CodedOutputStream
+  @Override
+  public int putVLong(int index, long val) {
+    int rPos = 0;
+    while (true) {
+      if ((val & ~0x7F) == 0) {
+        bytes[offset + index + rPos] = (byte) val;
+        break;
+      } else {
+        bytes[offset + index + rPos] = (byte) ((val & 0x7F) | 0x80);
+        val >>>= 7;
+      }
+      rPos++;
+    }
+    clearHashCache();
+    return rPos + 1;
+  }
+
+  @Override
   public ByteRange put(int index, byte[] val) {
     if (0 == val.length) return this;
     return put(index, val, 0, val.length);
@@ -209,6 +322,7 @@ public class SimpleByteRange implements ByteRange {
   public ByteRange put(int index, byte[] val, int offset, int length) {
     if (0 == length) return this;
     System.arraycopy(val, offset, this.bytes, this.offset + index, length);
+    clearHashCache();
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/14c2c002/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
index c13539d..c436b91 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.util;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -157,6 +158,34 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi
   public byte get() { return get(position++); }
 
   @Override
+  public short getShort() {
+    short s = getShort(position);
+    position += Bytes.SIZEOF_SHORT;
+    return s;
+  }
+
+  @Override
+  public int getInt() {
+    int i = getInt(position);
+    position += Bytes.SIZEOF_INT;
+    return i;
+  }
+
+  @Override
+  public long getLong() {
+    long l = getLong(position);
+    position += Bytes.SIZEOF_LONG;
+    return l;
+  }
+
+  @Override
+  public long getVLong() {
+    long p = getVLong(position);
+    position += getVLongSize(p);
+    return p;
+  }
+
+  @Override
   public PositionedByteRange get(byte[] dst) {
     if (0 == dst.length) return this;
     return this.get(dst, 0, dst.length); // be clear we're calling self, not super
@@ -177,6 +206,34 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi
   }
 
   @Override
+  public PositionedByteRange putShort(short val) {
+    putShort(position, val);
+    position += Bytes.SIZEOF_SHORT;
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange putInt(int val) {
+    putInt(position, val);
+    position += Bytes.SIZEOF_INT;
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange putLong(long val) {
+    putLong(position, val);
+    position += Bytes.SIZEOF_LONG;
+    return this;
+  }
+
+  @Override
+  public int putVLong(long val) {
+    int len = putVLong(position, val);
+    position += len;
+    return len;
+  }
+
+  @Override
   public PositionedByteRange put(byte[] val) {
     if (0 == val.length) return this;
     return this.put(val, 0, val.length);
@@ -229,6 +286,24 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi
   public PositionedByteRange put(int index, byte val) { super.put(index, val); return this; }
 
   @Override
+  public PositionedByteRange putShort(int index, short val) {
+    super.putShort(index, val);
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange putInt(int index, int val) {
+    super.putInt(index, val);
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange putLong(int index, long val) {
+    super.putLong(index, val);
+    return this;
+  }
+
+  @Override
   public PositionedByteRange put(int index, byte[] val) { super.put(index, val); return this; }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/14c2c002/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java
new file mode 100644
index 0000000..9319fde
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java
@@ -0,0 +1,89 @@
+/**
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.Tag;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestByteRangeWithKVSerialization {
+
+  static void writeCell(PositionedByteRange pbr, KeyValue kv) throws Exception {
+    pbr.putInt(kv.getKeyLength());
+    pbr.putInt(kv.getValueLength());
+    pbr.put(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
+    pbr.put(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+    pbr.putShort(kv.getTagsLength());
+    pbr.put(kv.getTagsArray(), kv.getTagsOffset(), kv.getTagsLength());
+    pbr.putVLong(kv.getMvccVersion());
+  }
+
+  static KeyValue readCell(PositionedByteRange pbr) throws Exception {
+    int kvStartPos = pbr.getPosition();
+    int keyLen = pbr.getInt();
+    int valLen = pbr.getInt();
+    pbr.setPosition(pbr.getPosition() + keyLen + valLen); // Skip the key and value section
+    short tagsLen = pbr.getShort();
+    pbr.setPosition(pbr.getPosition() + tagsLen); // Skip the tags section
+    long mvcc = pbr.getVLong();
+    KeyValue kv = new KeyValue(pbr.getBytes(), kvStartPos,
+        (int) KeyValue.getKeyValueDataStructureSize(keyLen, valLen, tagsLen));
+    kv.setMvccVersion(mvcc);
+    return kv;
+  }
+
+  @Test
+  public void testWritingAndReadingCells() throws Exception {
+    final byte[] FAMILY = Bytes.toBytes("f1");
+    final byte[] QUALIFIER = Bytes.toBytes("q1");
+    final byte[] VALUE = Bytes.toBytes("v");
+    int kvCount = 1000000;
+    List<KeyValue> kvs = new ArrayList<KeyValue>(kvCount);
+    int totalSize = 0;
+    Tag[] tags = new Tag[] { new Tag((byte) 1, "tag1") };
+    for (int i = 0; i < kvCount; i++) {
+      KeyValue kv = new KeyValue(Bytes.toBytes(i), FAMILY, QUALIFIER, i, VALUE, tags);
+      kv.setMvccVersion(i);
+      kvs.add(kv);
+      totalSize += kv.getLength() + Bytes.SIZEOF_LONG;
+    }
+    PositionedByteRange pbr = new SimplePositionedByteRange(totalSize);
+    for (KeyValue kv : kvs) {
+      writeCell(pbr, kv);
+    }
+
+    PositionedByteRange pbr1 = new SimplePositionedByteRange(pbr.getBytes(), 0, pbr.getPosition());
+    for (int i = 0; i < kvCount; i++) {
+      KeyValue kv = readCell(pbr1);
+      KeyValue kv1 = kvs.get(i);
+      Assert.assertTrue(kv.equals(kv1));
+      Assert.assertTrue(Bytes.equals(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(),
+          kv1.getValueArray(), kv1.getValueOffset(), kv1.getValueLength()));
+      Assert.assertTrue(Bytes.equals(kv.getTagsArray(), kv.getTagsOffset(), kv.getTagsLength(),
+          kv1.getTagsArray(), kv1.getTagsOffset(), kv1.getTagsLength()));
+      Assert.assertEquals(kv1.getMvccVersion(), kv.getMvccVersion());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/14c2c002/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
index 422ea78..ab43ef3 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import java.nio.ByteBuffer;
+
 import org.apache.hadoop.hbase.SmallTests;
 import org.junit.Assert;
 import org.junit.Test;
@@ -65,4 +67,55 @@ public class TestPositionedByteRange {
     // set position to the end of the range; this should not throw.
     r.setPosition(3);
   }
+
+  @Test
+  public void testPutAndGetPrimitiveTypes() throws Exception {
+    PositionedByteRange pbr = new SimplePositionedByteRange(100);
+    int i1 = 18, i2 = 2;
+    short s1 = 0;
+    long l1 = 1234L;
+    pbr.putInt(i1);
+    pbr.putInt(i2);
+    pbr.putShort(s1);
+    pbr.putLong(l1);
+    pbr.putVLong(0);
+    pbr.putVLong(l1);
+    pbr.putVLong(Long.MAX_VALUE);
+    pbr.putVLong(Long.MIN_VALUE);
+    // rewind
+    pbr.setPosition(0);
+    Assert.assertEquals(i1, pbr.getInt());
+    Assert.assertEquals(i2, pbr.getInt());
+    Assert.assertEquals(s1, pbr.getShort());
+    Assert.assertEquals(l1, pbr.getLong());
+    Assert.assertEquals(0, pbr.getVLong());
+    Assert.assertEquals(l1, pbr.getVLong());
+    Assert.assertEquals(Long.MAX_VALUE, pbr.getVLong());
+    Assert.assertEquals(Long.MIN_VALUE, pbr.getVLong());
+  }
+
+  @Test
+  public void testPutGetAPIsCompareWithBBAPIs() throws Exception {
+    // confirm that the long/int/short writing is same as BBs
+    PositionedByteRange pbr = new SimplePositionedByteRange(100);
+    int i1 = -234, i2 = 2;
+    short s1 = 0;
+    long l1 = 1234L;
+    pbr.putInt(i1);
+    pbr.putShort(s1);
+    pbr.putInt(i2);
+    pbr.putLong(l1);
+    // rewind
+    pbr.setPosition(0);
+    Assert.assertEquals(i1, pbr.getInt());
+    Assert.assertEquals(s1, pbr.getShort());
+    Assert.assertEquals(i2, pbr.getInt());
+    Assert.assertEquals(l1, pbr.getLong());
+    // Read back using BB APIs
+    ByteBuffer bb = ByteBuffer.wrap(pbr.getBytes());
+    Assert.assertEquals(i1, bb.getInt());
+    Assert.assertEquals(s1, bb.getShort());
+    Assert.assertEquals(i2, bb.getInt());
+    Assert.assertEquals(l1, bb.getLong());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/14c2c002/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
index 4ee4335..1bd0969 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
@@ -68,4 +68,45 @@ public class TestSimpleByteRange {
     r.setLength(2);//verify we retained the 2nd byte, but dangerous in real code
     Assert.assertTrue(Bytes.equals(new byte[]{1, 3}, r.deepCopyToNewArray()));
   }
+
+  @Test
+  public void testPutandGetPrimitiveTypes() throws Exception {
+    ByteRange r = new SimpleByteRange(100);
+    int offset = 0;
+    int i1 = 18, i2 = 2;
+    short s1 = 0;
+    long l1 = 1234L, l2 = 0;
+    r.putInt(offset, i1);
+    offset += Bytes.SIZEOF_INT;
+    r.putInt(offset, i2);
+    offset += Bytes.SIZEOF_INT;
+    r.putShort(offset, s1);
+    offset += Bytes.SIZEOF_SHORT;
+    r.putLong(offset, l1);
+    offset += Bytes.SIZEOF_LONG;
+    int len = r.putVLong(offset, l1);
+    offset += len;
+    len = r.putVLong(offset, l2);
+    offset += len;
+    len = r.putVLong(offset, Long.MAX_VALUE);
+    offset += len;
+    len = r.putVLong(offset, Long.MIN_VALUE);
+
+    offset = 0;
+    Assert.assertEquals(i1, r.getInt(offset));
+    offset += Bytes.SIZEOF_INT;
+    Assert.assertEquals(i2, r.getInt(offset));
+    offset += Bytes.SIZEOF_INT;
+    Assert.assertEquals(s1, r.getShort(offset));
+    offset += Bytes.SIZEOF_SHORT;
+    Assert.assertEquals(l1, r.getLong(offset));
+    offset += Bytes.SIZEOF_LONG;
+    Assert.assertEquals(l1, r.getVLong(offset));
+    offset += SimpleByteRange.getVLongSize(l1);
+    Assert.assertEquals(l2, r.getVLong(offset));
+    offset += SimpleByteRange.getVLongSize(l2);
+    Assert.assertEquals(Long.MAX_VALUE, r.getVLong(offset));
+    offset += SimpleByteRange.getVLongSize(Long.MAX_VALUE);
+    Assert.assertEquals(Long.MIN_VALUE, r.getVLong(offset));
+  }
 }