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