001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.util;
019
020import static org.junit.Assert.assertArrayEquals;
021import static org.junit.Assert.assertEquals;
022import static org.junit.Assert.assertFalse;
023import static org.junit.Assert.assertTrue;
024
025import java.io.ByteArrayInputStream;
026import java.io.ByteArrayOutputStream;
027import java.io.DataInputStream;
028import java.io.DataOutputStream;
029import java.io.IOException;
030import java.lang.reflect.Field;
031import java.lang.reflect.Modifier;
032import java.nio.ByteBuffer;
033import java.util.ArrayList;
034import java.util.Arrays;
035import java.util.Collection;
036import java.util.Collections;
037import java.util.List;
038import java.util.Set;
039import java.util.SortedSet;
040import java.util.TreeSet;
041import java.util.concurrent.CountDownLatch;
042import java.util.concurrent.ExecutorService;
043import java.util.concurrent.Executors;
044import java.util.concurrent.TimeUnit;
045import java.util.stream.Collectors;
046import org.apache.hadoop.hbase.HBaseClassTestRule;
047import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
048import org.apache.hadoop.hbase.HConstants;
049import org.apache.hadoop.hbase.testclassification.MediumTests;
050import org.apache.hadoop.hbase.testclassification.MiscTests;
051import org.apache.hadoop.hbase.unsafe.HBasePlatformDependent;
052import org.apache.hadoop.io.WritableUtils;
053import org.junit.AfterClass;
054import org.junit.Before;
055import org.junit.ClassRule;
056import org.junit.Test;
057import org.junit.experimental.categories.Category;
058import org.junit.runner.RunWith;
059import org.junit.runners.Parameterized;
060
061@Category({ MiscTests.class, MediumTests.class })
062@RunWith(Parameterized.class)
063public class TestByteBufferUtils {
064
065  @ClassRule
066  public static final HBaseClassTestRule CLASS_RULE =
067    HBaseClassTestRule.forClass(TestByteBufferUtils.class);
068
069  private static final String UNSAFE_AVAIL_NAME = "UNSAFE_AVAIL";
070  private static final String UNSAFE_UNALIGNED_NAME = "UNSAFE_UNALIGNED";
071  private byte[] array;
072
073  @AfterClass
074  public static void afterClass() throws Exception {
075    detectAvailabilityOfUnsafe();
076  }
077
078  @Parameterized.Parameters
079  public static Collection<Object[]> parameters() {
080    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
081  }
082
083  private static void setUnsafe(String fieldName, boolean value) throws Exception {
084    Field field = ByteBufferUtils.class.getDeclaredField(fieldName);
085    field.setAccessible(true);
086    Field modifiersField = Field.class.getDeclaredField("modifiers");
087    modifiersField.setAccessible(true);
088    int oldModifiers = field.getModifiers();
089    modifiersField.setInt(field, oldModifiers & ~Modifier.FINAL);
090    try {
091      field.set(null, value);
092    } finally {
093      modifiersField.setInt(field, oldModifiers);
094    }
095  }
096
097  static void disableUnsafe() throws Exception {
098    if (ByteBufferUtils.UNSAFE_AVAIL) {
099      setUnsafe(UNSAFE_AVAIL_NAME, false);
100    }
101    if (ByteBufferUtils.UNSAFE_UNALIGNED) {
102      setUnsafe(UNSAFE_UNALIGNED_NAME, false);
103    }
104    assertFalse(ByteBufferUtils.UNSAFE_AVAIL);
105    assertFalse(ByteBufferUtils.UNSAFE_UNALIGNED);
106  }
107
108  static void detectAvailabilityOfUnsafe() throws Exception {
109    if (ByteBufferUtils.UNSAFE_AVAIL != HBasePlatformDependent.isUnsafeAvailable()) {
110      setUnsafe(UNSAFE_AVAIL_NAME, HBasePlatformDependent.isUnsafeAvailable());
111    }
112    if (ByteBufferUtils.UNSAFE_UNALIGNED != HBasePlatformDependent.unaligned()) {
113      setUnsafe(UNSAFE_UNALIGNED_NAME, HBasePlatformDependent.unaligned());
114    }
115    assertEquals(ByteBufferUtils.UNSAFE_AVAIL, HBasePlatformDependent.isUnsafeAvailable());
116    assertEquals(ByteBufferUtils.UNSAFE_UNALIGNED, HBasePlatformDependent.unaligned());
117  }
118
119  public TestByteBufferUtils(boolean useUnsafeIfPossible) throws Exception {
120    if (useUnsafeIfPossible) {
121      detectAvailabilityOfUnsafe();
122    } else {
123      disableUnsafe();
124    }
125  }
126
127  /**
128   * Create an array with sample data.
129   */
130  @Before
131  public void setUp() {
132    array = new byte[8];
133    for (int i = 0; i < array.length; ++i) {
134      array[i] = (byte) ('a' + i);
135    }
136  }
137
138  private static final int MAX_VLONG_LENGTH = 9;
139  private static final Collection<Long> testNumbers;
140
141  private static void addNumber(Set<Long> a, long l) {
142    if (l != Long.MIN_VALUE) {
143      a.add(l - 1);
144    }
145    a.add(l);
146    if (l != Long.MAX_VALUE) {
147      a.add(l + 1);
148    }
149    for (long divisor = 3; divisor <= 10; ++divisor) {
150      for (long delta = -1; delta <= 1; ++delta) {
151        a.add(l / divisor + delta);
152      }
153    }
154  }
155
156  static {
157    SortedSet<Long> a = new TreeSet<>();
158    for (int i = 0; i <= 63; ++i) {
159      long v = -1L << i;
160      assertTrue(v < 0);
161      addNumber(a, v);
162      v = (1L << i) - 1;
163      assertTrue(v >= 0);
164      addNumber(a, v);
165    }
166
167    testNumbers = Collections.unmodifiableSet(a);
168    System.err.println("Testing variable-length long serialization using: " + testNumbers
169      + " (count: " + testNumbers.size() + ")");
170    assertEquals(1753, testNumbers.size());
171    assertEquals(Long.MIN_VALUE, a.first().longValue());
172    assertEquals(Long.MAX_VALUE, a.last().longValue());
173  }
174
175  @Test
176  public void testReadWriteVLong() {
177    for (long l : testNumbers) {
178      ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH);
179      ByteBufferUtils.writeVLong(b, l);
180      b.flip();
181      assertEquals(l, ByteBufferUtils.readVLong(b));
182    }
183  }
184
185  @Test
186  public void testConsistencyWithHadoopVLong() throws IOException {
187    ByteArrayOutputStream baos = new ByteArrayOutputStream();
188    DataOutputStream dos = new DataOutputStream(baos);
189    for (long l : testNumbers) {
190      baos.reset();
191      ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH);
192      ByteBufferUtils.writeVLong(b, l);
193      String bufStr = Bytes.toStringBinary(b.array(), b.arrayOffset(), b.position());
194      WritableUtils.writeVLong(dos, l);
195      String baosStr = Bytes.toStringBinary(baos.toByteArray());
196      assertEquals(baosStr, bufStr);
197    }
198  }
199
200  /**
201   * Test copying to stream from buffer.
202   */
203  @Test
204  public void testMoveBufferToStream() throws IOException {
205    final int arrayOffset = 7;
206    final int initialPosition = 10;
207    final int endPadding = 5;
208    byte[] arrayWrapper = new byte[arrayOffset + initialPosition + array.length + endPadding];
209    System.arraycopy(array, 0, arrayWrapper, arrayOffset + initialPosition, array.length);
210    ByteBuffer buffer =
211      ByteBuffer.wrap(arrayWrapper, arrayOffset, initialPosition + array.length).slice();
212    assertEquals(initialPosition + array.length, buffer.limit());
213    assertEquals(0, buffer.position());
214    buffer.position(initialPosition);
215    ByteArrayOutputStream bos = new ByteArrayOutputStream();
216    ByteBufferUtils.moveBufferToStream(bos, buffer, array.length);
217    assertArrayEquals(array, bos.toByteArray());
218    assertEquals(initialPosition + array.length, buffer.position());
219  }
220
221  /**
222   * Test copying to stream from buffer with offset.
223   * @throws IOException On test failure.
224   */
225  @Test
226  public void testCopyToStreamWithOffset() throws IOException {
227    ByteBuffer buffer = ByteBuffer.wrap(array);
228
229    ByteArrayOutputStream bos = new ByteArrayOutputStream();
230
231    ByteBufferUtils.copyBufferToStream(bos, buffer, array.length / 2, array.length / 2);
232
233    byte[] returnedArray = bos.toByteArray();
234    for (int i = 0; i < array.length / 2; ++i) {
235      int pos = array.length / 2 + i;
236      assertEquals(returnedArray[i], array[pos]);
237    }
238  }
239
240  /**
241   * Test copying data from stream.
242   * @throws IOException On test failure.
243   */
244  @Test
245  public void testCopyFromStream() throws IOException {
246    ByteBuffer buffer = ByteBuffer.allocate(array.length);
247    ByteArrayInputStream bis = new ByteArrayInputStream(array);
248    DataInputStream dis = new DataInputStream(bis);
249
250    ByteBufferUtils.copyFromStreamToBuffer(buffer, dis, array.length / 2);
251    ByteBufferUtils.copyFromStreamToBuffer(buffer, dis, array.length - array.length / 2);
252    for (int i = 0; i < array.length; ++i) {
253      assertEquals(array[i], buffer.get(i));
254    }
255  }
256
257  /**
258   * Test copying from buffer.
259   */
260  @Test
261  public void testCopyFromBuffer() {
262    ByteBuffer srcBuffer = ByteBuffer.allocate(array.length);
263    ByteBuffer dstBuffer = ByteBuffer.allocate(array.length);
264    srcBuffer.put(array);
265
266    ByteBufferUtils.copyFromBufferToBuffer(srcBuffer, dstBuffer, array.length / 2,
267      array.length / 4);
268    for (int i = 0; i < array.length / 4; ++i) {
269      assertEquals(srcBuffer.get(i + array.length / 2), dstBuffer.get(i));
270    }
271  }
272
273  /**
274   * Test 7-bit encoding of integers.
275   * @throws IOException On test failure.
276   */
277  @Test
278  public void testCompressedInt() throws IOException {
279    testCompressedInt(0);
280    testCompressedInt(Integer.MAX_VALUE);
281    testCompressedInt(Integer.MIN_VALUE);
282
283    for (int i = 0; i < 3; i++) {
284      testCompressedInt((128 << i) - 1);
285    }
286
287    for (int i = 0; i < 3; i++) {
288      testCompressedInt((128 << i));
289    }
290  }
291
292  /**
293   * Test how much bytes we need to store integer.
294   */
295  @Test
296  public void testIntFitsIn() {
297    assertEquals(1, ByteBufferUtils.intFitsIn(0));
298    assertEquals(1, ByteBufferUtils.intFitsIn(1));
299    assertEquals(2, ByteBufferUtils.intFitsIn(1 << 8));
300    assertEquals(3, ByteBufferUtils.intFitsIn(1 << 16));
301    assertEquals(4, ByteBufferUtils.intFitsIn(-1));
302    assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MAX_VALUE));
303    assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MIN_VALUE));
304  }
305
306  /**
307   * Test how much bytes we need to store long.
308   */
309  @Test
310  public void testLongFitsIn() {
311    assertEquals(1, ByteBufferUtils.longFitsIn(0));
312    assertEquals(1, ByteBufferUtils.longFitsIn(1));
313    assertEquals(3, ByteBufferUtils.longFitsIn(1L << 16));
314    assertEquals(5, ByteBufferUtils.longFitsIn(1L << 32));
315    assertEquals(8, ByteBufferUtils.longFitsIn(-1));
316    assertEquals(8, ByteBufferUtils.longFitsIn(Long.MIN_VALUE));
317    assertEquals(8, ByteBufferUtils.longFitsIn(Long.MAX_VALUE));
318  }
319
320  /**
321   * Test if we are comparing equal bytes.
322   */
323  @Test
324  public void testArePartEqual() {
325    byte[] array = new byte[] { 1, 2, 3, 4, 5, 1, 2, 3, 4 };
326    ByteBuffer buffer = ByteBuffer.wrap(array);
327    assertTrue(ByteBufferUtils.arePartsEqual(buffer, 0, 4, 5, 4));
328    assertTrue(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 2));
329    assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 3));
330    assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 3, 6, 2));
331    assertFalse(ByteBufferUtils.arePartsEqual(buffer, 0, 3, 6, 3));
332  }
333
334  /**
335   * Test serializing int to bytes
336   */
337  @Test
338  public void testPutInt() {
339    testPutInt(0);
340    testPutInt(Integer.MAX_VALUE);
341
342    for (int i = 0; i < 3; i++) {
343      testPutInt((128 << i) - 1);
344    }
345
346    for (int i = 0; i < 3; i++) {
347      testPutInt((128 << i));
348    }
349  }
350
351  // Utility methods invoked from test methods
352
353  private void testCompressedInt(int value) throws IOException {
354    ByteArrayOutputStream bos = new ByteArrayOutputStream();
355    ByteBufferUtils.putCompressedInt(bos, value);
356    ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
357    int parsedValue = ByteBufferUtils.readCompressedInt(bis);
358    assertEquals(value, parsedValue);
359  }
360
361  private void testPutInt(int value) {
362    ByteArrayOutputStream baos = new ByteArrayOutputStream();
363    try {
364      ByteBufferUtils.putInt(baos, value);
365    } catch (IOException e) {
366      throw new RuntimeException("Bug in putIn()", e);
367    }
368
369    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
370    DataInputStream dis = new DataInputStream(bais);
371    try {
372      assertEquals(dis.readInt(), value);
373    } catch (IOException e) {
374      throw new RuntimeException("Bug in test!", e);
375    }
376  }
377
378  @Test
379  public void testToBytes() {
380    ByteBuffer buffer = ByteBuffer.allocate(5);
381    buffer.put(new byte[] { 0, 1, 2, 3, 4 });
382    assertEquals(5, buffer.position());
383    assertEquals(5, buffer.limit());
384    byte[] copy = ByteBufferUtils.toBytes(buffer, 2);
385    assertArrayEquals(new byte[] { 2, 3, 4 }, copy);
386    assertEquals(5, buffer.position());
387    assertEquals(5, buffer.limit());
388  }
389
390  @Test
391  public void testToPrimitiveTypes() {
392    ByteBuffer buffer = ByteBuffer.allocate(15);
393    long l = 988L;
394    int i = 135;
395    short s = 7;
396    buffer.putLong(l);
397    buffer.putShort(s);
398    buffer.putInt(i);
399    assertEquals(l, ByteBufferUtils.toLong(buffer, 0));
400    assertEquals(s, ByteBufferUtils.toShort(buffer, 8));
401    assertEquals(i, ByteBufferUtils.toInt(buffer, 10));
402  }
403
404  @Test
405  public void testCopyFromArrayToBuffer() {
406    byte[] b = new byte[15];
407    b[0] = -1;
408    long l = 988L;
409    int i = 135;
410    short s = 7;
411    Bytes.putLong(b, 1, l);
412    Bytes.putShort(b, 9, s);
413    Bytes.putInt(b, 11, i);
414    ByteBuffer buffer = ByteBuffer.allocate(14);
415    ByteBufferUtils.copyFromArrayToBuffer(buffer, b, 1, 14);
416    buffer.rewind();
417    assertEquals(l, buffer.getLong());
418    assertEquals(s, buffer.getShort());
419    assertEquals(i, buffer.getInt());
420  }
421
422  private void testCopyFromSrcToDestWithThreads(Object input, Object output, List<Integer> lengthes,
423    List<Integer> offsets) throws InterruptedException {
424    assertTrue((input instanceof ByteBuffer) || (input instanceof byte[]));
425    assertTrue((output instanceof ByteBuffer) || (output instanceof byte[]));
426    assertEquals(lengthes.size(), offsets.size());
427
428    final int threads = lengthes.size();
429    CountDownLatch latch = new CountDownLatch(1);
430    List<Runnable> exes = new ArrayList<>(threads);
431    int oldInputPos = (input instanceof ByteBuffer) ? ((ByteBuffer) input).position() : 0;
432    int oldOutputPos = (output instanceof ByteBuffer) ? ((ByteBuffer) output).position() : 0;
433    for (int i = 0; i != threads; ++i) {
434      int offset = offsets.get(i);
435      int length = lengthes.get(i);
436      exes.add(() -> {
437        try {
438          latch.await();
439          if (input instanceof ByteBuffer && output instanceof byte[]) {
440            ByteBufferUtils.copyFromBufferToArray((byte[]) output, (ByteBuffer) input, offset,
441              offset, length);
442          }
443          if (input instanceof byte[] && output instanceof ByteBuffer) {
444            ByteBufferUtils.copyFromArrayToBuffer((ByteBuffer) output, offset, (byte[]) input,
445              offset, length);
446          }
447          if (input instanceof ByteBuffer && output instanceof ByteBuffer) {
448            ByteBufferUtils.copyFromBufferToBuffer((ByteBuffer) input, (ByteBuffer) output, offset,
449              offset, length);
450          }
451        } catch (InterruptedException ex) {
452          throw new RuntimeException(ex);
453        }
454      });
455    }
456    ExecutorService service = Executors.newFixedThreadPool(threads);
457    exes.forEach(service::execute);
458    latch.countDown();
459    service.shutdown();
460    assertTrue(service.awaitTermination(5, TimeUnit.SECONDS));
461    if (input instanceof ByteBuffer) {
462      assertEquals(oldInputPos, ((ByteBuffer) input).position());
463    }
464    if (output instanceof ByteBuffer) {
465      assertEquals(oldOutputPos, ((ByteBuffer) output).position());
466    }
467    String inputString = (input instanceof ByteBuffer)
468      ? Bytes.toString(Bytes.toBytes((ByteBuffer) input))
469      : Bytes.toString((byte[]) input);
470    String outputString = (output instanceof ByteBuffer)
471      ? Bytes.toString(Bytes.toBytes((ByteBuffer) output))
472      : Bytes.toString((byte[]) output);
473    assertEquals(inputString, outputString);
474  }
475
476  @Test
477  public void testCopyFromSrcToDestWithThreads() throws InterruptedException {
478    List<byte[]> words =
479      Arrays.asList(Bytes.toBytes("with"), Bytes.toBytes("great"), Bytes.toBytes("power"),
480        Bytes.toBytes("comes"), Bytes.toBytes("great"), Bytes.toBytes("responsibility"));
481    List<Integer> lengthes = words.stream().map(v -> v.length).collect(Collectors.toList());
482    List<Integer> offsets = new ArrayList<>(words.size());
483    for (int i = 0; i != words.size(); ++i) {
484      offsets.add(words.subList(0, i).stream().mapToInt(v -> v.length).sum());
485    }
486
487    int totalSize = words.stream().mapToInt(v -> v.length).sum();
488    byte[] fullContent = new byte[totalSize];
489    int offset = 0;
490    for (byte[] w : words) {
491      offset = Bytes.putBytes(fullContent, offset, w, 0, w.length);
492    }
493
494    // test copyFromBufferToArray
495    for (ByteBuffer input : Arrays.asList(ByteBuffer.allocateDirect(totalSize),
496      ByteBuffer.allocate(totalSize))) {
497      words.forEach(input::put);
498      byte[] output = new byte[totalSize];
499      testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets);
500    }
501
502    // test copyFromArrayToBuffer
503    for (ByteBuffer output : Arrays.asList(ByteBuffer.allocateDirect(totalSize),
504      ByteBuffer.allocate(totalSize))) {
505      byte[] input = fullContent;
506      testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets);
507    }
508
509    // test copyFromBufferToBuffer
510    for (ByteBuffer input : Arrays.asList(ByteBuffer.allocateDirect(totalSize),
511      ByteBuffer.allocate(totalSize))) {
512      words.forEach(input::put);
513      for (ByteBuffer output : Arrays.asList(ByteBuffer.allocateDirect(totalSize),
514        ByteBuffer.allocate(totalSize))) {
515        testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets);
516      }
517    }
518  }
519
520  @Test
521  public void testCopyFromBufferToArray() {
522    ByteBuffer buffer = ByteBuffer.allocate(15);
523    buffer.put((byte) -1);
524    long l = 988L;
525    int i = 135;
526    short s = 7;
527    buffer.putShort(s);
528    buffer.putInt(i);
529    buffer.putLong(l);
530    byte[] b = new byte[15];
531    ByteBufferUtils.copyFromBufferToArray(b, buffer, 1, 1, 14);
532    assertEquals(s, Bytes.toShort(b, 1));
533    assertEquals(i, Bytes.toInt(b, 3));
534    assertEquals(l, Bytes.toLong(b, 7));
535  }
536
537  @Test
538  public void testRelativeCopyFromBuffertoBuffer() {
539    ByteBuffer bb1 = ByteBuffer.allocate(135);
540    ByteBuffer bb2 = ByteBuffer.allocate(135);
541    fillBB(bb1, (byte) 5);
542    ByteBufferUtils.copyFromBufferToBuffer(bb1, bb2);
543    assertTrue(bb1.position() == bb2.position());
544    assertTrue(bb1.limit() == bb2.limit());
545    bb1 = ByteBuffer.allocateDirect(135);
546    bb2 = ByteBuffer.allocateDirect(135);
547    fillBB(bb1, (byte) 5);
548    ByteBufferUtils.copyFromBufferToBuffer(bb1, bb2);
549    assertTrue(bb1.position() == bb2.position());
550    assertTrue(bb1.limit() == bb2.limit());
551  }
552
553  @Test
554  public void testCompareTo() {
555    ByteBuffer bb1 = ByteBuffer.allocate(135);
556    ByteBuffer bb2 = ByteBuffer.allocate(135);
557    byte[] b = new byte[71];
558    fillBB(bb1, (byte) 5);
559    fillBB(bb2, (byte) 5);
560    fillArray(b, (byte) 5);
561    assertEquals(0, ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()));
562    assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), b, 0, b.length) > 0);
563    bb2.put(134, (byte) 6);
564    assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) < 0);
565    bb2.put(6, (byte) 4);
566    assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) > 0);
567    // Assert reverse comparing BB and bytearray works.
568    ByteBuffer bb3 = ByteBuffer.allocate(135);
569    fillBB(bb3, (byte) 0);
570    byte[] b3 = new byte[135];
571    fillArray(b3, (byte) 1);
572    int result = ByteBufferUtils.compareTo(b3, 0, b3.length, bb3, 0, bb3.remaining());
573    assertTrue(result > 0);
574    result = ByteBufferUtils.compareTo(bb3, 0, bb3.remaining(), b3, 0, b3.length);
575    assertTrue(result < 0);
576    byte[] b4 = Bytes.toBytes("123");
577    ByteBuffer bb4 = ByteBuffer.allocate(10 + b4.length);
578    for (int i = 10; i < bb4.capacity(); ++i) {
579      bb4.put(i, b4[i - 10]);
580    }
581    result = ByteBufferUtils.compareTo(b4, 0, b4.length, bb4, 10, b4.length);
582    assertEquals(0, result);
583  }
584
585  @Test
586  public void testEquals() {
587    byte[] a = Bytes.toBytes("http://A");
588    ByteBuffer bb = ByteBuffer.wrap(a);
589
590    assertTrue(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0,
591      HConstants.EMPTY_BYTE_BUFFER, 0, 0));
592
593    assertFalse(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, bb, 0, a.length));
594
595    assertFalse(ByteBufferUtils.equals(bb, 0, 0, HConstants.EMPTY_BYTE_BUFFER, 0, a.length));
596
597    assertTrue(ByteBufferUtils.equals(bb, 0, a.length, bb, 0, a.length));
598
599    assertTrue(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0,
600      HConstants.EMPTY_BYTE_ARRAY, 0, 0));
601
602    assertFalse(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, a, 0, a.length));
603
604    assertFalse(ByteBufferUtils.equals(bb, 0, a.length, HConstants.EMPTY_BYTE_ARRAY, 0, 0));
605
606    assertTrue(ByteBufferUtils.equals(bb, 0, a.length, a, 0, a.length));
607  }
608
609  private static void fillBB(ByteBuffer bb, byte b) {
610    for (int i = bb.position(); i < bb.limit(); i++) {
611      bb.put(i, b);
612    }
613  }
614
615  private static void fillArray(byte[] bb, byte b) {
616    for (int i = 0; i < bb.length; i++) {
617      bb[i] = b;
618    }
619  }
620
621}