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