You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2013/08/11 15:32:21 UTC

svn commit: r1512921 - in /hbase/trunk: hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apache/hadoop/hbase/util/ hbase-common/src/test/java/org/apache/hadoop/hbase/util/ hbase-prefix-tree/src/main/java/org/apache/had...

Author: nkeywal
Date: Sun Aug 11 13:32:20 2013
New Revision: 1512921

URL: http://svn.apache.org/r1512921
Log:
HBASE-9091  Update ByteRange to maintain consumer's position (Nick Dimiduck)

Added:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
Removed:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeTool.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRange.java
Modified:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowNodeWriter.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerNode.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/ByteRangeSet.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/ByteRangeTreeSet.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizer.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTreeDepth.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnBuilder.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataRandom.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataSimple.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataUrls.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/bytes/TestByteRange.java

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java Sun Aug 11 13:32:20 2013
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IterableUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -175,7 +175,7 @@ public class KeyValueUtil {
    * Increment the row bytes and clear the other fields
    */
   public static KeyValue createFirstKeyInIncrementedRow(final Cell in){
-    byte[] thisRow = new ByteRange(in.getRowArray(), in.getRowOffset(), in.getRowLength())
+    byte[] thisRow = new SimpleByteRange(in.getRowArray(), in.getRowOffset(), in.getRowLength())
         .deepCopyToNewArray();
     byte[] nextRow = Bytes.unsignedCopyAndIncrement(thisRow);
     return KeyValue.createFirstOnRow(nextRow);

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java Sun Aug 11 13:32:20 2013
@@ -18,282 +18,229 @@
 
 package org.apache.hadoop.hbase.util;
 
-
-
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * Lightweight, reusable class for specifying ranges of byte[]'s. CompareTo and equals methods are
- * lexicographic, which is native to HBase.
- * <p/>
- * This class differs from ByteBuffer:
- * <li/>On-heap bytes only
- * <li/>Implements equals, hashCode, and compareTo so that it can be used in standard java
- * Collections, similar to String.
- * <li/>Does not maintain mark/position iterator state inside the class. Doing so leads to many bugs
- * in complex applications.
- * <li/>Allows the addition of simple core methods like this.copyTo(that, offset).
- * <li/>Can be reused in tight loops like a major compaction which can save significant amounts of
- * garbage.
- * <li/>(Without reuse, we throw off garbage like this thing:
- * http://www.youtube.com/watch?v=lkmBH-MjZF4
- * <p/>
- * Mutable, and always evaluates equals, hashCode, and compareTo based on the current contents.
- * <p/>
- * Can contain convenience methods for comparing, printing, cloning, spawning new arrays, copying to
- * other arrays, etc. Please place non-core methods into {@link ByteRangeTool}.
- * <p/>
- * We may consider converting this to an interface and creating separate implementations for a
- * single byte[], a paged byte[] (growable byte[][]), a ByteBuffer, etc
+ * Lightweight, reusable class for specifying ranges of byte[]'s.
+ * <p>
+ * {@code ByteRange} maintains an underlying byte[] and a viewport into that
+ * byte[] as a range of bytes. The {@code ByteRange} is a mutable, reusable
+ * object, so the underlying byte[] can be modified after instantiation. This
+ * is done using the {@link #set(byte[])} and {@link #unset()} methods. Direct
+ * access to the byte[] is also available via {@link #getBytes()}. The viewport
+ * is defined by an {@code offset} into the byte[] and a {@code length}. The
+ * range of bytes is 0-indexed, and is accessed by index via the
+ * {@link #get(int)} and {@link #put(int, byte)} methods.
+ * </p>
+ * <p>
+ * This interface differs from ByteBuffer:
+ * <li>On-heap bytes only</li>
+ * <li>Raw {@code byte} access only; does not encode other primitives.</li>
+ * <li>Implements {@code equals(Object)}, {@code #hashCode()}, and
+ * {@code #compareTo(ByteRange)} so that it can be used in standard java
+ * Collections. Comparison operations are lexicographic, which is native to
+ * HBase.</li>
+ * <li>Allows the addition of simple core methods like the deep and shallow
+ * copy methods.</li>
+ * <li>Can be reused in tight loops like a major compaction which can save
+ * significant amounts of garbage. (Without reuse, we throw off garbage like
+ * <a href="http://www.youtube.com/watch?v=lkmBH-MjZF4">this thing</a>.)</li>
+ * </p>
+ * <p>
+ * Mutable, and always evaluates {@code #equals(Object)}, {@code #hashCode()},
+ * and {@code #compareTo(ByteRange)} based on the current contents.
+ * </p>
+ * <p>
+ * Can contain convenience methods for comparing, printing, cloning, spawning
+ * new arrays, copying to other arrays, etc. Please place non-core methods into
+ * {@link ByteRangeUtils}.
+ * </p>
  */
-public class ByteRange implements Comparable<ByteRange> {
-
-  private static final int UNSET_HASH_VALUE = -1;
-
-
-  /********************** fields *****************************/
-
-  // Do not make these final, as the intention is to reuse objects of this class
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface ByteRange extends Comparable<ByteRange> {
 
   /**
-   * The array containing the bytes in this range.  It will be >= length.
+   * The underlying byte[].
    */
-  private byte[] bytes;
+  public byte[] getBytes();
 
   /**
-   * The index of the first byte in this range.  ByteRange.get(0) will return bytes[offset].
+   * Nullifies this ByteRange. That is, it becomes a husk, being a range over
+   * no byte[] whatsoever.
+   * @return this
    */
-  private int offset;
+  public ByteRange unset();
 
   /**
-   * The number of bytes in the range.  Offset + length must be <= bytes.length
+   * Reuse this {@code ByteRange} over a new byte[]. {@code offset} is set to
+   * 0 and {@code length} is set to {@code capacity}.
+   * @param capacity the size of a new byte[].
+   * @return this
    */
-  private int length;
+  public ByteRange set(int capacity);
 
   /**
-   * Variable for lazy-caching the hashCode of this range.  Useful for frequently used ranges,
-   * long-lived ranges, or long ranges.
+   * Reuse this {@code ByteRange} over a new byte[]. {@code offset} is set to
+   * 0 and {@code length} is set to {@code bytes.length}. A null {@code bytes}
+   * IS supported, in which case this method will behave equivalently to
+   * {@link #unset()}.
+   * @param bytes the array to wrap.
+   * @return this
    */
-  private int hash = UNSET_HASH_VALUE;
-
+  public ByteRange set(byte[] bytes);
 
-  /********************** construct ***********************/
-
-  public ByteRange() {
-    set(new byte[0]);//Could probably get away with a null array if the need arises.
-  }
+  /**
+   * Reuse this {@code ByteRange} over a new byte[]. A null {@code bytes} IS
+   * supported, in which case this method will behave equivalently to
+   * {@link #unset()}, regardless of the values of {@code offset} and
+   * {@code length}.
+   * @param bytes The array to wrap.
+   * @param offset The offset into {@code bytes} considered the beginning of
+   *            this range.
+   * @param length The length of this range.
+   * @return this.
+   */
+  public ByteRange set(byte[] bytes, int offset, int length);
 
-  public ByteRange(byte[] bytes) {
-    set(bytes);
-  }
+  /**
+   * The offset, the index into the underlying byte[] at which this range
+   * begins.
+   * @see #getBytes()
+   */
+  public int getOffset();
 
-  public ByteRange(byte[] bytes, int offset, int length) {
-    set(bytes, offset, length);
-  }
+  /**
+   * Update the beginning of this range. {@code offset + length} may not be
+   * greater than {@code bytes.length}.
+   * @param offset the new start of this range.
+   * @return this.
+   */
+  public ByteRange setOffset(int offset);
 
+  /**
+   * The length of the range.
+   */
+  public int getLength();
 
-  /********************** write methods *************************/
+  /**
+   * Update the length of this range. {@code offset + length} should not be
+   * greater than {@code bytes.length}.
+   * @param length The new length of this range.
+   * @return this.
+   */
+  public ByteRange setLength(int length);
 
-  public ByteRange clear() {
-    clearHashCache();
-    bytes = null;
-    offset = 0;
-    length = 0;
-    return this;
-  }
+  /**
+   * @return true when this range is of zero length, false otherwise.
+   */
+  public boolean isEmpty();
 
-  public ByteRange set(byte[] bytes) {
-    clearHashCache();
-    this.bytes = bytes;
-    this.offset = 0;
-    this.length = ArrayUtils.length(bytes);
-    return this;
-  }
+  /**
+   * Retrieve the byte at {@code index}.
+   * @param index zero-based index into this range.
+   * @return single byte at index.
+   */
+  public byte get(int index);
 
-  public ByteRange set(byte[] bytes, int offset, int length) {
-    clearHashCache();
-    this.bytes = bytes;
-    this.offset = offset;
-    this.length = length;
-    return this;
-  }
+  /**
+   * 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.
+   * @return this.
+   */
+  public ByteRange get(int index, byte[] dst);
 
-  public void setLength(int length) {
-    clearHashCache();
-    this.length = length;
-  }
+  /**
+   * Fill {@code dst} with bytes from the range, starting from {@code index}.
+   * {@code length} bytes are copied into {@code dst}, starting at {@code offset}.
+   * @param index zero-based index into this range.
+   * @param dst the destination of the copy.
+   * @param offset the offset into {@code dst} to start the copy.
+   * @param length the number of bytes to copy into {@code dst}.
+   * @return this.
+   */
+  public ByteRange get(int index, byte[] dst, int offset, int length);
 
+  /**
+   * Store {@code val} 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 put(int index, byte val);
 
-  /*********** read methods (add non-core methods to ByteRangeUtils) *************/
+  /**
+   * Store {@code val} 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 put(int index, byte[] val);
 
   /**
-   * @param index zero-based index
-   * @return single byte at index
+   * Store {@code length} bytes from {@code val} into this range, starting at
+   * {@code index}. Bytes from {@code val} are copied starting at {@code offset}
+   * into the range.
+   * @param index position in this range to start the copy.
+   * @param val the value to store.
+   * @param offset the offset in {@code val} from which to start copying.
+   * @param length the number of bytes to copy from {@code val}.
+   * @return this.
    */
-  public byte get(int index) {
-    return bytes[offset + index];
-  }
+  public ByteRange put(int index, byte[] val, int offset, int length);
 
   /**
-   * Instantiate a new byte[] with exact length, which is at least 24 bytes + length.  Copy the
-   * contents of this range into it.
+   * Instantiate a new byte[] with exact length, which is at least 24 bytes +
+   * length. Copy the contents of this range into it.
    * @return The newly cloned byte[].
    */
-  public byte[] deepCopyToNewArray() {
-    byte[] result = new byte[length];
-    System.arraycopy(bytes, offset, result, 0, length);
-    return result;
-  }
+  public byte[] deepCopyToNewArray();
 
   /**
-   * Create a new ByteRange with new backing byte[] and copy the state of this range into the new
-   * range.  Copy the hash over if it is already calculated.
+   * Create a new {@code ByteRange} with new backing byte[] containing a copy
+   * of the content from {@code this} range's window.
    * @return Deep copy
    */
-  public ByteRange deepCopy() {
-    ByteRange clone = new ByteRange(deepCopyToNewArray());
-    if (isHashCached()) {
-      clone.hash = hash;
-    }
-    return clone;
-  }
+  public ByteRange deepCopy();
 
   /**
-   * Wrapper for System.arraycopy.  Copy the contents of this range into the provided array.
+   * Wrapper for System.arraycopy. Copy the contents of this range into the
+   * provided array.
    * @param destination Copy to this array
    * @param destinationOffset First index in the destination array.
    */
-  public void deepCopyTo(byte[] destination, int destinationOffset) {
-    System.arraycopy(bytes, offset, destination, destinationOffset, length);
-  }
+  public void deepCopyTo(byte[] destination, int destinationOffset);
 
   /**
-   * Wrapper for System.arraycopy. Copy the contents of this range into the provided array.
-   * @param innerOffset Start copying from this index in this source ByteRange. First byte copied is
-   *          bytes[offset + innerOffset]
+   * Wrapper for System.arraycopy. Copy the contents of this range into the
+   * provided array.
+   * @param innerOffset Start copying from this index in this source
+   *          ByteRange. First byte copied is bytes[offset + innerOffset]
    * @param copyLength Copy this many bytes
    * @param destination Copy to this array
    * @param destinationOffset First index in the destination array.
    */
   public void deepCopySubRangeTo(int innerOffset, int copyLength, byte[] destination,
-      int destinationOffset) {
-    System.arraycopy(bytes, offset + innerOffset, destination, destinationOffset, copyLength);
-  }
+      int destinationOffset);
 
   /**
-   * Create a new ByteRange that points at this range's byte[]. The new range can have different
-   * values for offset and length, but modifying the shallowCopy will modify the bytes in this
-   * range's array. Pass over the hash code if it is already cached.
+   * Create a new {@code ByteRange} that points at this range's byte[].
+   * Modifying the shallowCopy will modify the bytes in this range's array.
+   * Pass over the hash code if it is already cached.
+   * @return new {@code ByteRange} object referencing this range's byte[].
+   */
+  public ByteRange shallowCopy();
+
+  /**
+   * Create a new {@code ByteRange} that points at this range's byte[]. The new
+   * range can have different values for offset and length, but modifying the
+   * shallowCopy will modify the bytes in this range's array. Pass over the
+   * hash code if it is already cached.
    * @param innerOffset First byte of clone will be this.offset + copyOffset.
    * @param copyLength Number of bytes in the clone.
-   * @return new ByteRange object referencing this range's byte[].
+   * @return new {@code ByteRange} object referencing this range's byte[].
    */
-  public ByteRange shallowCopySubRange(int innerOffset, int copyLength) {
-    ByteRange clone = new ByteRange(bytes, offset + innerOffset, copyLength);
-    if (isHashCached()) {
-      clone.hash = hash;
-    }
-    return clone;
-  }
-
-  //TODO move to ByteRangeUtils because it is non-core method
-  public int numEqualPrefixBytes(ByteRange that, int thatInnerOffset) {
-    int maxCompares = Math.min(length, that.length - thatInnerOffset);
-    for (int i = 0; i < maxCompares; ++i) {
-      if (bytes[offset + i] != that.bytes[that.offset + thatInnerOffset + i]) {
-        return i;
-      }
-    }
-    return maxCompares;
-  }
-
-  public byte[] getBytes() {
-    return bytes;
-  }
-
-  public int getOffset() {
-    return offset;
-  }
-
-  public int getLength() {
-    return length;
-  }
-
-  public boolean isEmpty(){
-    return isEmpty(this);
-  }
-
-  public boolean notEmpty(){
-    return notEmpty(this);
-  }
-
-
-  /******************* static methods ************************/
-
-  public static boolean isEmpty(ByteRange range){
-    return range == null || range.length == 0;
-  }
-
-  public static boolean notEmpty(ByteRange range){
-    return range != null && range.length > 0;
-  }
-
-  /******************* standard methods *********************/
-
-  @Override
-  public boolean equals(Object thatObject) {
-    if (thatObject == null){
-      return false;
-    }
-    if (this == thatObject) {
-      return true;
-    }
-    if (hashCode() != thatObject.hashCode()) {
-      return false;
-    }
-    if (!(thatObject instanceof ByteRange)) {
-      return false;
-    }
-    ByteRange that = (ByteRange) thatObject;
-    return Bytes.equals(bytes, offset, length, that.bytes, that.offset, that.length);
-  }
-
-  @Override
-  public int hashCode() {
-    if (isHashCached()) {// hash is already calculated and cached
-      return hash;
-    }
-    if (this.isEmpty()) {// return 0 for empty ByteRange
-      hash = 0;
-      return hash;
-    }
-    int off = offset;
-    hash = 0;
-    for (int i = 0; i < length; i++) {
-      hash = 31 * hash + bytes[off++];
-    }
-    return hash;
-  }
-
-  private boolean isHashCached() {
-    return hash != UNSET_HASH_VALUE;
-  }
-
-  private void clearHashCache() {
-    hash = UNSET_HASH_VALUE;
-  }
-
-  /**
-   * Bitwise comparison of each byte in the array.  Unsigned comparison, not paying attention to
-   * java's signed bytes.
-   */
-  @Override
-  public int compareTo(ByteRange other) {
-    return Bytes.compareTo(bytes, offset, length, other.bytes, other.offset, other.length);
-  }
-
-  @Override
-  public String toString() {
-    return Bytes.toStringBinary(bytes, offset, length);
-  }
-
-}
+  public ByteRange shallowCopySubRange(int innerOffset, int copyLength);
+}
\ No newline at end of file

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java?rev=1512921&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java Sun Aug 11 13:32:20 2013
@@ -0,0 +1,82 @@
+/*
+ * 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.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Utility methods for working with {@link ByteRange}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ByteRangeUtils {
+
+  public static int numEqualPrefixBytes(ByteRange left, ByteRange right, int rightInnerOffset) {
+    int maxCompares = Math.min(left.getLength(), right.getLength() - rightInnerOffset);
+    final byte[] lbytes = left.getBytes(), rbytes = right.getBytes();
+    final int loffset = left.getOffset(), roffset = right.getOffset();
+    for (int i = 0; i < maxCompares; ++i) {
+      if (lbytes[loffset + i] != rbytes[roffset + rightInnerOffset + i]) {
+        return i;
+      }
+    }
+    return maxCompares;
+  }
+
+  public static ArrayList<byte[]> copyToNewArrays(Collection<ByteRange> ranges) {
+    if (ranges == null) {
+      return new ArrayList<byte[]>(0);
+    }
+    ArrayList<byte[]> arrays = Lists.newArrayListWithCapacity(ranges.size());
+    for (ByteRange range : ranges) {
+      arrays.add(range.deepCopyToNewArray());
+    }
+    return arrays;
+  }
+
+  public static ArrayList<ByteRange> fromArrays(Collection<byte[]> arrays) {
+    if (arrays == null) {
+      return new ArrayList<ByteRange>(0);
+    }
+    ArrayList<ByteRange> ranges = Lists.newArrayListWithCapacity(arrays.size());
+    for (byte[] array : arrays) {
+      ranges.add(new SimpleByteRange(array));
+    }
+    return ranges;
+  }
+
+  public static void write(OutputStream os, ByteRange byteRange) throws IOException {
+    os.write(byteRange.getBytes(), byteRange.getOffset(), byteRange.getLength());
+  }
+
+  public static void write(OutputStream os, ByteRange byteRange, int byteRangeInnerOffset)
+      throws IOException {
+    os.write(byteRange.getBytes(), byteRange.getOffset() + byteRangeInnerOffset,
+      byteRange.getLength() - byteRangeInnerOffset);
+  }
+
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java?rev=1512921&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java Sun Aug 11 13:32:20 2013
@@ -0,0 +1,161 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * <p>
+ * Extends {@link ByteRange} with additional methods to support tracking a
+ * consumers position within the viewport. The API is extended with methods
+ * {@link #get()} and {@link #put(byte)} for interacting with the backing
+ * array from the current position forward. This frees the caller from managing
+ * their own index into the array.
+ * </p>
+ * <p>
+ * Designed to be a slimmed-down, mutable alternative to {@link ByteBuffer}.
+ * </p>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface PositionedByteRange extends ByteRange {
+
+  // net new API is here.
+
+  /**
+   * The current {@code position} marker. This valuae is 0-indexed, relative to
+   * the beginning of the range.
+   */
+  public int getPosition();
+
+  /**
+   * Update the {@code position} index. May not be greater than {@code length}.
+   * @param position the new position in this range.
+   * @return this.
+   */
+  public PositionedByteRange setPosition(int position);
+
+  /**
+   * The number of bytes remaining between position and the end of the range.
+   */
+  public int getRemaining();
+
+  /**
+   * Retrieve the next byte from this range without incrementing position.
+   */
+  public byte peek();
+
+  /**
+   * Retrieve the next byte from this range.
+   */
+  public byte get();
+
+  /**
+   * 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.
+   * @param dst the destination of the copy.
+   * @return this.
+   */
+  public PositionedByteRange get(byte[] dst);
+
+  /**
+   * Fill {@code dst} with bytes from the range, starting from the current
+   * {@code position}. {@code length} bytes are copied into {@code dst},
+   * starting at {@code offset}. This range's {@code position} is incremented
+   * by the number of bytes copied.
+   * @param dst the destination of the copy.
+   * @param offset the offset into {@code dst} to start the copy.
+   * @param length the number of bytes to copy into {@code dst}.
+   * @return this.
+   */
+  public PositionedByteRange get(byte[] dst, int offset, int length);
+
+  /**
+   * Store {@code val} at the next position in this range.
+   * @param val the new value.
+   * @return this.
+   */
+  public PositionedByteRange put(byte val);
+
+  /**
+   * Store the content of {@code val} in this range, starting at the next position.
+   * @param val the new value.
+   * @return this.
+   */
+  public PositionedByteRange put(byte[] val);
+
+  /**
+   * Store {@code length} bytes from {@code val} into this range. Bytes from
+   * {@code val} are copied starting at {@code offset} into the range, starting at
+   * the current position.
+   * @param val the new value.
+   * @param offset the offset in {@code val} from which to start copying.
+   * @param length the number of bytes to copy from {@code val}.
+   * @return this.
+   */
+  public PositionedByteRange put(byte[] val, int offset, int length);
+
+  // override parent interface declarations to return this interface.
+
+  @Override
+  public PositionedByteRange unset();
+
+  @Override
+  public PositionedByteRange set(int capacity);
+
+  @Override
+  public PositionedByteRange set(byte[] bytes);
+
+  @Override
+  public PositionedByteRange set(byte[] bytes, int offset, int length);
+
+  @Override
+  public PositionedByteRange setOffset(int offset);
+
+  @Override
+  public PositionedByteRange setLength(int length);
+
+  @Override
+  public PositionedByteRange get(int index, byte[] dst);
+
+  @Override
+  public PositionedByteRange get(int index, byte[] dst, int offset, int length);
+
+  @Override
+  public PositionedByteRange put(int index, byte val);
+
+  @Override
+  public PositionedByteRange put(int index, byte[] val);
+
+  @Override
+  public PositionedByteRange put(int index, byte[] val, int offset, int length);
+
+  @Override
+  public PositionedByteRange deepCopy();
+
+  @Override
+  public PositionedByteRange shallowCopy();
+
+  @Override
+  public PositionedByteRange shallowCopySubRange(int innerOffset, int copyLength);
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java?rev=1512921&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java Sun Aug 11 13:32:20 2013
@@ -0,0 +1,325 @@
+/*
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A basic {@link ByteRange} implementation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class SimpleByteRange implements ByteRange {
+
+  private static final int UNSET_HASH_VALUE = -1;
+
+  // Note to maintainers: Do not make these final, as the intention is to
+  // reuse objects of this class
+
+  /**
+   * The array containing the bytes in this range. It will be >= length.
+   */
+  protected byte[] bytes;
+
+  /**
+   * The index of the first byte in this range. {@code ByteRange.get(0)} will
+   * return bytes[offset].
+   */
+  protected int offset;
+
+  /**
+   * The number of bytes in the range.  Offset + length must be <= bytes.length
+   */
+  protected int length;
+
+  /**
+   * Variable for lazy-caching the hashCode of this range. Useful for
+   * frequently used ranges, long-lived ranges, or long ranges.
+   */
+  private int hash = UNSET_HASH_VALUE;
+
+  /**
+   * Create a new {@code ByteRange} lacking a backing array and with an
+   * undefined viewport.
+   */
+  public SimpleByteRange() {
+    unset();
+  }
+
+  /**
+   * Create a new {@code ByteRange} over a new backing array of size
+   * {@code capacity}. The range's offset and length are 0 and {@code capacity},
+   * respectively.
+   * @param capacity the size of the backing array.
+   */
+  public SimpleByteRange(int capacity) {
+    this(new byte[capacity]);
+  }
+
+  /**
+   * Create a new {@code ByteRange} over the provided {@code bytes}.
+   * @param bytes The array to wrap.
+   */
+  public SimpleByteRange(byte[] bytes) {
+    set(bytes);
+  }
+
+  /**
+   * Create a new {@code ByteRange} over the provided {@code bytes}.
+   * @param bytes The array to wrap.
+   * @param offset The offset into {@code bytes} considered the beginning
+   *          of this range.
+   * @param length The length of this range.
+   */
+  public SimpleByteRange(byte[] bytes, int offset, int length) {
+    set(bytes, offset, length);
+  }
+
+  //
+  // methods for managing the backing array and range viewport
+  //
+
+  @Override
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  @Override
+  public ByteRange unset() {
+    clearHashCache();
+    this.bytes = null;
+    this.offset = 0;
+    this.length = 0;
+    return this;
+  }
+
+  @Override
+  public ByteRange set(int capacity) {
+    return set(new byte[capacity]);
+  }
+
+  @Override
+  public ByteRange set(byte[] bytes) {
+    if (null == bytes) return unset();
+    clearHashCache();
+    this.bytes = bytes;
+    this.offset = 0;
+    this.length = bytes.length;
+    return this;
+  }
+
+  @Override
+  public ByteRange set(byte[] bytes, int offset, int length) {
+    if (null == bytes) return unset();
+    clearHashCache();
+    this.bytes = bytes;
+    this.offset = offset;
+    this.length = length;
+    return this;
+  }
+
+  @Override
+  public int getOffset() {
+    return offset;
+  }
+
+  @Override
+  public ByteRange setOffset(int offset) {
+    clearHashCache();
+    this.offset = offset;
+    return this;
+  }
+
+  @Override
+  public int getLength() {
+    return length;
+  }
+
+  @Override
+  public ByteRange setLength(int length) {
+    clearHashCache();
+    this.length = length;
+    return this;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return isEmpty(this);
+  }
+
+  /**
+   * @return true when {@code range} is of zero length, false otherwise.
+   */
+  public static boolean isEmpty(ByteRange range) {
+    return range == null || range.getLength() == 0;
+  }
+
+  //
+  // methods for retrieving data
+  //
+
+  @Override
+  public byte get(int index) {
+    return bytes[offset + index];
+  }
+
+  @Override
+  public ByteRange get(int index, byte[] dst) {
+    if (0 == dst.length) return this;
+    return get(index, dst, 0, dst.length);
+  }
+
+  @Override
+  public ByteRange get(int index, byte[] dst, int offset, int length) {
+    if (0 == length) return this;
+    System.arraycopy(this.bytes, this.offset + index, dst, offset, length);
+    return this;
+  }
+
+  @Override
+  public ByteRange put(int index, byte val) {
+    bytes[offset + index] = val;
+    return this;
+  }
+
+  @Override
+  public ByteRange put(int index, byte[] val) {
+    if (0 == val.length) return this;
+    return put(index, val, 0, val.length);
+  }
+
+  @Override
+  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);
+    return this;
+  }
+
+  //
+  // methods for duplicating the current instance
+  //
+
+  @Override
+  public byte[] deepCopyToNewArray() {
+    byte[] result = new byte[length];
+    System.arraycopy(bytes, offset, result, 0, length);
+    return result;
+  }
+
+  @Override
+  public ByteRange deepCopy() {
+    SimpleByteRange clone = new SimpleByteRange(deepCopyToNewArray());
+    if (isHashCached()) {
+      clone.hash = hash;
+    }
+    return clone;
+  }
+
+  @Override
+  public void deepCopyTo(byte[] destination, int destinationOffset) {
+    System.arraycopy(bytes, offset, destination, destinationOffset, length);
+  }
+
+  @Override
+  public void deepCopySubRangeTo(int innerOffset, int copyLength, byte[] destination,
+      int destinationOffset) {
+    System.arraycopy(bytes, offset + innerOffset, destination, destinationOffset, copyLength);
+  }
+
+  @Override
+  public ByteRange shallowCopy() {
+    SimpleByteRange clone = new SimpleByteRange(bytes, offset, length);
+    if (isHashCached()) {
+      clone.hash = hash;
+    }
+    return clone;
+  }
+
+  @Override
+  public ByteRange shallowCopySubRange(int innerOffset, int copyLength) {
+    SimpleByteRange clone = new SimpleByteRange(bytes, offset + innerOffset, copyLength);
+    if (isHashCached()) {
+      clone.hash = hash;
+    }
+    return clone;
+  }
+
+  //
+  // methods used for comparison
+  //
+
+  @Override
+  public boolean equals(Object thatObject) {
+    if (thatObject == null){
+      return false;
+    }
+    if (this == thatObject) {
+      return true;
+    }
+    if (hashCode() != thatObject.hashCode()) {
+      return false;
+    }
+    if (!(thatObject instanceof SimpleByteRange)) {
+      return false;
+    }
+    SimpleByteRange that = (SimpleByteRange) thatObject;
+    return Bytes.equals(bytes, offset, length, that.bytes, that.offset, that.length);
+  }
+
+  @Override
+  public int hashCode() {
+    if (isHashCached()) {// hash is already calculated and cached
+      return hash;
+    }
+    if (this.isEmpty()) {// return 0 for empty ByteRange
+      hash = 0;
+      return hash;
+    }
+    int off = offset;
+    hash = 0;
+    for (int i = 0; i < length; i++) {
+      hash = 31 * hash + bytes[off++];
+    }
+    return hash;
+  }
+
+  private boolean isHashCached() {
+    return hash != UNSET_HASH_VALUE;
+  }
+
+  protected void clearHashCache() {
+    hash = UNSET_HASH_VALUE;
+  }
+
+  /**
+   * Bitwise comparison of each byte in the array.  Unsigned comparison, not
+   * paying attention to java's signed bytes.
+   */
+  @Override
+  public int compareTo(ByteRange other) {
+    return Bytes.compareTo(bytes, offset, length, other.getBytes(), other.getOffset(),
+      other.getLength());
+  }
+
+  @Override
+  public String toString() {
+    return Bytes.toStringBinary(bytes, offset, length);
+  }
+}

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java?rev=1512921&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java Sun Aug 11 13:32:20 2013
@@ -0,0 +1,259 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Extends the basic {@link SimpleByteRange} implementation with position
+ * support. {@code position} is considered transient, not fundamental to the
+ * definition of the range, and does not participate in comparison or copy
+ * operations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class SimplePositionedByteRange extends SimpleByteRange implements PositionedByteRange {
+
+  /**
+   * The current index into the range. Like {@link ByteBuffer} position, it
+   * points to the next value that will be read/written in the array. It
+   * provides the appearance of being 0-indexed, even though its value is
+   * calculated according to offset.
+   * <p>
+   * Position is considered transient and does not participate in
+   * {@link #equals(Object)} or {@link #hashCode()} comparisons.
+   * </p>
+   */
+  private int position = 0;
+
+  /**
+   * Create a new {@code PositionedByteRange} lacking a backing array and with
+   * an undefined viewport.
+   */
+  public SimplePositionedByteRange() {
+    super();
+  }
+
+  /**
+   * Create a new {@code PositionedByteRange} over a new backing array of
+   * size {@code capacity}. The range's offset and length are 0 and
+   * {@code capacity}, respectively.
+   * @param capacity the size of the backing array.
+   */
+  public SimplePositionedByteRange(int capacity) {
+    super(capacity);
+  }
+
+  /**
+   * Create a new {@code PositionedByteRange} over the provided {@code bytes}.
+   * @param bytes The array to wrap.
+   */
+  public SimplePositionedByteRange(byte[] bytes) {
+    super(bytes);
+  }
+
+  /**
+   * Create a new {@code PositionedByteRange} over the provided {@code bytes}.
+   * @param bytes The array to wrap.
+   * @param offset The offset into {@code bytes} considered the beginning
+   *          of this range.
+   * @param length The length of this range.
+   */
+  public SimplePositionedByteRange(byte[] bytes, int offset, int length) {
+    super(bytes, offset, length);
+  }
+
+  @Override
+  public PositionedByteRange unset() {
+    this.position = 0;
+    super.unset();
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange set(int capacity) {
+    this.position = 0;
+    super.set(capacity);
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange set(byte[] bytes) {
+    this.position = 0;
+    super.set(bytes);
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange set(byte[] bytes, int offset, int length) {
+    this.position = 0;
+    super.set(bytes, offset, length);
+    return this;
+  }
+
+  /**
+   * Update the beginning of this range. {@code offset + length} may not be greater than
+   * {@code bytes.length}. Resets {@code position} to 0.
+   * @param offset the new start of this range.
+   * @return this.
+   */
+  @Override
+  public PositionedByteRange setOffset(int offset) {
+    this.position = 0;
+    super.setOffset(offset);
+    return this;
+  }
+
+  /**
+   * Update the length of this range. {@code offset + length} should not be
+   * greater than {@code bytes.length}. If {@code position} is greater than
+   * the new {@code length}, sets {@code position} to {@code length}.
+   * @param length The new length of this range.
+   * @return this.
+   */
+  @Override
+  public PositionedByteRange setLength(int length) {
+    this.position = Math.min(position, length);
+    super.setLength(length);
+    return this;
+  }
+
+  @Override
+  public int getPosition() { return position; }
+
+  @Override
+  public PositionedByteRange setPosition(int position) { this.position = position; return this; }
+
+  @Override
+  public int getRemaining() { return length - position; }
+
+  @Override
+  public byte peek() { return bytes[offset + position]; }
+
+  @Override
+  public byte get() { return get(position++); }
+
+  @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
+  }
+
+  @Override
+  public PositionedByteRange get(byte[] dst, int offset, int length) {
+    if (0 == length) return this;
+    super.get(this.position, dst, offset, length);
+    this.position += length;
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange put(byte val) {
+    put(position++, val);
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange put(byte[] val) {
+    if (0 == val.length) return this;
+    return this.put(val, 0, val.length);
+  }
+
+  @Override
+  public PositionedByteRange put(byte[] val, int offset, int length) {
+    if (0 == length) return this;
+    super.put(position, val, offset, length);
+    this.position += length;
+    return this;
+  }
+
+  /**
+   * Similar to {@link ByteBuffer#flip()}. Sets length to position, position
+   * to offset.
+   */
+  @VisibleForTesting
+  PositionedByteRange flip() {
+    clearHashCache();
+    length = position;
+    position = offset;
+    return this;
+  }
+
+  /**
+   * Similar to {@link ByteBuffer#clear()}. Sets position to 0, length to
+   * capacity.
+   */
+  @VisibleForTesting
+  PositionedByteRange clear() {
+    clearHashCache();
+    position = 0;
+    length = bytes.length - offset;
+    return this;
+  }
+
+  // java boilerplate
+
+  @Override
+  public PositionedByteRange get(int index, byte[] dst) { super.get(index, dst); return this; }
+
+  @Override
+  public PositionedByteRange get(int index, byte[] dst, int offset, int length) {
+    super.get(index, dst, offset, length);
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange put(int index, byte val) { super.put(index, val); return this; }
+
+  @Override
+  public PositionedByteRange put(int index, byte[] val) { super.put(index, val); return this; }
+
+  @Override
+  public PositionedByteRange put(int index, byte[] val, int offset, int length) {
+    super.put(index, val, offset, length);
+    return this;
+  }
+
+  @Override
+  public PositionedByteRange deepCopy() {
+    SimplePositionedByteRange clone = new SimplePositionedByteRange(deepCopyToNewArray());
+    clone.position = this.position;
+    return clone;
+  }
+
+  @Override
+  public PositionedByteRange shallowCopy() {
+    SimplePositionedByteRange clone = new SimplePositionedByteRange(bytes, offset, length);
+    clone.position = this.position;
+    return clone;
+  }
+
+  @Override
+  public PositionedByteRange shallowCopySubRange(int innerOffset, int copyLength) {
+    SimplePositionedByteRange clone =
+        new SimplePositionedByteRange(bytes, offset + innerOffset, copyLength);
+    clone.position = this.position;
+    return clone;
+  }
+}

Added: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java?rev=1512921&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java (added)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java Sun Aug 11 13:32:20 2013
@@ -0,0 +1,68 @@
+/**
+ * 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 org.apache.hadoop.hbase.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestPositionedByteRange {
+  @Test
+  public void testPosition() {
+    PositionedByteRange r = new SimplePositionedByteRange(new byte[5], 1, 3);
+
+    // exercise single-byte put
+    r.put(Bytes.toBytes("f")[0])
+     .put(Bytes.toBytes("o")[0])
+     .put(Bytes.toBytes("o")[0]);
+    Assert.assertEquals(3, r.getPosition());
+    Assert.assertArrayEquals(
+      new byte[] { 0, Bytes.toBytes("f")[0], Bytes.toBytes("o")[0], Bytes.toBytes("o")[0], 0 },
+      r.getBytes());
+
+    // exercise multi-byte put
+    r.setPosition(0);
+    r.put(Bytes.toBytes("f"))
+     .put(Bytes.toBytes("o"))
+     .put(Bytes.toBytes("o"));
+    Assert.assertEquals(3, r.getPosition());
+    Assert.assertArrayEquals(
+      new byte[] { 0, Bytes.toBytes("f")[0], Bytes.toBytes("o")[0], Bytes.toBytes("o")[0], 0 },
+      r.getBytes());
+
+    // exercise single-byte get
+    r.setPosition(0);
+    Assert.assertEquals(Bytes.toBytes("f")[0], r.get());
+    Assert.assertEquals(Bytes.toBytes("o")[0], r.get());
+    Assert.assertEquals(Bytes.toBytes("o")[0], r.get());
+
+    r.setPosition(1);
+    Assert.assertEquals(Bytes.toBytes("o")[0], r.get());
+
+    // exercise multi-byte get
+    r.setPosition(0);
+    byte[] dst = new byte[3];
+    r.get(dst);
+    Assert.assertArrayEquals(Bytes.toBytes("foo"), dst);
+
+    // set position to the end of the range; this should not throw.
+    r.setPosition(3);
+  }
+}

Added: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java?rev=1512921&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java (added)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java Sun Aug 11 13:32:20 2013
@@ -0,0 +1,71 @@
+/**
+ * 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 org.apache.hadoop.hbase.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestSimpleByteRange {
+
+  @Test
+  public void testEmpty(){
+    Assert.assertTrue(SimpleByteRange.isEmpty(null));
+    ByteRange r = new SimpleByteRange();
+    Assert.assertTrue(SimpleByteRange.isEmpty(r));
+    Assert.assertTrue(r.isEmpty());
+    r.set(new byte[0]);
+    Assert.assertEquals(0, r.getBytes().length);
+    Assert.assertEquals(0, r.getOffset());
+    Assert.assertEquals(0, r.getLength());
+    Assert.assertTrue(Bytes.equals(new byte[0], r.deepCopyToNewArray()));
+    Assert.assertEquals(0, r.compareTo(new SimpleByteRange(new byte[0], 0, 0)));
+    Assert.assertEquals(0, r.hashCode());
+  }
+
+  @Test
+  public void testBasics() {
+    ByteRange r = new SimpleByteRange(new byte[] { 1, 3, 2 });
+    Assert.assertFalse(SimpleByteRange.isEmpty(r));
+    Assert.assertNotNull(r.getBytes());//should be empty byte[], but could change this behavior
+    Assert.assertEquals(3, r.getBytes().length);
+    Assert.assertEquals(0, r.getOffset());
+    Assert.assertEquals(3, r.getLength());
+
+    //cloning (deep copying)
+    Assert.assertTrue(Bytes.equals(new byte[]{1, 3, 2}, r.deepCopyToNewArray()));
+    Assert.assertNotSame(r.getBytes(), r.deepCopyToNewArray());
+
+    //hash code
+    Assert.assertTrue(r.hashCode() > 0);
+    Assert.assertEquals(r.hashCode(), r.deepCopy().hashCode());
+
+    //copying to arrays
+    byte[] destination = new byte[]{-59};//junk
+    r.deepCopySubRangeTo(2, 1, destination, 0);
+    Assert.assertTrue(Bytes.equals(new byte[]{2}, destination));
+
+    //set length
+    r.setLength(1);
+    Assert.assertTrue(Bytes.equals(new byte[]{1}, r.deepCopyToNewArray()));
+    r.setLength(2);//verify we retained the 2nd byte, but dangerous in real code
+    Assert.assertTrue(Bytes.equals(new byte[]{1, 3}, r.deepCopyToNewArray()));
+  }
+}

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java Sun Aug 11 13:32:20 2013
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.codec.pr
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
-import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.vint.UFIntTool;
 import org.apache.hadoop.hbase.util.vint.UVIntTool;
@@ -202,7 +202,7 @@ public class RowNodeReader {
 
   public byte[] getToken() {
     // TODO pass in reusable ByteRange
-    return new ByteRange(block, tokenOffset, tokenLength).deepCopyToNewArray();
+    return new SimpleByteRange(block, tokenOffset, tokenLength).deepCopyToNewArray();
   }
 
   public int getOffset() {

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java Sun Aug 11 13:32:20 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.codec.pre
 import org.apache.hadoop.hbase.io.CellOutputStream;
 import org.apache.hadoop.hbase.util.ArrayUtils;
 import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.byterange.ByteRangeSet;
 import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeHashSet;
 import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeTreeSet;
@@ -146,9 +147,9 @@ public class PrefixTreeEncoder implement
   public PrefixTreeEncoder(OutputStream outputStream, boolean includeMvccVersion) {
     // used during cell accumulation
     this.blockMeta = new PrefixTreeBlockMeta();
-    this.rowRange = new ByteRange();
-    this.familyRange = new ByteRange();
-    this.qualifierRange = new ByteRange();
+    this.rowRange = new SimpleByteRange();
+    this.familyRange = new SimpleByteRange();
+    this.qualifierRange = new SimpleByteRange();
     this.timestamps = new long[INITIAL_PER_CELL_ARRAY_SIZES];
     this.mvccVersions = new long[INITIAL_PER_CELL_ARRAY_SIZES];
     this.typeBytes = new byte[INITIAL_PER_CELL_ARRAY_SIZES];
@@ -488,7 +489,7 @@ public class PrefixTreeEncoder implement
   }
 
   public ByteRange getValueByteRange() {
-    return new ByteRange(values, 0, totalValueBytes);
+    return new SimpleByteRange(values, 0, totalValueBytes);
   }
 
 }

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowNodeWriter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowNodeWriter.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowNodeWriter.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowNodeWriter.java Sun Aug 11 13:32:20 2013
@@ -28,7 +28,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
 import org.apache.hadoop.hbase.codec.prefixtree.encode.PrefixTreeEncoder;
 import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.TokenizerNode;
-import org.apache.hadoop.hbase.util.ByteRangeTool;
+import org.apache.hadoop.hbase.util.ByteRangeUtils;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.util.vint.UFIntTool;
 import org.apache.hadoop.hbase.util.vint.UVIntTool;
@@ -155,7 +155,7 @@ public class RowNodeWriter{
   protected void writeRowToken(OutputStream os) throws IOException {
     UVIntTool.writeBytes(tokenWidth, os);
     int tokenStartIndex = tokenizerNode.isRoot() ? 0 : 1;
-    ByteRangeTool.write(os, tokenizerNode.getToken(), tokenStartIndex);
+    ByteRangeUtils.write(os, tokenizerNode.getToken(), tokenStartIndex);
   }
 
   /**

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerNode.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerNode.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerNode.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerNode.java Sun Aug 11 13:32:20 2013
@@ -23,8 +23,10 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.ByteRangeUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.Strings;
 
 import com.google.common.collect.Lists;
@@ -135,7 +137,7 @@ public class TokenizerNode{
 
   public TokenizerNode(Tokenizer builder, TokenizerNode parent, int nodeDepth,
       int tokenStartOffset, int tokenOffset, int tokenLength) {
-    this.token = new ByteRange();
+    this.token = new SimpleByteRange();
     reconstruct(builder, parent, nodeDepth, tokenStartOffset, tokenOffset, tokenLength);
     this.children = Lists.newArrayList();
   }
@@ -164,7 +166,7 @@ public class TokenizerNode{
     parent = null;
     nodeDepth = 0;
     tokenStartOffset = 0;
-    token.clear();
+    token.unset();
     numOccurrences = 0;
     children.clear();// branches & nubs
 
@@ -298,7 +300,7 @@ public class TokenizerNode{
   }
 
   protected int numIdenticalBytes(ByteRange bytes) {
-    return token.numEqualPrefixBytes(bytes, tokenStartOffset);
+    return ByteRangeUtils.numEqualPrefixBytes(token, bytes, tokenStartOffset);
   }
 
 

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/ByteRangeSet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/ByteRangeSet.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/ByteRangeSet.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/ByteRangeSet.java Sun Aug 11 13:32:20 2013
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.util.ArrayUtils;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 
 import com.google.common.collect.Lists;
 
@@ -105,7 +106,7 @@ public abstract class ByteRangeSet {
   protected int store(ByteRange bytes) {
     int indexOfNewElement = numUniqueRanges;
     if (uniqueRanges.size() <= numUniqueRanges) {
-      uniqueRanges.add(new ByteRange());
+      uniqueRanges.add(new SimpleByteRange());
     }
     ByteRange storedRange = uniqueRanges.get(numUniqueRanges);
     int neededBytes = numBytes + bytes.getLength();

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/ByteRangeTreeSet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/ByteRangeTreeSet.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/ByteRangeTreeSet.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/ByteRangeTreeSet.java Sun Aug 11 13:32:20 2013
@@ -36,7 +36,7 @@ public class ByteRangeTreeSet extends By
   /************************ constructors *****************************/
 
   public ByteRangeTreeSet() {
-    this.uniqueIndexByUniqueRange = new TreeMap<ByteRange,Integer>();
+    this.uniqueIndexByUniqueRange = new TreeMap<ByteRange, Integer>();
   }
 
   public ByteRangeTreeSet(List<ByteRange> rawByteArrays) {

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizer.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizer.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizer.java Sun Aug 11 13:32:20 2013
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.Tokenizer;
 import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.TokenizerNode;
 import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.TokenizerRowSearchResult;
-import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -48,7 +48,7 @@ public class TestTokenizer {
     this.inputs = sortedByteArrays.getInputs();
     this.builder = new Tokenizer();
     for (byte[] array : inputs) {
-      builder.addSorted(new ByteRange(array));
+      builder.addSorted(new SimpleByteRange(array));
     }
     this.roundTripped = builder.getArrays();
   }

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTreeDepth.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTreeDepth.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTreeDepth.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTreeDepth.java Sun Aug 11 13:32:20 2013
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.codec.pr
 import java.util.List;
 
 import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.Tokenizer;
-import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -75,7 +75,7 @@ public class TestTreeDepth {
   protected void testInternal(List<String> inputs, int expectedTreeDepth) {
     Tokenizer builder = new Tokenizer();
     for (String s : inputs) {
-      ByteRange b = new ByteRange(Bytes.toBytes(s));
+      SimpleByteRange b = new SimpleByteRange(Bytes.toBytes(s));
       builder.addSorted(b);
     }
     Assert.assertEquals(1, builder.getRoot().getNodeDepth());

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnBuilder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnBuilder.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnBuilder.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnBuilder.java Sun Aug 11 13:32:20 2013
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.codec.pre
 import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.Tokenizer;
 import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.TokenizerNode;
 import org.apache.hadoop.hbase.util.ByteRange;
-import org.apache.hadoop.hbase.util.ByteRangeTool;
+import org.apache.hadoop.hbase.util.ByteRangeUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeTreeSet;
 import org.junit.Assert;
@@ -67,7 +67,7 @@ public class TestColumnBuilder {
     List<ByteRange> inputs = columns.getInputs();
     this.columnSorter = new ByteRangeTreeSet(inputs);
     this.sortedUniqueColumns = columnSorter.compile().getSortedRanges();
-    List<byte[]> copies = ByteRangeTool.copyToNewArrays(sortedUniqueColumns);
+    List<byte[]> copies = ByteRangeUtils.copyToNewArrays(sortedUniqueColumns);
     Assert.assertTrue(Bytes.isSorted(copies));
     this.blockMeta = new PrefixTreeBlockMeta();
     this.blockMeta.setNumMetaBytes(0);

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataRandom.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataRandom.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataRandom.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataRandom.java Sun Aug 11 13:32:20 2013
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.codec.prefixtree.column.TestColumnData;
 import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.byterange.ByteRangeSet;
 import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeTreeSet;
 import org.apache.hadoop.hbase.util.test.RedundantKVGenerator;
@@ -39,7 +40,7 @@ public class TestColumnDataRandom implem
     ByteRangeSet sortedColumns = new ByteRangeTreeSet();
     List<KeyValue> d = generator.generateTestKeyValues(numColumns);
     for (KeyValue col : d) {
-      ByteRange colRange = new ByteRange(col.getQualifier());
+      ByteRange colRange = new SimpleByteRange(col.getQualifier());
       inputs.add(colRange);
       sortedColumns.add(colRange);
     }

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataSimple.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataSimple.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataSimple.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataSimple.java Sun Aug 11 13:32:20 2013
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.codec.prefixtree.column.TestColumnData;
 import org.apache.hadoop.hbase.util.ByteRange;
-import org.apache.hadoop.hbase.util.ByteRangeTool;
+import org.apache.hadoop.hbase.util.ByteRangeUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.collect.Lists;
@@ -37,7 +37,7 @@ public class TestColumnDataSimple implem
     d.add("abc");
     d.add("bbc");
     d.add("abc");
-    return ByteRangeTool.fromArrays(Bytes.getUtf8ByteArrays(d));
+    return ByteRangeUtils.fromArrays(Bytes.getUtf8ByteArrays(d));
   }
 
   @Override
@@ -46,7 +46,7 @@ public class TestColumnDataSimple implem
     d.add("abc");
     d.add("abcde");
     d.add("bbc");
-    return ByteRangeTool.fromArrays(Bytes.getUtf8ByteArrays(d));
+    return ByteRangeUtils.fromArrays(Bytes.getUtf8ByteArrays(d));
   }
 
 }

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java Sun Aug 11 13:32:20 2013
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.codec.pre
 import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeTreeSet;
 import org.junit.Assert;
 
@@ -40,7 +41,7 @@ import com.google.common.collect.Lists;
  */
 public class TestRowDataExerciseFInts extends BaseTestRowData{
 
-	static List<ByteRange> rows;
+  static List<ByteRange> rows;
 	static{
 		List<String> rowStrings = new ArrayList<String>();
         rowStrings.add("com.edsBlog/directoryAa/pageAaa");
@@ -61,7 +62,7 @@ public class TestRowDataExerciseFInts ex
         rowStrings.add("com.isabellasBlog/directoryBb/pageHhh");
         ByteRangeTreeSet ba = new ByteRangeTreeSet();
         for(String row : rowStrings){
-        	ba.add(new ByteRange(Bytes.toBytes(row)));
+        	ba.add(new SimpleByteRange(Bytes.toBytes(row)));
         }
         rows = ba.compile().getSortedRanges();
 	}

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataUrls.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataUrls.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataUrls.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataUrls.java Sun Aug 11 13:32:20 2013
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.codec.pre
 import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeTreeSet;
 
 import com.google.common.collect.Lists;
@@ -38,7 +39,7 @@ import com.google.common.collect.Lists;
  */
 public class TestRowDataUrls extends BaseTestRowData{
 
-	static List<ByteRange> rows;
+  static List<ByteRange> rows;
 	static{
     List<String> rowStrings = new ArrayList<String>();
     rowStrings.add("com.edsBlog/directoryAa/pageAaa");
@@ -59,7 +60,7 @@ public class TestRowDataUrls extends Bas
     rowStrings.add("com.isabellasBlog/directoryBb/pageHhh");
     ByteRangeTreeSet ba = new ByteRangeTreeSet();
     for (String row : rowStrings) {
-      ba.add(new ByteRange(Bytes.toBytes(row)));
+      ba.add(new SimpleByteRange(Bytes.toBytes(row)));
     }
     rows = ba.compile().getSortedRanges();
   }

Modified: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/bytes/TestByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/bytes/TestByteRange.java?rev=1512921&r1=1512920&r2=1512921&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/bytes/TestByteRange.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/bytes/TestByteRange.java Sun Aug 11 13:32:20 2013
@@ -21,13 +21,14 @@ package org.apache.hadoop.hbase.util.byt
 import junit.framework.Assert;
 
 import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.SimpleByteRange;
 import org.junit.Test;
 
 public class TestByteRange {
 
   @Test
   public void testConstructor() {
-    ByteRange b = new ByteRange(new byte[] { 0, 1, 2 });
+    ByteRange b = new SimpleByteRange(new byte[] { 0, 1, 2 });
     Assert.assertEquals(3, b.getLength());
   }