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