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:29:37 UTC

svn commit: r1512920 - in /hbase/branches/0.95/hbase-common/src: main/java/org/apache/hadoop/hbase/util/ test/java/org/apache/hadoop/hbase/util/

Author: nkeywal
Date: Sun Aug 11 13:29:37 2013
New Revision: 1512920

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

Added:
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
    hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
    hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
Removed:
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeTool.java
    hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRange.java

Added: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java?rev=1512920&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java Sun Aug 11 13:29:37 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/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java?rev=1512920&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java Sun Aug 11 13:29:37 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/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java?rev=1512920&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java Sun Aug 11 13:29:37 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/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java?rev=1512920&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java (added)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java Sun Aug 11 13:29:37 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/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java?rev=1512920&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestPositionedByteRange.java Sun Aug 11 13:29:37 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/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java?rev=1512920&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java (added)
+++ hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleByteRange.java Sun Aug 11 13:29:37 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()));
+  }
+}