|
38 | 38 | import static org.apache.hadoop.fs.contract.ContractTestUtils.compareByteArrays;
|
39 | 39 | import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
40 | 40 | import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
|
| 41 | +import static org.apache.hadoop.fs.contract.ContractTestUtils.readDataset; |
| 42 | +import static org.apache.hadoop.fs.contract.ContractTestUtils.readDatasetSingleByteReads; |
| 43 | +import static org.apache.hadoop.fs.contract.ContractTestUtils.readNBytes; |
41 | 44 | import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
|
42 | 45 | import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
43 | 46 | import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
|
@@ -408,4 +411,137 @@ public void testFloatingPointLength() throws Throwable {
|
408 | 411 | .isEqualTo(len);
|
409 | 412 | }
|
410 | 413 |
|
| 414 | + @Test |
| 415 | + public void testInputStreamReadNullBuffer() throws Throwable { |
| 416 | + // The JDK base InputStream (and by extension LocalFSFileInputStream) throws |
| 417 | + // NullPointerException. Historically, DFSInputStream has thrown IllegalArgumentException |
| 418 | + // instead. Allow either behavior. |
| 419 | + describe("Attempting to read into a null buffer should throw IllegalArgumentException or " + |
| 420 | + "NullPointerException"); |
| 421 | + Path path = methodPath(); |
| 422 | + FileSystem fs = getFileSystem(); |
| 423 | + int len = 4096; |
| 424 | + createFile(fs, path, true, |
| 425 | + dataset(len, 0x40, 0x80)); |
| 426 | + try (FSDataInputStream is = fs.openFile(path).build().get()) { |
| 427 | + Assertions.assertThatThrownBy(() -> is.read(null, 0, 10)) |
| 428 | + .isInstanceOfAny(IllegalArgumentException.class, NullPointerException.class); |
| 429 | + } |
| 430 | + } |
| 431 | + |
| 432 | + @Test |
| 433 | + public void testInputStreamReadNegativePosition() throws Throwable { |
| 434 | + describe("Attempting to read into a negative position should throw IndexOutOfBoundsException"); |
| 435 | + Path path = methodPath(); |
| 436 | + FileSystem fs = getFileSystem(); |
| 437 | + int len = 4096; |
| 438 | + createFile(fs, path, true, |
| 439 | + dataset(len, 0x40, 0x80)); |
| 440 | + try (FSDataInputStream is = fs.openFile(path).build().get()) { |
| 441 | + Assertions.assertThatThrownBy(() -> is.read(new byte[10], -1, 10)) |
| 442 | + .isInstanceOf(IndexOutOfBoundsException.class); |
| 443 | + } |
| 444 | + } |
| 445 | + |
| 446 | + @Test |
| 447 | + public void testInputStreamReadNegativeLength() throws Throwable { |
| 448 | + // The JDK base InputStream (and by extension LocalFSFileInputStream) throws |
| 449 | + // IndexOutOfBoundsException. Historically, DFSInputStream has thrown IllegalArgumentException |
| 450 | + // instead. Allow either behavior. |
| 451 | + describe("Attempting to read into a null buffer should throw IllegalArgumentException or " + |
| 452 | + "IndexOutOfBoundsException"); |
| 453 | + Path path = methodPath(); |
| 454 | + FileSystem fs = getFileSystem(); |
| 455 | + int len = 4096; |
| 456 | + createFile(fs, path, true, |
| 457 | + dataset(len, 0x40, 0x80)); |
| 458 | + try (FSDataInputStream is = fs.openFile(path).build().get()) { |
| 459 | + Assertions.assertThatThrownBy(() -> is.read(new byte[10], 0, -1)) |
| 460 | + .isInstanceOfAny(IllegalArgumentException.class, IndexOutOfBoundsException.class); |
| 461 | + } |
| 462 | + } |
| 463 | + |
| 464 | + @Test |
| 465 | + public void testInputStreamReadTooLong() throws Throwable { |
| 466 | + describe("Attempting a read longer than the buffer should throw IndexOutOfBoundsException"); |
| 467 | + Path path = methodPath(); |
| 468 | + FileSystem fs = getFileSystem(); |
| 469 | + int len = 4096; |
| 470 | + createFile(fs, path, true, |
| 471 | + dataset(len, 0x40, 0x80)); |
| 472 | + try (FSDataInputStream is = fs.openFile(path).build().get()) { |
| 473 | + Assertions.assertThatThrownBy(() -> is.read(new byte[10], 0, 11)) |
| 474 | + .isInstanceOf(IndexOutOfBoundsException.class); |
| 475 | + } |
| 476 | + } |
| 477 | + |
| 478 | + @Test |
| 479 | + public void testInputStreamReadZeroLengthRead() throws Throwable { |
| 480 | + describe("Reading 0 bytes is a no-op"); |
| 481 | + Path path = methodPath(); |
| 482 | + FileSystem fs = getFileSystem(); |
| 483 | + int len = 4096; |
| 484 | + createFile(fs, path, true, |
| 485 | + dataset(len, 0x40, 0x80)); |
| 486 | + try (FSDataInputStream is = fs.openFile(path).build().get()) { |
| 487 | + Assertions.assertThat(is.read(new byte[10], 0, 0)).describedAs("bytes read").isEqualTo(0); |
| 488 | + } |
| 489 | + } |
| 490 | + |
| 491 | + @Test |
| 492 | + public void testInputStreamConsistentEOF() throws Throwable { |
| 493 | + describe("Both single-byte and multi-byte read should report EOF after consuming stream"); |
| 494 | + Path path = methodPath(); |
| 495 | + FileSystem fs = getFileSystem(); |
| 496 | + int len = 4096; |
| 497 | + createFile(fs, path, true, |
| 498 | + dataset(len, 0x40, 0x80)); |
| 499 | + try (FSDataInputStream is = fs.openFile(path).build().get()) { |
| 500 | + IOUtils.skipFully(is, len); |
| 501 | + Assertions.assertThat(is.read()).describedAs("single byte EOF").isEqualTo(-1); |
| 502 | + Assertions.assertThat(is.read(new byte[10], 0, 10)).describedAs("multi byte EOF") |
| 503 | + .isEqualTo(-1); |
| 504 | + } |
| 505 | + } |
| 506 | + |
| 507 | + @Test |
| 508 | + public void testInputStreamSingleAndMultiByteReadsEqual() throws Throwable { |
| 509 | + describe("Single-byte and multi-byte read should return the same bytes"); |
| 510 | + Path path = methodPath(); |
| 511 | + FileSystem fs = getFileSystem(); |
| 512 | + int len = 4096; |
| 513 | + createFile(fs, path, true, |
| 514 | + dataset(len, 0x40, 0x80)); |
| 515 | + byte[] multiByteReads = readDataset(fs, path, len); |
| 516 | + byte[] singleByteReads = readDatasetSingleByteReads(fs, path, len); |
| 517 | + compareByteArrays(multiByteReads, singleByteReads, len); |
| 518 | + } |
| 519 | + |
| 520 | + @Test |
| 521 | + public void testInputStreamMixedSingleAndMultiByteReadsEqual() throws Throwable { |
| 522 | + describe("Mixed single and multi-byte reads on the same stream should return the same bytes"); |
| 523 | + Path path = methodPath(); |
| 524 | + FileSystem fs = getFileSystem(); |
| 525 | + int len = 4096; |
| 526 | + createFile(fs, path, true, |
| 527 | + dataset(len, 0x40, 0x80)); |
| 528 | + byte[] expected = readDataset(fs, path, len); |
| 529 | + byte[] actual = new byte[len]; |
| 530 | + int readSize = 128; |
| 531 | + try (FSDataInputStream is = fs.openFile(path).build().get()) { |
| 532 | + for (int offset = 0; offset < len; offset = offset + readSize + readSize) { |
| 533 | + if (readNBytes(is, actual, offset, readSize) != readSize) { |
| 534 | + fail("End of file reached before reading fully."); |
| 535 | + } |
| 536 | + for (int i = 0; i < readSize; ++i) { |
| 537 | + int nextByte = is.read(); |
| 538 | + if (-1 == nextByte) { |
| 539 | + fail("End of file reached before reading fully."); |
| 540 | + } |
| 541 | + actual[offset + readSize + i] = (byte)nextByte; |
| 542 | + } |
| 543 | + } |
| 544 | + } |
| 545 | + compareByteArrays(expected, actual, len); |
| 546 | + } |
411 | 547 | }
|
0 commit comments