Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0135E11AB5 for ; Thu, 26 Jun 2014 08:25:52 +0000 (UTC) Received: (qmail 61839 invoked by uid 500); 26 Jun 2014 08:25:51 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 61602 invoked by uid 500); 26 Jun 2014 08:25:51 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 61588 invoked by uid 99); 26 Jun 2014 08:25:51 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 26 Jun 2014 08:25:51 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 6C2DA8347BD; Thu, 26 Jun 2014 08:25:51 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ramkrishna@apache.org To: commits@hbase.apache.org Date: Thu, 26 Jun 2014 08:25:52 -0000 Message-Id: In-Reply-To: <5b59e506db024e0f947a00787a52ea46@git.apache.org> References: <5b59e506db024e0f947a00787a52ea46@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/2] git commit: HBASE-10861 Extend ByteRange to create Mutable and Immutable ByteRange (Ram) HBASE-10861 Extend ByteRange to create Mutable and Immutable ByteRange (Ram) Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/7f2f4e80 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7f2f4e80 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7f2f4e80 Branch: refs/heads/master Commit: 7f2f4e80c4d67decbe2a9c2a032b70295fe4376f Parents: cb9a6d1 Author: Ramkrishna Authored: Thu Jun 26 13:54:48 2014 +0530 Committer: Ramkrishna Committed: Thu Jun 26 13:54:48 2014 +0530 ---------------------------------------------------------------------- .../org/apache/hadoop/hbase/KeyValueUtil.java | 6 +- .../hadoop/hbase/types/FixedLengthWrapper.java | 4 +- .../hadoop/hbase/types/OrderedNumeric.java | 4 +- .../hadoop/hbase/types/TerminatedWrapper.java | 4 +- .../hadoop/hbase/util/AbstractByteRange.java | 293 ++++++++++++++++ .../hbase/util/AbstractPositionedByteRange.java | 274 +++++++++++++++ .../org/apache/hadoop/hbase/util/ByteRange.java | 1 + .../hadoop/hbase/util/ByteRangeUtils.java | 2 +- .../apache/hadoop/hbase/util/OrderedBytes.java | 5 +- .../hadoop/hbase/util/PositionedByteRange.java | 16 + .../hbase/util/ReadOnlyByteRangeException.java | 28 ++ .../hadoop/hbase/util/SimpleByteRange.java | 345 ++----------------- .../hbase/util/SimpleMutableByteRange.java | 212 ++++++++++++ .../hbase/util/SimplePositionedByteRange.java | 233 +++---------- .../util/SimplePositionedMutableByteRange.java | 310 +++++++++++++++++ .../hbase/types/TestFixedLengthWrapper.java | 10 +- .../hadoop/hbase/types/TestOrderedBlob.java | 4 +- .../hadoop/hbase/types/TestOrderedBlobVar.java | 4 +- .../hadoop/hbase/types/TestOrderedString.java | 4 +- .../hadoop/hbase/types/TestRawString.java | 4 +- .../apache/hadoop/hbase/types/TestStruct.java | 8 +- .../hbase/types/TestStructNullExtension.java | 8 +- .../hbase/types/TestTerminatedWrapper.java | 14 +- .../apache/hadoop/hbase/types/TestUnion2.java | 6 +- .../util/TestByteRangeWithKVSerialization.java | 5 +- .../hadoop/hbase/util/TestOrderedBytes.java | 63 ++-- .../hbase/util/TestPositionedByteRange.java | 121 ------- .../hadoop/hbase/util/TestSimpleByteRange.java | 112 ------ .../hbase/util/TestSimpleMutableByteRange.java | 112 ++++++ .../TestSimplePositionedMutableByteRange.java | 121 +++++++ .../prefixtree/decode/row/RowNodeReader.java | 4 +- .../prefixtree/encode/PrefixTreeEncoder.java | 12 +- .../encode/tokenize/TokenizerNode.java | 4 +- .../hbase/util/byterange/ByteRangeSet.java | 4 +- .../codec/prefixtree/builder/TestTokenizer.java | 4 +- .../codec/prefixtree/builder/TestTreeDepth.java | 4 +- .../column/data/TestColumnDataRandom.java | 4 +- .../row/data/TestRowDataExerciseFInts.java | 4 +- .../prefixtree/row/data/TestRowDataUrls.java | 4 +- .../hadoop/hbase/util/bytes/TestByteRange.java | 4 +- .../hbase/regionserver/HeapMemStoreLAB.java | 4 +- .../security/access/AccessControlFilter.java | 6 +- .../hbase/security/access/AccessController.java | 9 +- .../visibility/VisibilityController.java | 4 +- .../visibility/VisibilityLabelFilter.java | 6 +- 45 files changed, 1576 insertions(+), 834 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java index 4026b1b..4c1f345 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IterableUtils; -import org.apache.hadoop.hbase.util.SimpleByteRange; +import org.apache.hadoop.hbase.util.SimpleMutableByteRange; import org.apache.hadoop.io.WritableUtils; import com.google.common.base.Function; @@ -197,8 +197,8 @@ public class KeyValueUtil { * Increment the row bytes and clear the other fields */ public static KeyValue createFirstKeyInIncrementedRow(final Cell in){ - byte[] thisRow = new SimpleByteRange(in.getRowArray(), in.getRowOffset(), in.getRowLength()) - .deepCopyToNewArray(); + byte[] thisRow = new SimpleMutableByteRange(in.getRowArray(), in.getRowOffset(), + in.getRowLength()).deepCopyToNewArray(); byte[] nextRow = Bytes.unsignedCopyAndIncrement(thisRow); return createFirstOnRow(nextRow); } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java index 024f9ff..6f94a51 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java @@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; /** * Wraps an existing {@link DataType} implementation as a fixed-length @@ -83,7 +83,7 @@ public class FixedLengthWrapper implements DataType { + src.getPosition() + " max length: " + length); } // create a copy range limited to length bytes. boo. - PositionedByteRange b = new SimplePositionedByteRange(length); + PositionedByteRange b = new SimplePositionedMutableByteRange(length); src.get(b.getBytes()); return base.decode(b); } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java index 42636f1..d2c837c 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java @@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.OrderedBytes; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; /** * An {@link Number} of arbitrary precision and variable-length encoding. The @@ -47,7 +47,7 @@ public class OrderedNumeric extends OrderedBytesBase { @Override public int encodedLength(Number val) { // TODO: this could be done better. - PositionedByteRange buff = new SimplePositionedByteRange(100); + PositionedByteRange buff = new SimplePositionedMutableByteRange(100); return encode(buff, val); } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java index 17bd3fa..85d11fb 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; /** * Wraps an existing {@code DataType} implementation as a terminated @@ -134,7 +134,7 @@ public class TerminatedWrapper implements DataType { byte[] b = new byte[term - src.getPosition()]; src.get(b); // TODO: should we assert that b.position == b.length? - T ret = wrapped.decode(new SimplePositionedByteRange(b)); + T ret = wrapped.decode(new SimplePositionedMutableByteRange(b)); src.get(this.term); return ret; } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractByteRange.java new file mode 100644 index 0000000..c5aebab --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractByteRange.java @@ -0,0 +1,293 @@ +/* + * 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; + +/** + * An abstract implementation of the ByteRange API + */ +public abstract class AbstractByteRange implements ByteRange { + + public 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. + */ + protected int hash = UNSET_HASH_VALUE; + + // + // methods for managing the backing array and range viewport + // + @Override + public byte[] getBytes() { + return bytes; + } + + @Override + public abstract ByteRange unset(); + + @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 short getShort(int index) { + int offset = this.offset + index; + short n = 0; + n ^= bytes[offset] & 0xFF; + n <<= 8; + n ^= bytes[offset + 1] & 0xFF; + return n; + } + + @Override + public int getInt(int index) { + int offset = this.offset + index; + int n = 0; + for (int i = offset; i < (offset + Bytes.SIZEOF_INT); i++) { + n <<= 8; + n ^= bytes[i] & 0xFF; + } + return n; + } + + @Override + public long getLong(int index) { + int offset = this.offset + index; + long l = 0; + for (int i = offset; i < offset + Bytes.SIZEOF_LONG; i++) { + l <<= 8; + l ^= bytes[i] & 0xFF; + } + return l; + } + + // Copied from com.google.protobuf.CodedInputStream + @Override + public long getVLong(int index) { + int shift = 0; + long result = 0; + while (shift < 64) { + final byte b = get(index++); + result |= (long) (b & 0x7F) << shift; + if ((b & 0x80) == 0) { + break; + } + shift += 7; + } + return result; + } + + public static int getVLongSize(long val) { + int rPos = 0; + while ((val & ~0x7F) != 0) { + val >>>= 7; + rPos++; + } + return rPos + 1; + } + + @Override + public abstract ByteRange put(int index, byte val); + + @Override + public abstract ByteRange put(int index, byte[] val); + + @Override + public abstract ByteRange put(int index, byte[] val, int offset, int length); + + @Override + public abstract ByteRange putInt(int index, int val); + + @Override + public abstract ByteRange putLong(int index, long val); + + @Override + public abstract ByteRange putShort(int index, short val); + + @Override + public abstract int putVLong(int index, long val); + + // + // 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 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); + } + + // + // methods used for comparison + // + + @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; + } + + protected 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); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractPositionedByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractPositionedByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractPositionedByteRange.java new file mode 100644 index 0000000..26bcf97 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractPositionedByteRange.java @@ -0,0 +1,274 @@ +/* + * 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 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 + * {@link #compareTo(ByteRange)}, {@link #hashCode()}, or + * {@link #equals(Object)}. {@code Position} is retained by copy operations. + */ +public abstract class AbstractPositionedByteRange extends AbstractByteRange 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. + *

+ * Position is considered transient and does not participate in + * {@link #equals(Object)} or {@link #hashCode()} comparisons. + *

+ */ + protected int position = 0; + + protected int limit = 0; + + @Override + public abstract PositionedByteRange unset(); + + @Override + public PositionedByteRange set(int capacity) { + this.position = 0; + super.set(capacity); + this.limit = capacity; + return this; + } + + @Override + public PositionedByteRange set(byte[] bytes) { + this.position = 0; + super.set(bytes); + this.limit = bytes.length; + return this; + } + + @Override + public PositionedByteRange set(byte[] bytes, int offset, int length) { + this.position = 0; + super.set(bytes, offset, length); + limit = 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 abstract PositionedByteRange put(byte val); + + @Override + public abstract PositionedByteRange put(byte[] val); + + @Override + public abstract PositionedByteRange put(byte[] val, int offset, int length); + + @Override + public abstract PositionedByteRange putInt(int index, int val); + + @Override + public abstract PositionedByteRange putLong(int index, long val); + + @Override + public abstract PositionedByteRange putShort(int index, short val); + + @Override + public abstract PositionedByteRange putInt(int val); + + @Override + public abstract PositionedByteRange putLong(long val); + + @Override + public abstract PositionedByteRange putShort(short val); + + @Override + public abstract int putVLong(int index, long val); + + @Override + public abstract int putVLong(long val); + /** + * 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 short getShort() { + short s = getShort(position); + position += Bytes.SIZEOF_SHORT; + return s; + } + + @Override + public int getInt() { + int i = getInt(position); + position += Bytes.SIZEOF_INT; + return i; + } + + @Override + public long getLong() { + long l = getLong(position); + position += Bytes.SIZEOF_LONG; + return l; + } + + @Override + public long getVLong() { + long p = getVLong(position); + position += getVLongSize(p); + return p; + } + + @Override + public abstract PositionedByteRange put(int index, byte val); + + @Override + public abstract PositionedByteRange put(int index, byte[] val); + + @Override + public abstract PositionedByteRange put(int index, byte[] val, int offset, int length); + + @Override + public abstract PositionedByteRange deepCopy(); + + @Override + public abstract PositionedByteRange shallowCopy(); + + @Override + public abstract PositionedByteRange shallowCopySubRange(int innerOffset, int copyLength); + + @Override + public PositionedByteRange setLimit(int limit) { + this.limit = limit; + return this; + } + + @Override + public int getLimit() { + return this.limit; + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java index 90cfa09..6323a40 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java @@ -303,4 +303,5 @@ public interface ByteRange extends Comparable { * @return new {@code ByteRange} object referencing this range's byte[]. */ public ByteRange shallowCopySubRange(int innerOffset, int copyLength); + } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java index d0c3873..a571471 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java @@ -64,7 +64,7 @@ public class ByteRangeUtils { } ArrayList ranges = Lists.newArrayListWithCapacity(arrays.size()); for (byte[] array : arrays) { - ranges.add(new SimpleByteRange(array)); + ranges.add(new SimpleMutableByteRange(array)); } return ranges; } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java index d8ee65e..bb858c4 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java @@ -1104,7 +1104,8 @@ public class OrderedBytes { ; end++; // increment end to 1-past last byte // create ret buffer using length of encoded data + 1 (header byte) - PositionedByteRange ret = new SimplePositionedByteRange(blobVarDecodedLength(end - start + 1)); + PositionedByteRange ret = new SimplePositionedMutableByteRange(blobVarDecodedLength(end - start + + 1)); int s = 6; byte t = (byte) ((ord.apply(a[offset + start]) << 1) & 0xff); for (int i = start + 1; i < end; i++) { @@ -1730,7 +1731,7 @@ public class OrderedBytes { */ public static int length(PositionedByteRange buff) { PositionedByteRange b = - new SimplePositionedByteRange(buff.getBytes(), buff.getOffset(), buff.getLength()); + new SimplePositionedMutableByteRange(buff.getBytes(), buff.getOffset(), buff.getLength()); b.setPosition(buff.getPosition()); int cnt = 0; for (; isEncodedValue(b); skip(buff), cnt++) http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java index 5aebf31..0d31154 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java @@ -164,6 +164,22 @@ public interface PositionedByteRange extends ByteRange { */ public PositionedByteRange put(byte[] val, int offset, int length); + /** + * Limits the byte range upto a specified value. Limit cannot be greater than + * capacity + * + * @param limit + * @return PositionedByteRange + */ + public PositionedByteRange setLimit(int limit); + + /** + * Return the current limit + * + * @return limit + */ + public int getLimit(); + // override parent interface declarations to return this interface. @Override http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java new file mode 100644 index 0000000..6f508e2 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java @@ -0,0 +1,28 @@ +/* + * 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; + +/** + * Exception thrown when a read only byte range is modified + */ +public class ReadOnlyByteRangeException extends UnsupportedOperationException { + public ReadOnlyByteRangeException() { + + } + +} http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java index 3205854..db3ca0f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java @@ -15,60 +15,15 @@ * 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. + * A read only version of the {@link ByteRange}. */ -@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 class SimpleByteRange extends AbstractByteRange { 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]); } @@ -97,233 +52,50 @@ public class SimpleByteRange implements ByteRange { // @Override - public byte[] getBytes() { - return bytes; - } - - @Override public ByteRange unset() { - clearHashCache(); - this.bytes = null; - this.offset = 0; - this.length = 0; - return this; + throw new ReadOnlyByteRangeException(); } @Override public ByteRange set(int capacity) { - return set(new byte[capacity]); + if (super.bytes != null) { + throw new ReadOnlyByteRangeException(); + } + return super.set(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; + if (super.bytes != null) { + throw new ReadOnlyByteRangeException(); + } + return super.set(bytes); } @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; + if (super.bytes != null) { + throw new ReadOnlyByteRangeException(); + } + return super.set(bytes, offset, length); } // // methods for retrieving data // - - @Override - public byte get(int index) { - return bytes[offset + index]; - } - - @Override - public short getShort(int index) { - int offset = this.offset + index; - short n = 0; - n ^= bytes[offset] & 0xFF; - n <<= 8; - n ^= bytes[offset + 1] & 0xFF; - return n; - } - - @Override - public int getInt(int index) { - int offset = this.offset + index; - int n = 0; - for (int i = offset; i < (offset + Bytes.SIZEOF_INT); i++) { - n <<= 8; - n ^= bytes[i] & 0xFF; - } - return n; - } - - @Override - public long getLong(int index) { - int offset = this.offset + index; - long l = 0; - for (int i = offset; i < offset + Bytes.SIZEOF_LONG; i++) { - l <<= 8; - l ^= bytes[i] & 0xFF; - } - return l; - } - - // Copied from com.google.protobuf.CodedInputStream - @Override - public long getVLong(int index) { - int shift = 0; - long result = 0; - while (shift < 64) { - final byte b = get(index++); - result |= (long) (b & 0x7F) << shift; - if ((b & 0x80) == 0) { - break; - } - shift += 7; - } - return result; - } - - public static int getVLongSize(long val) { - int rPos = 0; - while ((val & ~0x7F) != 0) { - val >>>= 7; - rPos++; - } - return rPos + 1; - } - - @Override - public ByteRange get(int index, byte[] dst) { - if (0 == dst.length) return this; - return get(index, dst, 0, dst.length); - } - - @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; - clearHashCache(); - return this; - } - - @Override - public ByteRange putShort(int index, short val) { - // This writing is same as BB's putShort. When byte[] is wrapped in a BB and call putShort(), - // one can get the same result. - bytes[offset + index + 1] = (byte) val; - val >>= 8; - bytes[offset + index] = (byte) val; - clearHashCache(); - return this; - } - - @Override - public ByteRange putInt(int index, int val) { - // This writing is same as BB's putInt. When byte[] is wrapped in a BB and call getInt(), one - // can get the same result. - for (int i = Bytes.SIZEOF_INT - 1; i > 0; i--) { - bytes[offset + index + i] = (byte) val; - val >>>= 8; - } - bytes[offset + index] = (byte) val; - clearHashCache(); - return this; - } - - @Override - public ByteRange putLong(int index, long val) { - // This writing is same as BB's putLong. When byte[] is wrapped in a BB and call putLong(), one - // can get the same result. - for (int i = Bytes.SIZEOF_LONG - 1; i > 0; i--) { - bytes[offset + index + i] = (byte) val; - val >>>= 8; - } - bytes[offset + index] = (byte) val; - clearHashCache(); - return this; - } - - // Copied from com.google.protobuf.CodedOutputStream - @Override - public int putVLong(int index, long val) { - int rPos = 0; - while (true) { - if ((val & ~0x7F) == 0) { - bytes[offset + index + rPos] = (byte) val; - break; - } else { - bytes[offset + index + rPos] = (byte) ((val & 0x7F) | 0x80); - val >>>= 7; - } - rPos++; - } - clearHashCache(); - return rPos + 1; + throw new ReadOnlyByteRangeException(); } @Override public ByteRange put(int index, byte[] val) { - if (0 == val.length) return this; - return put(index, val, 0, val.length); + throw new ReadOnlyByteRangeException(); } @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); - clearHashCache(); - return this; + throw new ReadOnlyByteRangeException(); } // @@ -331,33 +103,6 @@ public class SimpleByteRange implements ByteRange { // @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()) { @@ -365,20 +110,17 @@ public class SimpleByteRange implements ByteRange { } return clone; } - + @Override public ByteRange shallowCopySubRange(int innerOffset, int copyLength) { - SimpleByteRange clone = new SimpleByteRange(bytes, offset + innerOffset, 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){ @@ -398,42 +140,31 @@ public class SimpleByteRange implements ByteRange { } @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++]; + public ByteRange deepCopy() { + SimpleByteRange clone = new SimpleByteRange(deepCopyToNewArray()); + if (isHashCached()) { + clone.hash = hash; } - return hash; + return clone; } - private boolean isHashCached() { - return hash != UNSET_HASH_VALUE; + @Override + public ByteRange putInt(int index, int val) { + throw new ReadOnlyByteRangeException(); } - protected void clearHashCache() { - hash = UNSET_HASH_VALUE; + @Override + public ByteRange putLong(int index, long val) { + throw new ReadOnlyByteRangeException(); } - /** - * 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()); + public ByteRange putShort(int index, short val) { + throw new ReadOnlyByteRangeException(); } @Override - public String toString() { - return Bytes.toStringBinary(bytes, offset, length); + public int putVLong(int index, long val) { + throw new ReadOnlyByteRangeException(); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java new file mode 100644 index 0000000..65f0147 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java @@ -0,0 +1,212 @@ +/* + * 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 mutable {@link ByteRange} implementation. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class SimpleMutableByteRange extends AbstractByteRange { + + /** + * Create a new {@code ByteRange} lacking a backing array and with an + * undefined viewport. + */ + public SimpleMutableByteRange() { + 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 SimpleMutableByteRange(int capacity) { + this(new byte[capacity]); + } + + /** + * Create a new {@code ByteRange} over the provided {@code bytes}. + * + * @param bytes + * The array to wrap. + */ + public SimpleMutableByteRange(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 SimpleMutableByteRange(byte[] bytes, int offset, int length) { + set(bytes, offset, length); + } + + @Override + public ByteRange unset() { + clearHashCache(); + bytes = null; + offset = 0; + length = 0; + return this; + } + + @Override + public ByteRange put(int index, byte val) { + bytes[offset + index] = val; + clearHashCache(); + 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); + clearHashCache(); + return this; + } + + @Override + public ByteRange putShort(int index, short val) { + // This writing is same as BB's putShort. When byte[] is wrapped in a BB and + // call putShort(), + // one can get the same result. + bytes[offset + index + 1] = (byte) val; + val >>= 8; + bytes[offset + index] = (byte) val; + clearHashCache(); + return this; + } + + @Override + public ByteRange putInt(int index, int val) { + // This writing is same as BB's putInt. When byte[] is wrapped in a BB and + // call getInt(), one + // can get the same result. + for (int i = Bytes.SIZEOF_INT - 1; i > 0; i--) { + bytes[offset + index + i] = (byte) val; + val >>>= 8; + } + bytes[offset + index] = (byte) val; + clearHashCache(); + return this; + } + + @Override + public ByteRange putLong(int index, long val) { + // This writing is same as BB's putLong. When byte[] is wrapped in a BB and + // call putLong(), one + // can get the same result. + for (int i = Bytes.SIZEOF_LONG - 1; i > 0; i--) { + bytes[offset + index + i] = (byte) val; + val >>>= 8; + } + bytes[offset + index] = (byte) val; + clearHashCache(); + return this; + } + + // Copied from com.google.protobuf.CodedOutputStream + @Override + public int putVLong(int index, long val) { + int rPos = 0; + while (true) { + if ((val & ~0x7F) == 0) { + bytes[offset + index + rPos] = (byte) val; + break; + } else { + bytes[offset + index + rPos] = (byte) ((val & 0x7F) | 0x80); + val >>>= 7; + } + rPos++; + } + clearHashCache(); + return rPos + 1; + } + + @Override + public ByteRange deepCopy() { + SimpleMutableByteRange clone = new SimpleMutableByteRange(deepCopyToNewArray()); + if (isHashCached()) { + clone.hash = hash; + } + return clone; + } + + @Override + public ByteRange shallowCopy() { + SimpleMutableByteRange clone = new SimpleMutableByteRange(bytes, offset, length); + if (isHashCached()) { + clone.hash = hash; + } + return clone; + } + + @Override + public ByteRange shallowCopySubRange(int innerOffset, int copyLength) { + SimpleMutableByteRange clone = new SimpleMutableByteRange(bytes, offset + innerOffset, + copyLength); + if (isHashCached()) { + clone.hash = hash; + } + return clone; + } + + @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 SimpleMutableByteRange)) { + return false; + } + SimpleMutableByteRange that = (SimpleMutableByteRange) thatObject; + return Bytes.equals(bytes, offset, length, that.bytes, that.offset, that.length); + } + +} http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java index c436b91..2f7e5c3 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java @@ -18,37 +18,20 @@ package org.apache.hadoop.hbase.util; -import java.io.IOException; -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 - * {@link #compareTo(ByteRange)}, {@link #hashCode()}, or + * Extends the basic {@link SimpleMutableByteRange} implementation with position + * support and it is a readonly version. {@code position} is considered + * transient, not fundamental to the definition of the range, and does not + * participate in {@link #compareTo(ByteRange)}, {@link #hashCode()}, or * {@link #equals(Object)}. {@code Position} is retained by copy operations. */ @InterfaceAudience.Public @InterfaceStability.Evolving @edu.umd.cs.findbugs.annotations.SuppressWarnings("EQ_DOESNT_OVERRIDE_EQUALS") -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. - *

- * Position is considered transient and does not participate in - * {@link #equals(Object)} or {@link #hashCode()} comparisons. - *

- */ - private int position = 0; +public class SimplePositionedByteRange extends AbstractPositionedByteRange { /** * Create a new {@code PositionedByteRange} lacking a backing array and with @@ -65,7 +48,7 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi * @param capacity the size of the backing array. */ public SimplePositionedByteRange(int capacity) { - super(capacity); + this(new byte[capacity]); } /** @@ -73,7 +56,7 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi * @param bytes The array to wrap. */ public SimplePositionedByteRange(byte[] bytes) { - super(bytes); + set(bytes); } /** @@ -84,194 +67,68 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi * @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; + set(bytes, offset, length); } @Override public PositionedByteRange set(int capacity) { - this.position = 0; - super.set(capacity); - return this; + if (super.bytes != null) { + throw new ReadOnlyByteRangeException(); + } + return super.set(capacity); } @Override public PositionedByteRange set(byte[] bytes) { - this.position = 0; - super.set(bytes); - return this; + if (super.bytes != null) { + throw new ReadOnlyByteRangeException(); + } + return super.set(bytes); } @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 short getShort() { - short s = getShort(position); - position += Bytes.SIZEOF_SHORT; - return s; - } - - @Override - public int getInt() { - int i = getInt(position); - position += Bytes.SIZEOF_INT; - return i; - } - - @Override - public long getLong() { - long l = getLong(position); - position += Bytes.SIZEOF_LONG; - return l; - } - - @Override - public long getVLong() { - long p = getVLong(position); - position += getVLongSize(p); - return p; - } - - @Override - public PositionedByteRange get(byte[] dst) { - if (0 == dst.length) return this; - return this.get(dst, 0, dst.length); // be clear we're calling self, not super - } - - @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; + if (super.bytes != null) { + throw new ReadOnlyByteRangeException(); + } + return super.set(bytes, offset, length); } @Override public PositionedByteRange put(byte val) { - put(position++, val); - return this; + throw new ReadOnlyByteRangeException(); } @Override public PositionedByteRange putShort(short val) { - putShort(position, val); - position += Bytes.SIZEOF_SHORT; - return this; + throw new ReadOnlyByteRangeException(); } @Override public PositionedByteRange putInt(int val) { - putInt(position, val); - position += Bytes.SIZEOF_INT; - return this; + throw new ReadOnlyByteRangeException(); } @Override public PositionedByteRange putLong(long val) { - putLong(position, val); - position += Bytes.SIZEOF_LONG; - return this; + throw new ReadOnlyByteRangeException(); } @Override public int putVLong(long val) { - int len = putVLong(position, val); - position += len; - return len; + throw new ReadOnlyByteRangeException(); } @Override public PositionedByteRange put(byte[] val) { - if (0 == val.length) return this; - return this.put(val, 0, val.length); + throw new ReadOnlyByteRangeException(); } @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; + throw new ReadOnlyByteRangeException(); } - // java boilerplate @Override public PositionedByteRange get(int index, byte[] dst) { super.get(index, dst); return this; } @@ -283,33 +140,38 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi } @Override - public PositionedByteRange put(int index, byte val) { super.put(index, val); return this; } + public PositionedByteRange put(int index, byte val) { + throw new ReadOnlyByteRangeException(); + } @Override public PositionedByteRange putShort(int index, short val) { - super.putShort(index, val); - return this; + throw new ReadOnlyByteRangeException(); } @Override public PositionedByteRange putInt(int index, int val) { - super.putInt(index, val); - return this; + throw new ReadOnlyByteRangeException(); + } + + @Override + public int putVLong(int index, long val) { + throw new ReadOnlyByteRangeException(); } @Override public PositionedByteRange putLong(int index, long val) { - super.putLong(index, val); - return this; + throw new ReadOnlyByteRangeException(); } @Override - public PositionedByteRange put(int index, byte[] val) { super.put(index, val); return this; } + public PositionedByteRange put(int index, byte[] val) { + throw new ReadOnlyByteRangeException(); + } @Override public PositionedByteRange put(int index, byte[] val, int offset, int length) { - super.put(index, val, offset, length); - return this; + throw new ReadOnlyByteRangeException(); } @Override @@ -333,4 +195,15 @@ public class SimplePositionedByteRange extends SimpleByteRange implements Positi clone.position = this.position; return clone; } + + @Override + public PositionedByteRange setLimit(int limit) { + throw new ReadOnlyByteRangeException(); + } + + @Override + public PositionedByteRange unset() { + throw new ReadOnlyByteRangeException(); + } + } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java new file mode 100644 index 0000000..cb4ae86 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java @@ -0,0 +1,310 @@ +/* + * 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; + +/** + * Extends the basic {@link AbstractPositionedByteRange} implementation with + * position support and it is a mutable version. {@code position} is considered transient, + * not fundamental to the definition of the range, and does not participate in + * {@link #compareTo(ByteRange)}, {@link #hashCode()}, or + * {@link #equals(Object)}. {@code Position} is retained by copy operations. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +@edu.umd.cs.findbugs.annotations.SuppressWarnings("EQ_DOESNT_OVERRIDE_EQUALS") +public class SimplePositionedMutableByteRange extends AbstractPositionedByteRange { + /** + * Create a new {@code PositionedByteRange} lacking a backing array and with + * an undefined viewport. + */ + public SimplePositionedMutableByteRange() { + 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 SimplePositionedMutableByteRange(int capacity) { + this(new byte[capacity]); + } + + /** + * Create a new {@code PositionedByteRange} over the provided {@code bytes}. + * + * @param bytes + * The array to wrap. + */ + public SimplePositionedMutableByteRange(byte[] bytes) { + set(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 SimplePositionedMutableByteRange(byte[] bytes, int offset, int length) { + set(bytes, offset, length); + } + + @Override + public PositionedByteRange unset() { + this.position = 0; + clearHashCache(); + bytes = null; + offset = 0; + length = 0; + return this; + } + + @Override + public PositionedByteRange set(int capacity) { + this.position = 0; + super.set(capacity); + this.limit = capacity; + return this; + } + + @Override + public PositionedByteRange set(byte[] bytes) { + this.position = 0; + super.set(bytes); + this.limit = bytes.length; + return this; + } + + @Override + public PositionedByteRange set(byte[] bytes, int offset, int length) { + this.position = 0; + super.set(bytes, offset, length); + limit = 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 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; + put(position, val, offset, length); + this.position += length; + return this; + } + + @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) { + bytes[offset + index] = val; + return this; + } + + @Override + public PositionedByteRange put(int index, byte[] val) { + if (0 == val.length) + return this; + return put(index, val, 0, val.length); + } + + @Override + public PositionedByteRange 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; + } + + @Override + public PositionedByteRange deepCopy() { + SimplePositionedMutableByteRange clone = new SimplePositionedMutableByteRange( + deepCopyToNewArray()); + clone.position = this.position; + return clone; + } + + @Override + public PositionedByteRange shallowCopy() { + SimplePositionedMutableByteRange clone = new SimplePositionedMutableByteRange(bytes, offset, + length); + clone.position = this.position; + return clone; + } + + @Override + public PositionedByteRange shallowCopySubRange(int innerOffset, int copyLength) { + SimplePositionedMutableByteRange clone = new SimplePositionedMutableByteRange(bytes, offset + + innerOffset, copyLength); + clone.position = this.position; + return clone; + } + + @Override + public PositionedByteRange putShort(short val) { + putShort(position, val); + position += Bytes.SIZEOF_SHORT; + return this; + } + + @Override + public PositionedByteRange putInt(int val) { + putInt(position, val); + position += Bytes.SIZEOF_INT; + return this; + } + + @Override + public PositionedByteRange putLong(long val) { + putLong(position, val); + position += Bytes.SIZEOF_LONG; + return this; + } + + @Override + public int putVLong(long val) { + int len = putVLong(position, val); + position += len; + return len; + } + + @Override + public PositionedByteRange putShort(int index, short val) { + // This writing is same as BB's putShort. When byte[] is wrapped in a BB and + // call putShort(), + // one can get the same result. + bytes[offset + index + 1] = (byte) val; + val >>= 8; + bytes[offset + index] = (byte) val; + clearHashCache(); + return this; + } + + @Override + public PositionedByteRange putInt(int index, int val) { + // This writing is same as BB's putInt. When byte[] is wrapped in a BB and + // call getInt(), one + // can get the same result. + for (int i = Bytes.SIZEOF_INT - 1; i > 0; i--) { + bytes[offset + index + i] = (byte) val; + val >>>= 8; + } + bytes[offset + index] = (byte) val; + clearHashCache(); + return this; + } + + @Override + public PositionedByteRange putLong(int index, long val) { + // This writing is same as BB's putLong. When byte[] is wrapped in a BB and + // call putLong(), one + // can get the same result. + for (int i = Bytes.SIZEOF_LONG - 1; i > 0; i--) { + bytes[offset + index + i] = (byte) val; + val >>>= 8; + } + bytes[offset + index] = (byte) val; + clearHashCache(); + return this; + } + + // Copied from com.google.protobuf.CodedOutputStream + @Override + public int putVLong(int index, long val) { + int rPos = 0; + while (true) { + if ((val & ~0x7F) == 0) { + bytes[offset + index + rPos] = (byte) val; + break; + } else { + bytes[offset + index + rPos] = (byte) ((val & 0x7F) | 0x80); + val >>>= 7; + } + rPos++; + } + clearHashCache(); + return rPos + 1; + } + +} http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java index ddc7354..d9227c3 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestFixedLengthWrapper.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -46,7 +46,7 @@ public class TestFixedLengthWrapper { @Test public void testReadWrite() { for (int limit : limits) { - PositionedByteRange buff = new SimplePositionedByteRange(limit); + PositionedByteRange buff = new SimplePositionedMutableByteRange(limit); for (Order ord : new Order[] { Order.ASCENDING, Order.DESCENDING }) { for (byte[] val : VALUES) { buff.setPosition(0); @@ -65,21 +65,21 @@ public class TestFixedLengthWrapper { @Test(expected = IllegalArgumentException.class) public void testInsufficientRemainingRead() { - PositionedByteRange buff = new SimplePositionedByteRange(0); + PositionedByteRange buff = new SimplePositionedMutableByteRange(0); DataType type = new FixedLengthWrapper(new RawBytes(), 3); type.decode(buff); } @Test(expected = IllegalArgumentException.class) public void testInsufficientRemainingWrite() { - PositionedByteRange buff = new SimplePositionedByteRange(0); + PositionedByteRange buff = new SimplePositionedMutableByteRange(0); DataType type = new FixedLengthWrapper(new RawBytes(), 3); type.encode(buff, Bytes.toBytes("")); } @Test(expected = IllegalArgumentException.class) public void testOverflowPassthrough() { - PositionedByteRange buff = new SimplePositionedByteRange(3); + PositionedByteRange buff = new SimplePositionedMutableByteRange(3); DataType type = new FixedLengthWrapper(new RawBytes(), 0); type.encode(buff, Bytes.toBytes("foo")); } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java index 412bb2d..d24f4d4 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlob.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,7 +37,7 @@ public class TestOrderedBlob { @Test public void testEncodedLength() { - PositionedByteRange buff = new SimplePositionedByteRange(20); + PositionedByteRange buff = new SimplePositionedMutableByteRange(20); for (DataType type : new OrderedBlob[] { OrderedBlob.ASCENDING, OrderedBlob.DESCENDING }) { for (byte[] val : VALUES) { buff.setPosition(0); http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java index 18c6a66..4e88552 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedBlobVar.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,7 +37,7 @@ public class TestOrderedBlobVar { @Test public void testEncodedLength() { - PositionedByteRange buff = new SimplePositionedByteRange(20); + PositionedByteRange buff = new SimplePositionedMutableByteRange(20); for (DataType type : new OrderedBlobVar[] { OrderedBlobVar.ASCENDING, OrderedBlobVar.DESCENDING }) { for (byte[] val : VALUES) { http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java index 68c6f55..8306f00 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestOrderedString.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,7 +34,7 @@ public class TestOrderedString { @Test public void testEncodedLength() { - PositionedByteRange buff = new SimplePositionedByteRange(20); + PositionedByteRange buff = new SimplePositionedMutableByteRange(20); for (DataType type : new OrderedString[] { OrderedString.ASCENDING, OrderedString.DESCENDING }) { for (String val : VALUES) { buff.setPosition(0); http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java index dc00f2c..b022229 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestRawString.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,7 +41,7 @@ public class TestRawString { RawString type = Order.ASCENDING == ord ? RawString.ASCENDING : RawString.DESCENDING; for (String val : VALUES) { - PositionedByteRange buff = new SimplePositionedByteRange(Bytes.toBytes(val).length); + PositionedByteRange buff = new SimplePositionedMutableByteRange(Bytes.toBytes(val).length); assertEquals(buff.getLength(), type.encode(buff, val)); byte[] expected = Bytes.toBytes(val); ord.apply(expected); http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java index f679035..2b79eeb 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStruct.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -350,8 +350,10 @@ public class TestStruct { Constructor ctor = specialized.encodedClass().getConstructor(Object[].class); for (int i = 0; i < vals.length; i++) { vals[i] = ctor.newInstance(new Object[] { constructorArgs[i] }); - encodedGeneric[i] = new SimplePositionedByteRange(generic.encodedLength(constructorArgs[i])); - encodedSpecialized[i] = new SimplePositionedByteRange(specialized.encodedLength(vals[i])); + encodedGeneric[i] = new SimplePositionedMutableByteRange( + generic.encodedLength(constructorArgs[i])); + encodedSpecialized[i] = new SimplePositionedMutableByteRange( + specialized.encodedLength(vals[i])); } // populate our arrays http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java index 7fa7023..8e45793 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestStructNullExtension.java @@ -26,7 +26,7 @@ import java.util.Arrays; import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,7 +41,7 @@ public class TestStructNullExtension { Struct s = new StructBuilder() .add(new RawStringTerminated("|")) // not nullable .toStruct(); - PositionedByteRange buf = new SimplePositionedByteRange(4); + PositionedByteRange buf = new SimplePositionedMutableByteRange(4); s.encode(buf, new Object[1]); } @@ -61,7 +61,7 @@ public class TestStructNullExtension { .add(OrderedNumeric.ASCENDING) .toStruct(); - PositionedByteRange buf1 = new SimplePositionedByteRange(7); + PositionedByteRange buf1 = new SimplePositionedMutableByteRange(7); Object[] val1 = new Object[] { BigDecimal.ONE, "foo" }; // => 2 bytes + 5 bytes assertEquals("Encoding shorter value wrote a surprising number of bytes.", buf1.getLength(), shorter.encode(buf1, val1)); @@ -93,7 +93,7 @@ public class TestStructNullExtension { Arrays.copyOf(val1, 4), longer.decode(buf1)); // test omission of trailing members - PositionedByteRange buf2 = new SimplePositionedByteRange(7); + PositionedByteRange buf2 = new SimplePositionedMutableByteRange(7); buf1.setPosition(0); assertEquals( "Encoding a short value with extended struct should have same result as using short struct.", http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java index 6cccb03..6bb331a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestTerminatedWrapper.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -58,13 +58,13 @@ public class TestTerminatedWrapper { @Test(expected = IllegalArgumentException.class) public void testEncodedValueContainsTerm() { DataType type = new TerminatedWrapper(new RawBytes(), "foo"); - PositionedByteRange buff = new SimplePositionedByteRange(16); + PositionedByteRange buff = new SimplePositionedMutableByteRange(16); type.encode(buff, Bytes.toBytes("hello foobar!")); } @Test public void testReadWriteSkippable() { - PositionedByteRange buff = new SimplePositionedByteRange(14); + PositionedByteRange buff = new SimplePositionedMutableByteRange(14); for (OrderedString t : new OrderedString[] { OrderedString.ASCENDING, OrderedString.DESCENDING }) { @@ -83,7 +83,7 @@ public class TestTerminatedWrapper { @Test public void testReadWriteNonSkippable() { - PositionedByteRange buff = new SimplePositionedByteRange(12); + PositionedByteRange buff = new SimplePositionedMutableByteRange(12); for (Order ord : new Order[] { Order.ASCENDING, Order.DESCENDING }) { for (byte[] term : TERMINATORS) { for (byte[] val : VALUES_BYTES) { @@ -100,7 +100,7 @@ public class TestTerminatedWrapper { @Test public void testSkipSkippable() { - PositionedByteRange buff = new SimplePositionedByteRange(14); + PositionedByteRange buff = new SimplePositionedMutableByteRange(14); for (OrderedString t : new OrderedString[] { OrderedString.ASCENDING, OrderedString.DESCENDING }) { @@ -120,7 +120,7 @@ public class TestTerminatedWrapper { @Test public void testSkipNonSkippable() { - PositionedByteRange buff = new SimplePositionedByteRange(12); + PositionedByteRange buff = new SimplePositionedMutableByteRange(12); for (Order ord : new Order[] { Order.ASCENDING, Order.DESCENDING }) { for (byte[] term : TERMINATORS) { for (byte[] val : VALUES_BYTES) { @@ -137,7 +137,7 @@ public class TestTerminatedWrapper { @Test(expected = IllegalArgumentException.class) public void testInvalidSkip() { - PositionedByteRange buff = new SimplePositionedByteRange(Bytes.toBytes("foo")); + PositionedByteRange buff = new SimplePositionedMutableByteRange(Bytes.toBytes("foo")); DataType type = new TerminatedWrapper(new RawBytes(), new byte[] { 0x00 }); type.skip(buff); } http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java index 6396193..ff1699c 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/types/TestUnion2.java @@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.util.Order; import org.apache.hadoop.hbase.util.PositionedByteRange; -import org.apache.hadoop.hbase.util.SimplePositionedByteRange; +import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -109,7 +109,7 @@ public class TestUnion2 { public void testEncodeDecode() { Integer intVal = Integer.valueOf(10); String strVal = "hello"; - PositionedByteRange buff = new SimplePositionedByteRange(10); + PositionedByteRange buff = new SimplePositionedMutableByteRange(10); SampleUnion1 type = new SampleUnion1(); type.encode(buff, intVal); @@ -125,7 +125,7 @@ public class TestUnion2 { public void testSkip() { Integer intVal = Integer.valueOf(10); String strVal = "hello"; - PositionedByteRange buff = new SimplePositionedByteRange(10); + PositionedByteRange buff = new SimplePositionedMutableByteRange(10); SampleUnion1 type = new SampleUnion1(); int len = type.encode(buff, intVal); http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2f4e80/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java index 9319fde..e2af966 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java @@ -69,12 +69,13 @@ public class TestByteRangeWithKVSerialization { kvs.add(kv); totalSize += kv.getLength() + Bytes.SIZEOF_LONG; } - PositionedByteRange pbr = new SimplePositionedByteRange(totalSize); + PositionedByteRange pbr = new SimplePositionedMutableByteRange(totalSize); for (KeyValue kv : kvs) { writeCell(pbr, kv); } - PositionedByteRange pbr1 = new SimplePositionedByteRange(pbr.getBytes(), 0, pbr.getPosition()); + PositionedByteRange pbr1 = new SimplePositionedMutableByteRange(pbr.getBytes(), 0, + pbr.getPosition()); for (int i = 0; i < kvCount; i++) { KeyValue kv = readCell(pbr1); KeyValue kv1 = kvs.get(i);