|
@@ -38,6 +38,9 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LE
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.compareByteArrays;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
|
|
|
+import static org.apache.hadoop.fs.contract.ContractTestUtils.readDataset;
|
|
|
+import static org.apache.hadoop.fs.contract.ContractTestUtils.readDatasetSingleByteReads;
|
|
|
+import static org.apache.hadoop.fs.contract.ContractTestUtils.readNBytes;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
|
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
|
|
@@ -408,4 +411,137 @@ public abstract class AbstractContractOpenTest
|
|
|
.isEqualTo(len);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testInputStreamReadNullBuffer() throws Throwable {
|
|
|
+ // The JDK base InputStream (and by extension LocalFSFileInputStream) throws
|
|
|
+ // NullPointerException. Historically, DFSInputStream has thrown IllegalArgumentException
|
|
|
+ // instead. Allow either behavior.
|
|
|
+ describe("Attempting to read into a null buffer should throw IllegalArgumentException or " +
|
|
|
+ "NullPointerException");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ try (FSDataInputStream is = fs.openFile(path).build().get()) {
|
|
|
+ Assertions.assertThatThrownBy(() -> is.read(null, 0, 10))
|
|
|
+ .isInstanceOfAny(IllegalArgumentException.class, NullPointerException.class);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInputStreamReadNegativePosition() throws Throwable {
|
|
|
+ describe("Attempting to read into a negative position should throw IndexOutOfBoundsException");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ try (FSDataInputStream is = fs.openFile(path).build().get()) {
|
|
|
+ Assertions.assertThatThrownBy(() -> is.read(new byte[10], -1, 10))
|
|
|
+ .isInstanceOf(IndexOutOfBoundsException.class);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInputStreamReadNegativeLength() throws Throwable {
|
|
|
+ // The JDK base InputStream (and by extension LocalFSFileInputStream) throws
|
|
|
+ // IndexOutOfBoundsException. Historically, DFSInputStream has thrown IllegalArgumentException
|
|
|
+ // instead. Allow either behavior.
|
|
|
+ describe("Attempting to read into a null buffer should throw IllegalArgumentException or " +
|
|
|
+ "IndexOutOfBoundsException");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ try (FSDataInputStream is = fs.openFile(path).build().get()) {
|
|
|
+ Assertions.assertThatThrownBy(() -> is.read(new byte[10], 0, -1))
|
|
|
+ .isInstanceOfAny(IllegalArgumentException.class, IndexOutOfBoundsException.class);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInputStreamReadTooLong() throws Throwable {
|
|
|
+ describe("Attempting a read longer than the buffer should throw IndexOutOfBoundsException");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ try (FSDataInputStream is = fs.openFile(path).build().get()) {
|
|
|
+ Assertions.assertThatThrownBy(() -> is.read(new byte[10], 0, 11))
|
|
|
+ .isInstanceOf(IndexOutOfBoundsException.class);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInputStreamReadZeroLengthRead() throws Throwable {
|
|
|
+ describe("Reading 0 bytes is a no-op");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ try (FSDataInputStream is = fs.openFile(path).build().get()) {
|
|
|
+ Assertions.assertThat(is.read(new byte[10], 0, 0)).describedAs("bytes read").isEqualTo(0);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInputStreamConsistentEOF() throws Throwable {
|
|
|
+ describe("Both single-byte and multi-byte read should report EOF after consuming stream");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ try (FSDataInputStream is = fs.openFile(path).build().get()) {
|
|
|
+ IOUtils.skipFully(is, len);
|
|
|
+ Assertions.assertThat(is.read()).describedAs("single byte EOF").isEqualTo(-1);
|
|
|
+ Assertions.assertThat(is.read(new byte[10], 0, 10)).describedAs("multi byte EOF")
|
|
|
+ .isEqualTo(-1);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInputStreamSingleAndMultiByteReadsEqual() throws Throwable {
|
|
|
+ describe("Single-byte and multi-byte read should return the same bytes");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ byte[] multiByteReads = readDataset(fs, path, len);
|
|
|
+ byte[] singleByteReads = readDatasetSingleByteReads(fs, path, len);
|
|
|
+ compareByteArrays(multiByteReads, singleByteReads, len);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInputStreamMixedSingleAndMultiByteReadsEqual() throws Throwable {
|
|
|
+ describe("Mixed single and multi-byte reads on the same stream should return the same bytes");
|
|
|
+ Path path = methodPath();
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ int len = 4096;
|
|
|
+ createFile(fs, path, true,
|
|
|
+ dataset(len, 0x40, 0x80));
|
|
|
+ byte[] expected = readDataset(fs, path, len);
|
|
|
+ byte[] actual = new byte[len];
|
|
|
+ int readSize = 128;
|
|
|
+ try (FSDataInputStream is = fs.openFile(path).build().get()) {
|
|
|
+ for (int offset = 0; offset < len; offset = offset + readSize + readSize) {
|
|
|
+ if (readNBytes(is, actual, offset, readSize) != readSize) {
|
|
|
+ fail("End of file reached before reading fully.");
|
|
|
+ }
|
|
|
+ for (int i = 0; i < readSize; ++i) {
|
|
|
+ int nextByte = is.read();
|
|
|
+ if (-1 == nextByte) {
|
|
|
+ fail("End of file reached before reading fully.");
|
|
|
+ }
|
|
|
+ actual[offset + readSize + i] = (byte)nextByte;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ compareByteArrays(expected, actual, len);
|
|
|
+ }
|
|
|
}
|