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