You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/09/25 13:37:54 UTC
hbase git commit: HBASE-16692 Make ByteBufferUtils#equals safer and
correct (binlijin)
Repository: hbase
Updated Branches:
refs/heads/master 21969f515 -> 3896d9ed0
HBASE-16692 Make ByteBufferUtils#equals safer and correct (binlijin)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3896d9ed
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3896d9ed
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3896d9ed
Branch: refs/heads/master
Commit: 3896d9ed0a87c77330f3f2c998a6fdafe272e2d6
Parents: 21969f5
Author: tedyu <yu...@gmail.com>
Authored: Sun Sep 25 06:37:40 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sun Sep 25 06:37:40 2016 -0700
----------------------------------------------------------------------
.../hadoop/hbase/util/ByteBufferUtils.java | 9 +
.../hadoop/hbase/util/TestByteBufferUtils.java | 441 +++++++++++++++++++
.../hadoop/hbase/util/TestByteBufferUtils.java | 412 -----------------
3 files changed, 450 insertions(+), 412 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/3896d9ed/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index d788c70..c491fe1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -17,6 +17,7 @@
package org.apache.hadoop.hbase.util;
import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.InputStream;
@@ -573,6 +574,10 @@ public final class ByteBufferUtils {
}
public static boolean equals(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
+ if ((l1 == 0) || (l2 == 0)) {
+ // both 0 length, return true, or else false
+ return l1 == l2;
+ }
// Since we're often comparing adjacent sorted data,
// it's usual to have equal arrays except for the very last byte
// so check that first
@@ -627,6 +632,10 @@ public final class ByteBufferUtils {
}
public static boolean equals(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
+ if ((l1 == 0) || (l2 == 0)) {
+ // both 0 length, return true, or else false
+ return l1 == l2;
+ }
// Since we're often comparing adjacent sorted data,
// it's usual to have equal arrays except for the very last byte
// so check that first
http://git-wip-us.apache.org/repos/asf/hbase/blob/3896d9ed/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
new file mode 100644
index 0000000..dfbc015
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
@@ -0,0 +1,441 @@
+/*
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.io.WritableUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestByteBufferUtils {
+
+ private byte[] array;
+
+ /**
+ * Create an array with sample data.
+ */
+ @Before
+ public void setUp() {
+ array = new byte[8];
+ for (int i = 0; i < array.length; ++i) {
+ array[i] = (byte) ('a' + i);
+ }
+ }
+
+ private static final int MAX_VLONG_LENGTH = 9;
+ private static final Collection<Long> testNumbers;
+
+ private static void addNumber(Set<Long> a, long l) {
+ if (l != Long.MIN_VALUE) {
+ a.add(l - 1);
+ }
+ a.add(l);
+ if (l != Long.MAX_VALUE) {
+ a.add(l + 1);
+ }
+ for (long divisor = 3; divisor <= 10; ++divisor) {
+ for (long delta = -1; delta <= 1; ++delta) {
+ a.add(l / divisor + delta);
+ }
+ }
+ }
+
+ static {
+ SortedSet<Long> a = new TreeSet<Long>();
+ for (int i = 0; i <= 63; ++i) {
+ long v = (-1L) << i;
+ assertTrue(v < 0);
+ addNumber(a, v);
+ v = (1L << i) - 1;
+ assertTrue(v >= 0);
+ addNumber(a, v);
+ }
+
+ testNumbers = Collections.unmodifiableSet(a);
+ System.err.println("Testing variable-length long serialization using: "
+ + testNumbers + " (count: " + testNumbers.size() + ")");
+ assertEquals(1753, testNumbers.size());
+ assertEquals(Long.MIN_VALUE, a.first().longValue());
+ assertEquals(Long.MAX_VALUE, a.last().longValue());
+ }
+
+ @Test
+ public void testReadWriteVLong() {
+ for (long l : testNumbers) {
+ ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH);
+ ByteBufferUtils.writeVLong(b, l);
+ b.flip();
+ assertEquals(l, ByteBufferUtils.readVLong(b));
+ }
+ }
+
+ @Test
+ public void testConsistencyWithHadoopVLong() throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(baos);
+ for (long l : testNumbers) {
+ baos.reset();
+ ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH);
+ ByteBufferUtils.writeVLong(b, l);
+ String bufStr = Bytes.toStringBinary(b.array(),
+ b.arrayOffset(), b.position());
+ WritableUtils.writeVLong(dos, l);
+ String baosStr = Bytes.toStringBinary(baos.toByteArray());
+ assertEquals(baosStr, bufStr);
+ }
+ }
+
+ /**
+ * Test copying to stream from buffer.
+ */
+ @Test
+ public void testMoveBufferToStream() {
+ final int arrayOffset = 7;
+ final int initialPosition = 10;
+ final int endPadding = 5;
+ byte[] arrayWrapper =
+ new byte[arrayOffset + initialPosition + array.length + endPadding];
+ System.arraycopy(array, 0, arrayWrapper,
+ arrayOffset + initialPosition, array.length);
+ ByteBuffer buffer = ByteBuffer.wrap(arrayWrapper, arrayOffset,
+ initialPosition + array.length).slice();
+ assertEquals(initialPosition + array.length, buffer.limit());
+ assertEquals(0, buffer.position());
+ buffer.position(initialPosition);
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ try {
+ ByteBufferUtils.moveBufferToStream(bos, buffer, array.length);
+ } catch (IOException e) {
+ fail("IOException in testCopyToStream()");
+ }
+ assertArrayEquals(array, bos.toByteArray());
+ assertEquals(initialPosition + array.length, buffer.position());
+ }
+
+ /**
+ * Test copying to stream from buffer with offset.
+ * @throws IOException On test failure.
+ */
+ @Test
+ public void testCopyToStreamWithOffset() throws IOException {
+ ByteBuffer buffer = ByteBuffer.wrap(array);
+
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+
+ ByteBufferUtils.copyBufferToStream(bos, buffer, array.length / 2,
+ array.length / 2);
+
+ byte[] returnedArray = bos.toByteArray();
+ for (int i = 0; i < array.length / 2; ++i) {
+ int pos = array.length / 2 + i;
+ assertEquals(returnedArray[i], array[pos]);
+ }
+ }
+
+ /**
+ * Test copying data from stream.
+ * @throws IOException On test failure.
+ */
+ @Test
+ public void testCopyFromStream() throws IOException {
+ ByteBuffer buffer = ByteBuffer.allocate(array.length);
+ ByteArrayInputStream bis = new ByteArrayInputStream(array);
+ DataInputStream dis = new DataInputStream(bis);
+
+ ByteBufferUtils.copyFromStreamToBuffer(buffer, dis, array.length / 2);
+ ByteBufferUtils.copyFromStreamToBuffer(buffer, dis,
+ array.length - array.length / 2);
+ for (int i = 0; i < array.length; ++i) {
+ assertEquals(array[i], buffer.get(i));
+ }
+ }
+
+ /**
+ * Test copying from buffer.
+ */
+ @Test
+ public void testCopyFromBuffer() {
+ ByteBuffer srcBuffer = ByteBuffer.allocate(array.length);
+ ByteBuffer dstBuffer = ByteBuffer.allocate(array.length);
+ srcBuffer.put(array);
+
+ ByteBufferUtils.copyFromBufferToBuffer(srcBuffer, dstBuffer,
+ array.length / 2, array.length / 4);
+ for (int i = 0; i < array.length / 4; ++i) {
+ assertEquals(srcBuffer.get(i + array.length / 2),
+ dstBuffer.get(i));
+ }
+ }
+
+ /**
+ * Test 7-bit encoding of integers.
+ * @throws IOException On test failure.
+ */
+ @Test
+ public void testCompressedInt() throws IOException {
+ testCompressedInt(0);
+ testCompressedInt(Integer.MAX_VALUE);
+ testCompressedInt(Integer.MIN_VALUE);
+
+ for (int i = 0; i < 3; i++) {
+ testCompressedInt((128 << i) - 1);
+ }
+
+ for (int i = 0; i < 3; i++) {
+ testCompressedInt((128 << i));
+ }
+ }
+
+ /**
+ * Test how much bytes we need to store integer.
+ */
+ @Test
+ public void testIntFitsIn() {
+ assertEquals(1, ByteBufferUtils.intFitsIn(0));
+ assertEquals(1, ByteBufferUtils.intFitsIn(1));
+ assertEquals(2, ByteBufferUtils.intFitsIn(1 << 8));
+ assertEquals(3, ByteBufferUtils.intFitsIn(1 << 16));
+ assertEquals(4, ByteBufferUtils.intFitsIn(-1));
+ assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MAX_VALUE));
+ assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MIN_VALUE));
+ }
+
+ /**
+ * Test how much bytes we need to store long.
+ */
+ @Test
+ public void testLongFitsIn() {
+ assertEquals(1, ByteBufferUtils.longFitsIn(0));
+ assertEquals(1, ByteBufferUtils.longFitsIn(1));
+ assertEquals(3, ByteBufferUtils.longFitsIn(1l << 16));
+ assertEquals(5, ByteBufferUtils.longFitsIn(1l << 32));
+ assertEquals(8, ByteBufferUtils.longFitsIn(-1));
+ assertEquals(8, ByteBufferUtils.longFitsIn(Long.MIN_VALUE));
+ assertEquals(8, ByteBufferUtils.longFitsIn(Long.MAX_VALUE));
+ }
+
+ /**
+ * Test if we are comparing equal bytes.
+ */
+ @Test
+ public void testArePartEqual() {
+ byte[] array = new byte[] { 1, 2, 3, 4, 5, 1, 2, 3, 4 };
+ ByteBuffer buffer = ByteBuffer.wrap(array);
+ assertTrue(ByteBufferUtils.arePartsEqual(buffer, 0, 4, 5, 4));
+ assertTrue(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 2));
+ assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 3));
+ assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 3, 6, 2));
+ assertFalse(ByteBufferUtils.arePartsEqual(buffer, 0, 3, 6, 3));
+ }
+
+ /**
+ * Test serializing int to bytes
+ */
+ @Test
+ public void testPutInt() {
+ testPutInt(0);
+ testPutInt(Integer.MAX_VALUE);
+
+ for (int i = 0; i < 3; i++) {
+ testPutInt((128 << i) - 1);
+ }
+
+ for (int i = 0; i < 3; i++) {
+ testPutInt((128 << i));
+ }
+ }
+
+ // Utility methods invoked from test methods
+
+ private void testCompressedInt(int value) throws IOException {
+ int parsedValue = 0;
+
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ ByteBufferUtils.putCompressedInt(bos, value);
+
+ ByteArrayInputStream bis = new ByteArrayInputStream(
+ bos.toByteArray());
+ parsedValue = ByteBufferUtils.readCompressedInt(bis);
+
+ assertEquals(value, parsedValue);
+ }
+
+ private void testPutInt(int value) {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ try {
+ ByteBufferUtils.putInt(baos, value);
+ } catch (IOException e) {
+ throw new RuntimeException("Bug in putIn()", e);
+ }
+
+ ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+ DataInputStream dis = new DataInputStream(bais);
+ try {
+ assertEquals(dis.readInt(), value);
+ } catch (IOException e) {
+ throw new RuntimeException("Bug in test!", e);
+ }
+ }
+
+ @Test
+ public void testToBytes(){
+ ByteBuffer buffer = ByteBuffer.allocate(5);
+ buffer.put(new byte[]{0,1,2,3,4});
+ assertEquals(5, buffer.position());
+ assertEquals(5, buffer.limit());
+ byte[] copy = ByteBufferUtils.toBytes(buffer, 2);
+ assertArrayEquals(new byte[]{2,3,4}, copy);
+ assertEquals(5, buffer.position());
+ assertEquals(5, buffer.limit());
+ }
+
+ @Test
+ public void testToPrimitiveTypes() {
+ ByteBuffer buffer = ByteBuffer.allocate(15);
+ long l = 988L;
+ int i = 135;
+ short s = 7;
+ buffer.putLong(l);
+ buffer.putShort(s);
+ buffer.putInt(i);
+ assertEquals(l, ByteBufferUtils.toLong(buffer, 0));
+ assertEquals(s, ByteBufferUtils.toShort(buffer, 8));
+ assertEquals(i, ByteBufferUtils.toInt(buffer, 10));
+ }
+
+ @Test
+ public void testCopyFromArrayToBuffer() {
+ byte[] b = new byte[15];
+ b[0] = -1;
+ long l = 988L;
+ int i = 135;
+ short s = 7;
+ Bytes.putLong(b, 1, l);
+ Bytes.putShort(b, 9, s);
+ Bytes.putInt(b, 11, i);
+ ByteBuffer buffer = ByteBuffer.allocate(14);
+ ByteBufferUtils.copyFromArrayToBuffer(buffer, b, 1, 14);
+ buffer.rewind();
+ assertEquals(l, buffer.getLong());
+ assertEquals(s, buffer.getShort());
+ assertEquals(i, buffer.getInt());
+ }
+
+ @Test
+ public void testCopyFromBufferToArray() {
+ ByteBuffer buffer = ByteBuffer.allocate(15);
+ buffer.put((byte) -1);
+ long l = 988L;
+ int i = 135;
+ short s = 7;
+ buffer.putShort(s);
+ buffer.putInt(i);
+ buffer.putLong(l);
+ byte[] b = new byte[15];
+ ByteBufferUtils.copyFromBufferToArray(b, buffer, 1, 1, 14);
+ assertEquals(s, Bytes.toShort(b, 1));
+ assertEquals(i, Bytes.toInt(b, 3));
+ assertEquals(l, Bytes.toLong(b, 7));
+ }
+
+ @Test
+ public void testCompareTo() {
+ ByteBuffer bb1 = ByteBuffer.allocate(135);
+ ByteBuffer bb2 = ByteBuffer.allocate(135);
+ byte[] b = new byte[71];
+ fillBB(bb1, (byte) 5);
+ fillBB(bb2, (byte) 5);
+ fillArray(b, (byte) 5);
+ assertEquals(0, ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()));
+ assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), b, 0, b.length) > 0);
+ bb2.put(134, (byte) 6);
+ assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) < 0);
+ bb2.put(6, (byte) 4);
+ assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) > 0);
+ // Assert reverse comparing BB and bytearray works.
+ ByteBuffer bb3 = ByteBuffer.allocate(135);
+ fillBB(bb3, (byte)0);
+ byte[] b3 = new byte[135];
+ fillArray(b3, (byte)1);
+ int result = ByteBufferUtils.compareTo(b3, 0, b3.length, bb3, 0, bb3.remaining());
+ assertTrue(result > 0);
+ result = ByteBufferUtils.compareTo(bb3, 0, bb3.remaining(), b3, 0, b3.length);
+ assertTrue(result < 0);
+ }
+
+ @Test
+ public void testEquals() {
+ byte[] a = Bytes.toBytes("http://A");
+ ByteBuffer bb = ByteBuffer.wrap(a);
+
+ assertTrue(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0,
+ HConstants.EMPTY_BYTE_BUFFER, 0, 0));
+
+ assertFalse(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, bb,
+ 0, a.length));
+
+ assertFalse(ByteBufferUtils.equals(bb, 0, 0, HConstants.EMPTY_BYTE_BUFFER,
+ 0, a.length));
+
+ assertTrue(ByteBufferUtils.equals(bb, 0, a.length, bb, 0, a.length));
+
+ assertTrue(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0,
+ HConstants.EMPTY_BYTE_ARRAY, 0, 0));
+
+ assertFalse(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, a,
+ 0, a.length));
+
+ assertFalse(ByteBufferUtils.equals(bb, 0, a.length,
+ HConstants.EMPTY_BYTE_ARRAY, 0, 0));
+
+ assertTrue(ByteBufferUtils.equals(bb, 0, a.length, a, 0, a.length));
+ }
+
+ private static void fillBB(ByteBuffer bb, byte b) {
+ for (int i = bb.position(); i < bb.limit(); i++) {
+ bb.put(i, b);
+ }
+ }
+
+ private static void fillArray(byte[] bb, byte b) {
+ for (int i = 0; i < bb.length; i++) {
+ bb[i] = b;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/3896d9ed/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
deleted file mode 100644
index 8ef07d2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
+++ /dev/null
@@ -1,412 +0,0 @@
-/*
- * 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 static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.io.WritableUtils;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestByteBufferUtils {
-
- private byte[] array;
-
- /**
- * Create an array with sample data.
- */
- @Before
- public void setUp() {
- array = new byte[8];
- for (int i = 0; i < array.length; ++i) {
- array[i] = (byte) ('a' + i);
- }
- }
-
- private static final int MAX_VLONG_LENGTH = 9;
- private static final Collection<Long> testNumbers;
-
- private static void addNumber(Set<Long> a, long l) {
- if (l != Long.MIN_VALUE) {
- a.add(l - 1);
- }
- a.add(l);
- if (l != Long.MAX_VALUE) {
- a.add(l + 1);
- }
- for (long divisor = 3; divisor <= 10; ++divisor) {
- for (long delta = -1; delta <= 1; ++delta) {
- a.add(l / divisor + delta);
- }
- }
- }
-
- static {
- SortedSet<Long> a = new TreeSet<Long>();
- for (int i = 0; i <= 63; ++i) {
- long v = (-1L) << i;
- assertTrue(v < 0);
- addNumber(a, v);
- v = (1L << i) - 1;
- assertTrue(v >= 0);
- addNumber(a, v);
- }
-
- testNumbers = Collections.unmodifiableSet(a);
- System.err.println("Testing variable-length long serialization using: "
- + testNumbers + " (count: " + testNumbers.size() + ")");
- assertEquals(1753, testNumbers.size());
- assertEquals(Long.MIN_VALUE, a.first().longValue());
- assertEquals(Long.MAX_VALUE, a.last().longValue());
- }
-
- @Test
- public void testReadWriteVLong() {
- for (long l : testNumbers) {
- ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH);
- ByteBufferUtils.writeVLong(b, l);
- b.flip();
- assertEquals(l, ByteBufferUtils.readVLong(b));
- }
- }
-
- @Test
- public void testConsistencyWithHadoopVLong() throws IOException {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- DataOutputStream dos = new DataOutputStream(baos);
- for (long l : testNumbers) {
- baos.reset();
- ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH);
- ByteBufferUtils.writeVLong(b, l);
- String bufStr = Bytes.toStringBinary(b.array(),
- b.arrayOffset(), b.position());
- WritableUtils.writeVLong(dos, l);
- String baosStr = Bytes.toStringBinary(baos.toByteArray());
- assertEquals(baosStr, bufStr);
- }
- }
-
- /**
- * Test copying to stream from buffer.
- */
- @Test
- public void testMoveBufferToStream() {
- final int arrayOffset = 7;
- final int initialPosition = 10;
- final int endPadding = 5;
- byte[] arrayWrapper =
- new byte[arrayOffset + initialPosition + array.length + endPadding];
- System.arraycopy(array, 0, arrayWrapper,
- arrayOffset + initialPosition, array.length);
- ByteBuffer buffer = ByteBuffer.wrap(arrayWrapper, arrayOffset,
- initialPosition + array.length).slice();
- assertEquals(initialPosition + array.length, buffer.limit());
- assertEquals(0, buffer.position());
- buffer.position(initialPosition);
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- try {
- ByteBufferUtils.moveBufferToStream(bos, buffer, array.length);
- } catch (IOException e) {
- fail("IOException in testCopyToStream()");
- }
- assertArrayEquals(array, bos.toByteArray());
- assertEquals(initialPosition + array.length, buffer.position());
- }
-
- /**
- * Test copying to stream from buffer with offset.
- * @throws IOException On test failure.
- */
- @Test
- public void testCopyToStreamWithOffset() throws IOException {
- ByteBuffer buffer = ByteBuffer.wrap(array);
-
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
-
- ByteBufferUtils.copyBufferToStream(bos, buffer, array.length / 2,
- array.length / 2);
-
- byte[] returnedArray = bos.toByteArray();
- for (int i = 0; i < array.length / 2; ++i) {
- int pos = array.length / 2 + i;
- assertEquals(returnedArray[i], array[pos]);
- }
- }
-
- /**
- * Test copying data from stream.
- * @throws IOException On test failure.
- */
- @Test
- public void testCopyFromStream() throws IOException {
- ByteBuffer buffer = ByteBuffer.allocate(array.length);
- ByteArrayInputStream bis = new ByteArrayInputStream(array);
- DataInputStream dis = new DataInputStream(bis);
-
- ByteBufferUtils.copyFromStreamToBuffer(buffer, dis, array.length / 2);
- ByteBufferUtils.copyFromStreamToBuffer(buffer, dis,
- array.length - array.length / 2);
- for (int i = 0; i < array.length; ++i) {
- assertEquals(array[i], buffer.get(i));
- }
- }
-
- /**
- * Test copying from buffer.
- */
- @Test
- public void testCopyFromBuffer() {
- ByteBuffer srcBuffer = ByteBuffer.allocate(array.length);
- ByteBuffer dstBuffer = ByteBuffer.allocate(array.length);
- srcBuffer.put(array);
-
- ByteBufferUtils.copyFromBufferToBuffer(srcBuffer, dstBuffer,
- array.length / 2, array.length / 4);
- for (int i = 0; i < array.length / 4; ++i) {
- assertEquals(srcBuffer.get(i + array.length / 2),
- dstBuffer.get(i));
- }
- }
-
- /**
- * Test 7-bit encoding of integers.
- * @throws IOException On test failure.
- */
- @Test
- public void testCompressedInt() throws IOException {
- testCompressedInt(0);
- testCompressedInt(Integer.MAX_VALUE);
- testCompressedInt(Integer.MIN_VALUE);
-
- for (int i = 0; i < 3; i++) {
- testCompressedInt((128 << i) - 1);
- }
-
- for (int i = 0; i < 3; i++) {
- testCompressedInt((128 << i));
- }
- }
-
- /**
- * Test how much bytes we need to store integer.
- */
- @Test
- public void testIntFitsIn() {
- assertEquals(1, ByteBufferUtils.intFitsIn(0));
- assertEquals(1, ByteBufferUtils.intFitsIn(1));
- assertEquals(2, ByteBufferUtils.intFitsIn(1 << 8));
- assertEquals(3, ByteBufferUtils.intFitsIn(1 << 16));
- assertEquals(4, ByteBufferUtils.intFitsIn(-1));
- assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MAX_VALUE));
- assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MIN_VALUE));
- }
-
- /**
- * Test how much bytes we need to store long.
- */
- @Test
- public void testLongFitsIn() {
- assertEquals(1, ByteBufferUtils.longFitsIn(0));
- assertEquals(1, ByteBufferUtils.longFitsIn(1));
- assertEquals(3, ByteBufferUtils.longFitsIn(1l << 16));
- assertEquals(5, ByteBufferUtils.longFitsIn(1l << 32));
- assertEquals(8, ByteBufferUtils.longFitsIn(-1));
- assertEquals(8, ByteBufferUtils.longFitsIn(Long.MIN_VALUE));
- assertEquals(8, ByteBufferUtils.longFitsIn(Long.MAX_VALUE));
- }
-
- /**
- * Test if we are comparing equal bytes.
- */
- @Test
- public void testArePartEqual() {
- byte[] array = new byte[] { 1, 2, 3, 4, 5, 1, 2, 3, 4 };
- ByteBuffer buffer = ByteBuffer.wrap(array);
- assertTrue(ByteBufferUtils.arePartsEqual(buffer, 0, 4, 5, 4));
- assertTrue(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 2));
- assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 3));
- assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 3, 6, 2));
- assertFalse(ByteBufferUtils.arePartsEqual(buffer, 0, 3, 6, 3));
- }
-
- /**
- * Test serializing int to bytes
- */
- @Test
- public void testPutInt() {
- testPutInt(0);
- testPutInt(Integer.MAX_VALUE);
-
- for (int i = 0; i < 3; i++) {
- testPutInt((128 << i) - 1);
- }
-
- for (int i = 0; i < 3; i++) {
- testPutInt((128 << i));
- }
- }
-
- // Utility methods invoked from test methods
-
- private void testCompressedInt(int value) throws IOException {
- int parsedValue = 0;
-
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- ByteBufferUtils.putCompressedInt(bos, value);
-
- ByteArrayInputStream bis = new ByteArrayInputStream(
- bos.toByteArray());
- parsedValue = ByteBufferUtils.readCompressedInt(bis);
-
- assertEquals(value, parsedValue);
- }
-
- private void testPutInt(int value) {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- try {
- ByteBufferUtils.putInt(baos, value);
- } catch (IOException e) {
- throw new RuntimeException("Bug in putIn()", e);
- }
-
- ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
- DataInputStream dis = new DataInputStream(bais);
- try {
- assertEquals(dis.readInt(), value);
- } catch (IOException e) {
- throw new RuntimeException("Bug in test!", e);
- }
- }
-
- @Test
- public void testToBytes(){
- ByteBuffer buffer = ByteBuffer.allocate(5);
- buffer.put(new byte[]{0,1,2,3,4});
- assertEquals(5, buffer.position());
- assertEquals(5, buffer.limit());
- byte[] copy = ByteBufferUtils.toBytes(buffer, 2);
- assertArrayEquals(new byte[]{2,3,4}, copy);
- assertEquals(5, buffer.position());
- assertEquals(5, buffer.limit());
- }
-
- @Test
- public void testToPrimitiveTypes() {
- ByteBuffer buffer = ByteBuffer.allocate(15);
- long l = 988L;
- int i = 135;
- short s = 7;
- buffer.putLong(l);
- buffer.putShort(s);
- buffer.putInt(i);
- assertEquals(l, ByteBufferUtils.toLong(buffer, 0));
- assertEquals(s, ByteBufferUtils.toShort(buffer, 8));
- assertEquals(i, ByteBufferUtils.toInt(buffer, 10));
- }
-
- @Test
- public void testCopyFromArrayToBuffer() {
- byte[] b = new byte[15];
- b[0] = -1;
- long l = 988L;
- int i = 135;
- short s = 7;
- Bytes.putLong(b, 1, l);
- Bytes.putShort(b, 9, s);
- Bytes.putInt(b, 11, i);
- ByteBuffer buffer = ByteBuffer.allocate(14);
- ByteBufferUtils.copyFromArrayToBuffer(buffer, b, 1, 14);
- buffer.rewind();
- assertEquals(l, buffer.getLong());
- assertEquals(s, buffer.getShort());
- assertEquals(i, buffer.getInt());
- }
-
- @Test
- public void testCopyFromBufferToArray() {
- ByteBuffer buffer = ByteBuffer.allocate(15);
- buffer.put((byte) -1);
- long l = 988L;
- int i = 135;
- short s = 7;
- buffer.putShort(s);
- buffer.putInt(i);
- buffer.putLong(l);
- byte[] b = new byte[15];
- ByteBufferUtils.copyFromBufferToArray(b, buffer, 1, 1, 14);
- assertEquals(s, Bytes.toShort(b, 1));
- assertEquals(i, Bytes.toInt(b, 3));
- assertEquals(l, Bytes.toLong(b, 7));
- }
-
- @Test
- public void testCompareTo() {
- ByteBuffer bb1 = ByteBuffer.allocate(135);
- ByteBuffer bb2 = ByteBuffer.allocate(135);
- byte[] b = new byte[71];
- fillBB(bb1, (byte) 5);
- fillBB(bb2, (byte) 5);
- fillArray(b, (byte) 5);
- assertEquals(0, ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()));
- assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), b, 0, b.length) > 0);
- bb2.put(134, (byte) 6);
- assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) < 0);
- bb2.put(6, (byte) 4);
- assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) > 0);
- // Assert reverse comparing BB and bytearray works.
- ByteBuffer bb3 = ByteBuffer.allocate(135);
- fillBB(bb3, (byte)0);
- byte[] b3 = new byte[135];
- fillArray(b3, (byte)1);
- int result = ByteBufferUtils.compareTo(b3, 0, b3.length, bb3, 0, bb3.remaining());
- assertTrue(result > 0);
- result = ByteBufferUtils.compareTo(bb3, 0, bb3.remaining(), b3, 0, b3.length);
- assertTrue(result < 0);
- }
-
- private static void fillBB(ByteBuffer bb, byte b) {
- for (int i = bb.position(); i < bb.limit(); i++) {
- bb.put(i, b);
- }
- }
-
- private static void fillArray(byte[] bb, byte b) {
- for (int i = 0; i < bb.length; i++) {
- bb[i] = b;
- }
- }
-}