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; 025 026import java.io.ByteArrayInputStream; 027import java.io.ByteArrayOutputStream; 028import java.io.DataInputStream; 029import java.io.DataOutputStream; 030import java.io.IOException; 031import java.lang.reflect.Field; 032import java.lang.reflect.Modifier; 033import java.nio.ByteBuffer; 034import java.util.ArrayList; 035import java.util.Arrays; 036import java.util.Collection; 037import java.util.Collections; 038import java.util.List; 039import java.util.Set; 040import java.util.SortedSet; 041import java.util.TreeSet; 042import java.util.concurrent.CountDownLatch; 043import java.util.concurrent.ExecutorService; 044import java.util.concurrent.Executors; 045import java.util.concurrent.TimeUnit; 046import java.util.stream.Collectors; 047import org.apache.hadoop.hbase.HBaseClassTestRule; 048import org.apache.hadoop.hbase.HBaseCommonTestingUtility; 049import org.apache.hadoop.hbase.HConstants; 050import org.apache.hadoop.hbase.testclassification.MiscTests; 051import org.apache.hadoop.hbase.testclassification.SmallTests; 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, SmallTests.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 != UnsafeAvailChecker.isAvailable()) { 110 setUnsafe(UNSAFE_AVAIL_NAME, UnsafeAvailChecker.isAvailable()); 111 } 112 if (ByteBufferUtils.UNSAFE_UNALIGNED != UnsafeAvailChecker.unaligned()) { 113 setUnsafe(UNSAFE_UNALIGNED_NAME, UnsafeAvailChecker.unaligned()); 114 } 115 assertEquals(ByteBufferUtils.UNSAFE_AVAIL, UnsafeAvailChecker.isAvailable()); 116 assertEquals(ByteBufferUtils.UNSAFE_UNALIGNED, UnsafeAvailChecker.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: " 169 + testNumbers + " (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(), 194 b.arrayOffset(), b.position()); 195 WritableUtils.writeVLong(dos, l); 196 String baosStr = Bytes.toStringBinary(baos.toByteArray()); 197 assertEquals(baosStr, bufStr); 198 } 199 } 200 201 /** 202 * Test copying to stream from buffer. 203 */ 204 @Test 205 public void testMoveBufferToStream() { 206 final int arrayOffset = 7; 207 final int initialPosition = 10; 208 final int endPadding = 5; 209 byte[] arrayWrapper = 210 new byte[arrayOffset + initialPosition + array.length + endPadding]; 211 System.arraycopy(array, 0, arrayWrapper, 212 arrayOffset + initialPosition, array.length); 213 ByteBuffer buffer = ByteBuffer.wrap(arrayWrapper, arrayOffset, 214 initialPosition + array.length).slice(); 215 assertEquals(initialPosition + array.length, buffer.limit()); 216 assertEquals(0, buffer.position()); 217 buffer.position(initialPosition); 218 ByteArrayOutputStream bos = new ByteArrayOutputStream(); 219 try { 220 ByteBufferUtils.moveBufferToStream(bos, buffer, array.length); 221 } catch (IOException e) { 222 fail("IOException in testCopyToStream()"); 223 } 224 assertArrayEquals(array, bos.toByteArray()); 225 assertEquals(initialPosition + array.length, buffer.position()); 226 } 227 228 /** 229 * Test copying to stream from buffer with offset. 230 * @throws IOException On test failure. 231 */ 232 @Test 233 public void testCopyToStreamWithOffset() throws IOException { 234 ByteBuffer buffer = ByteBuffer.wrap(array); 235 236 ByteArrayOutputStream bos = new ByteArrayOutputStream(); 237 238 ByteBufferUtils.copyBufferToStream(bos, buffer, array.length / 2, 239 array.length / 2); 240 241 byte[] returnedArray = bos.toByteArray(); 242 for (int i = 0; i < array.length / 2; ++i) { 243 int pos = array.length / 2 + i; 244 assertEquals(returnedArray[i], array[pos]); 245 } 246 } 247 248 /** 249 * Test copying data from stream. 250 * @throws IOException On test failure. 251 */ 252 @Test 253 public void testCopyFromStream() throws IOException { 254 ByteBuffer buffer = ByteBuffer.allocate(array.length); 255 ByteArrayInputStream bis = new ByteArrayInputStream(array); 256 DataInputStream dis = new DataInputStream(bis); 257 258 ByteBufferUtils.copyFromStreamToBuffer(buffer, dis, array.length / 2); 259 ByteBufferUtils.copyFromStreamToBuffer(buffer, dis, 260 array.length - array.length / 2); 261 for (int i = 0; i < array.length; ++i) { 262 assertEquals(array[i], buffer.get(i)); 263 } 264 } 265 266 /** 267 * Test copying from buffer. 268 */ 269 @Test 270 public void testCopyFromBuffer() { 271 ByteBuffer srcBuffer = ByteBuffer.allocate(array.length); 272 ByteBuffer dstBuffer = ByteBuffer.allocate(array.length); 273 srcBuffer.put(array); 274 275 ByteBufferUtils.copyFromBufferToBuffer(srcBuffer, dstBuffer, 276 array.length / 2, array.length / 4); 277 for (int i = 0; i < array.length / 4; ++i) { 278 assertEquals(srcBuffer.get(i + array.length / 2), 279 dstBuffer.get(i)); 280 } 281 } 282 283 /** 284 * Test 7-bit encoding of integers. 285 * @throws IOException On test failure. 286 */ 287 @Test 288 public void testCompressedInt() throws IOException { 289 testCompressedInt(0); 290 testCompressedInt(Integer.MAX_VALUE); 291 testCompressedInt(Integer.MIN_VALUE); 292 293 for (int i = 0; i < 3; i++) { 294 testCompressedInt((128 << i) - 1); 295 } 296 297 for (int i = 0; i < 3; i++) { 298 testCompressedInt((128 << i)); 299 } 300 } 301 302 /** 303 * Test how much bytes we need to store integer. 304 */ 305 @Test 306 public void testIntFitsIn() { 307 assertEquals(1, ByteBufferUtils.intFitsIn(0)); 308 assertEquals(1, ByteBufferUtils.intFitsIn(1)); 309 assertEquals(2, ByteBufferUtils.intFitsIn(1 << 8)); 310 assertEquals(3, ByteBufferUtils.intFitsIn(1 << 16)); 311 assertEquals(4, ByteBufferUtils.intFitsIn(-1)); 312 assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MAX_VALUE)); 313 assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MIN_VALUE)); 314 } 315 316 /** 317 * Test how much bytes we need to store long. 318 */ 319 @Test 320 public void testLongFitsIn() { 321 assertEquals(1, ByteBufferUtils.longFitsIn(0)); 322 assertEquals(1, ByteBufferUtils.longFitsIn(1)); 323 assertEquals(3, ByteBufferUtils.longFitsIn(1L << 16)); 324 assertEquals(5, ByteBufferUtils.longFitsIn(1L << 32)); 325 assertEquals(8, ByteBufferUtils.longFitsIn(-1)); 326 assertEquals(8, ByteBufferUtils.longFitsIn(Long.MIN_VALUE)); 327 assertEquals(8, ByteBufferUtils.longFitsIn(Long.MAX_VALUE)); 328 } 329 330 /** 331 * Test if we are comparing equal bytes. 332 */ 333 @Test 334 public void testArePartEqual() { 335 byte[] array = new byte[] { 1, 2, 3, 4, 5, 1, 2, 3, 4 }; 336 ByteBuffer buffer = ByteBuffer.wrap(array); 337 assertTrue(ByteBufferUtils.arePartsEqual(buffer, 0, 4, 5, 4)); 338 assertTrue(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 2)); 339 assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 3)); 340 assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 3, 6, 2)); 341 assertFalse(ByteBufferUtils.arePartsEqual(buffer, 0, 3, 6, 3)); 342 } 343 344 /** 345 * Test serializing int to bytes 346 */ 347 @Test 348 public void testPutInt() { 349 testPutInt(0); 350 testPutInt(Integer.MAX_VALUE); 351 352 for (int i = 0; i < 3; i++) { 353 testPutInt((128 << i) - 1); 354 } 355 356 for (int i = 0; i < 3; i++) { 357 testPutInt((128 << i)); 358 } 359 } 360 361 // Utility methods invoked from test methods 362 363 private void testCompressedInt(int value) throws IOException { 364 int parsedValue = 0; 365 366 ByteArrayOutputStream bos = new ByteArrayOutputStream(); 367 ByteBufferUtils.putCompressedInt(bos, value); 368 369 ByteArrayInputStream bis = new ByteArrayInputStream( 370 bos.toByteArray()); 371 parsedValue = ByteBufferUtils.readCompressedInt(bis); 372 373 assertEquals(value, parsedValue); 374 } 375 376 private void testPutInt(int value) { 377 ByteArrayOutputStream baos = new ByteArrayOutputStream(); 378 try { 379 ByteBufferUtils.putInt(baos, value); 380 } catch (IOException e) { 381 throw new RuntimeException("Bug in putIn()", e); 382 } 383 384 ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); 385 DataInputStream dis = new DataInputStream(bais); 386 try { 387 assertEquals(dis.readInt(), value); 388 } catch (IOException e) { 389 throw new RuntimeException("Bug in test!", e); 390 } 391 } 392 393 @Test 394 public void testToBytes(){ 395 ByteBuffer buffer = ByteBuffer.allocate(5); 396 buffer.put(new byte[]{0,1,2,3,4}); 397 assertEquals(5, buffer.position()); 398 assertEquals(5, buffer.limit()); 399 byte[] copy = ByteBufferUtils.toBytes(buffer, 2); 400 assertArrayEquals(new byte[]{2,3,4}, copy); 401 assertEquals(5, buffer.position()); 402 assertEquals(5, buffer.limit()); 403 } 404 405 @Test 406 public void testToPrimitiveTypes() { 407 ByteBuffer buffer = ByteBuffer.allocate(15); 408 long l = 988L; 409 int i = 135; 410 short s = 7; 411 buffer.putLong(l); 412 buffer.putShort(s); 413 buffer.putInt(i); 414 assertEquals(l, ByteBufferUtils.toLong(buffer, 0)); 415 assertEquals(s, ByteBufferUtils.toShort(buffer, 8)); 416 assertEquals(i, ByteBufferUtils.toInt(buffer, 10)); 417 } 418 419 @Test 420 public void testCopyFromArrayToBuffer() { 421 byte[] b = new byte[15]; 422 b[0] = -1; 423 long l = 988L; 424 int i = 135; 425 short s = 7; 426 Bytes.putLong(b, 1, l); 427 Bytes.putShort(b, 9, s); 428 Bytes.putInt(b, 11, i); 429 ByteBuffer buffer = ByteBuffer.allocate(14); 430 ByteBufferUtils.copyFromArrayToBuffer(buffer, b, 1, 14); 431 buffer.rewind(); 432 assertEquals(l, buffer.getLong()); 433 assertEquals(s, buffer.getShort()); 434 assertEquals(i, buffer.getInt()); 435 } 436 437 private void testCopyFromSrcToDestWithThreads(Object input, Object output, 438 List<Integer> lengthes, List<Integer> offsets) throws InterruptedException { 439 assertTrue((input instanceof ByteBuffer) || (input instanceof byte[])); 440 assertTrue((output instanceof ByteBuffer) || (output instanceof byte[])); 441 assertEquals(lengthes.size(), offsets.size()); 442 443 final int threads = lengthes.size(); 444 CountDownLatch latch = new CountDownLatch(1); 445 List<Runnable> exes = new ArrayList<>(threads); 446 int oldInputPos = (input instanceof ByteBuffer) ? ((ByteBuffer) input).position() : 0; 447 int oldOutputPos = (output instanceof ByteBuffer) ? ((ByteBuffer) output).position() : 0; 448 for (int i = 0; i != threads; ++i) { 449 int offset = offsets.get(i); 450 int length = lengthes.get(i); 451 exes.add(() -> { 452 try { 453 latch.await(); 454 if (input instanceof ByteBuffer && output instanceof byte[]) { 455 ByteBufferUtils.copyFromBufferToArray((byte[]) output, 456 (ByteBuffer) input, offset, offset, length); 457 } 458 if (input instanceof byte[] && output instanceof ByteBuffer) { 459 ByteBufferUtils.copyFromArrayToBuffer((ByteBuffer) output, 460 offset, (byte[]) input, offset, length); 461 } 462 if (input instanceof ByteBuffer && output instanceof ByteBuffer) { 463 ByteBufferUtils.copyFromBufferToBuffer((ByteBuffer) input, 464 (ByteBuffer) output, offset, offset, length); 465 } 466 } catch (InterruptedException ex) { 467 throw new RuntimeException(ex); 468 } 469 }); 470 } 471 ExecutorService service = Executors.newFixedThreadPool(threads); 472 exes.forEach(service::execute); 473 latch.countDown(); 474 service.shutdown(); 475 assertTrue(service.awaitTermination(5, TimeUnit.SECONDS)); 476 if (input instanceof ByteBuffer) { 477 assertEquals(oldInputPos, ((ByteBuffer) input).position()); 478 } 479 if (output instanceof ByteBuffer) { 480 assertEquals(oldOutputPos, ((ByteBuffer) output).position()); 481 } 482 String inputString = (input instanceof ByteBuffer) ? 483 Bytes.toString(Bytes.toBytes((ByteBuffer) input)) : Bytes.toString((byte[]) input); 484 String outputString = (output instanceof ByteBuffer) ? 485 Bytes.toString(Bytes.toBytes((ByteBuffer) output)) : Bytes.toString((byte[]) output); 486 assertEquals(inputString, outputString); 487 } 488 489 @Test 490 public void testCopyFromSrcToDestWithThreads() throws InterruptedException { 491 List<byte[]> words = Arrays.asList( 492 Bytes.toBytes("with"), 493 Bytes.toBytes("great"), 494 Bytes.toBytes("power"), 495 Bytes.toBytes("comes"), 496 Bytes.toBytes("great"), 497 Bytes.toBytes("responsibility") 498 ); 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( 514 ByteBuffer.allocateDirect(totalSize), 515 ByteBuffer.allocate(totalSize))) { 516 words.forEach(input::put); 517 byte[] output = new byte[totalSize]; 518 testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets); 519 } 520 521 // test copyFromArrayToBuffer 522 for (ByteBuffer output : Arrays.asList( 523 ByteBuffer.allocateDirect(totalSize), 524 ByteBuffer.allocate(totalSize))) { 525 byte[] input = fullContent; 526 testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets); 527 } 528 529 // test copyFromBufferToBuffer 530 for (ByteBuffer input : Arrays.asList( 531 ByteBuffer.allocateDirect(totalSize), 532 ByteBuffer.allocate(totalSize))) { 533 words.forEach(input::put); 534 for (ByteBuffer output : Arrays.asList( 535 ByteBuffer.allocateDirect(totalSize), 536 ByteBuffer.allocate(totalSize))) { 537 testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets); 538 } 539 } 540 } 541 542 @Test 543 public void testCopyFromBufferToArray() { 544 ByteBuffer buffer = ByteBuffer.allocate(15); 545 buffer.put((byte) -1); 546 long l = 988L; 547 int i = 135; 548 short s = 7; 549 buffer.putShort(s); 550 buffer.putInt(i); 551 buffer.putLong(l); 552 byte[] b = new byte[15]; 553 ByteBufferUtils.copyFromBufferToArray(b, buffer, 1, 1, 14); 554 assertEquals(s, Bytes.toShort(b, 1)); 555 assertEquals(i, Bytes.toInt(b, 3)); 556 assertEquals(l, Bytes.toLong(b, 7)); 557 } 558 559 @Test 560 public void testRelativeCopyFromBuffertoBuffer() { 561 ByteBuffer bb1 = ByteBuffer.allocate(135); 562 ByteBuffer bb2 = ByteBuffer.allocate(135); 563 fillBB(bb1, (byte) 5); 564 ByteBufferUtils.copyFromBufferToBuffer(bb1, bb2); 565 assertTrue(bb1.position() == bb2.position()); 566 assertTrue(bb1.limit() == bb2.limit()); 567 bb1 = ByteBuffer.allocateDirect(135); 568 bb2 = ByteBuffer.allocateDirect(135); 569 fillBB(bb1, (byte) 5); 570 ByteBufferUtils.copyFromBufferToBuffer(bb1, bb2); 571 assertTrue(bb1.position() == bb2.position()); 572 assertTrue(bb1.limit() == bb2.limit()); 573 } 574 @Test 575 public void testCompareTo() { 576 ByteBuffer bb1 = ByteBuffer.allocate(135); 577 ByteBuffer bb2 = ByteBuffer.allocate(135); 578 byte[] b = new byte[71]; 579 fillBB(bb1, (byte) 5); 580 fillBB(bb2, (byte) 5); 581 fillArray(b, (byte) 5); 582 assertEquals(0, ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining())); 583 assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), b, 0, b.length) > 0); 584 bb2.put(134, (byte) 6); 585 assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) < 0); 586 bb2.put(6, (byte) 4); 587 assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) > 0); 588 // Assert reverse comparing BB and bytearray works. 589 ByteBuffer bb3 = ByteBuffer.allocate(135); 590 fillBB(bb3, (byte)0); 591 byte[] b3 = new byte[135]; 592 fillArray(b3, (byte)1); 593 int result = ByteBufferUtils.compareTo(b3, 0, b3.length, bb3, 0, bb3.remaining()); 594 assertTrue(result > 0); 595 result = ByteBufferUtils.compareTo(bb3, 0, bb3.remaining(), b3, 0, b3.length); 596 assertTrue(result < 0); 597 598 byte[] b4 = Bytes.toBytes("123"); 599 ByteBuffer bb4 = ByteBuffer.allocate(10 + b4.length); 600 for (int i = 10; i < (bb4.capacity()); ++i) { 601 bb4.put(i, b4[i - 10]); 602 } 603 result = ByteBufferUtils.compareTo(b4, 0, b4.length, bb4, 10, b4.length); 604 assertEquals(0, result); 605 } 606 607 @Test 608 public void testEquals() { 609 byte[] a = Bytes.toBytes("http://A"); 610 ByteBuffer bb = ByteBuffer.wrap(a); 611 612 assertTrue(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, 613 HConstants.EMPTY_BYTE_BUFFER, 0, 0)); 614 615 assertFalse(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, bb, 616 0, a.length)); 617 618 assertFalse(ByteBufferUtils.equals(bb, 0, 0, HConstants.EMPTY_BYTE_BUFFER, 619 0, a.length)); 620 621 assertTrue(ByteBufferUtils.equals(bb, 0, a.length, bb, 0, a.length)); 622 623 assertTrue(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, 624 HConstants.EMPTY_BYTE_ARRAY, 0, 0)); 625 626 assertFalse(ByteBufferUtils.equals(HConstants.EMPTY_BYTE_BUFFER, 0, 0, a, 627 0, a.length)); 628 629 assertFalse(ByteBufferUtils.equals(bb, 0, a.length, 630 HConstants.EMPTY_BYTE_ARRAY, 0, 0)); 631 632 assertTrue(ByteBufferUtils.equals(bb, 0, a.length, a, 0, a.length)); 633 } 634 635 private static void fillBB(ByteBuffer bb, byte b) { 636 for (int i = bb.position(); i < bb.limit(); i++) { 637 bb.put(i, b); 638 } 639 } 640 641 private static void fillArray(byte[] bb, byte b) { 642 for (int i = 0; i < bb.length; i++) { 643 bb[i] = b; 644 } 645 } 646 647}