[
https://issues.apache.org/jira/browse/HDFS-15042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17744709#comment-17744709
]
ASF GitHub Bot commented on HDFS-15042:
---------------------------------------
steveloughran commented on code in PR #1747:
URL: https://github.com/apache/hadoop/pull/1747#discussion_r1268335189
##########
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteBufferPread.java:
##########
@@ -161,130 +229,264 @@ private void testPreadWithFullByteBuffer(ByteBuffer
buffer)
* {@link ByteBuffer#limit()} on the buffer. Validates that only half of the
* testFile is loaded into the buffer.
*/
- private void testPreadWithLimitedByteBuffer(
- ByteBuffer buffer) throws IOException {
+ @Test
+ public void testPreadWithLimitedByteBuffer() throws IOException {
int bytesRead;
int totalBytesRead = 0;
// Set the buffer limit to half the size of the file
- buffer.limit(FILE_SIZE / 2);
+ buffer.limit(HALF_SIZE);
try (FSDataInputStream in = fs.open(testFile)) {
+ in.seek(EOF_POS);
while ((bytesRead = in.read(totalBytesRead, buffer)) > 0) {
totalBytesRead += bytesRead;
// Check that each call to read changes the position of the ByteBuffer
// correctly
- assertEquals(totalBytesRead, buffer.position());
+ assertBufferPosition(totalBytesRead);
}
// Since we set the buffer limit to half the size of the file, we should
// have only read half of the file into the buffer
- assertEquals(totalBytesRead, FILE_SIZE / 2);
+ assertEquals(HALF_SIZE, totalBytesRead);
// Check that the buffer is full and the contents equal the first half of
// the file
- assertFalse(buffer.hasRemaining());
- buffer.position(0);
- byte[] bufferContents = new byte[FILE_SIZE / 2];
- buffer.get(bufferContents);
- assertArrayEquals(bufferContents,
- Arrays.copyOfRange(fileContents, 0, FILE_SIZE / 2));
+ assertBufferIsFull();
+ assertBufferEqualsFileContents(0, HALF_SIZE, 0);
+
+ // position hasn't changed
+ assertStreamPosition(in, EOF_POS);
}
}
/**
* Reads half of the testFile into the {@link ByteBuffer} by setting the
* {@link ByteBuffer#position()} the half the size of the file. Validates
that
* only half of the testFile is loaded into the buffer.
+ * <p>
+ * This test interleaves reading from the stream by the classic input
+ * stream API, verifying those bytes are also as expected.
+ * This lets us validate the requirement that these positions reads must
+ * not interfere with the conventional read sequence.
*/
- private void testPreadWithPositionedByteBuffer(
- ByteBuffer buffer) throws IOException {
+ @Test
+ public void testPreadWithPositionedByteBuffer() throws IOException {
int bytesRead;
int totalBytesRead = 0;
// Set the buffer position to half the size of the file
- buffer.position(FILE_SIZE / 2);
+ buffer.position(HALF_SIZE);
+ int counter = 0;
try (FSDataInputStream in = fs.open(testFile)) {
+ assertEquals("Byte read from stream",
+ fileContents[counter++], in.read());
while ((bytesRead = in.read(totalBytesRead, buffer)) > 0) {
totalBytesRead += bytesRead;
// Check that each call to read changes the position of the ByteBuffer
// correctly
- assertEquals(totalBytesRead + FILE_SIZE / 2, buffer.position());
+ assertBufferPosition(totalBytesRead + HALF_SIZE);
+ // read the next byte.
+ assertEquals("Byte read from stream",
+ fileContents[counter++], in.read());
}
// Since we set the buffer position to half the size of the file, we
// should have only read half of the file into the buffer
- assertEquals(totalBytesRead, FILE_SIZE / 2);
+ assertEquals("bytes read",
+ HALF_SIZE, totalBytesRead);
// Check that the buffer is full and the contents equal the first half of
// the file
- assertFalse(buffer.hasRemaining());
- buffer.position(FILE_SIZE / 2);
- byte[] bufferContents = new byte[FILE_SIZE / 2];
- buffer.get(bufferContents);
- assertArrayEquals(bufferContents,
- Arrays.copyOfRange(fileContents, 0, FILE_SIZE / 2));
+ assertBufferIsFull();
+ assertBufferEqualsFileContents(HALF_SIZE, HALF_SIZE, 0);
}
}
+ /**
+ * Assert the buffer ranges matches that in the file.
+ * @param bufferPosition buffer position
+ * @param length length of data to check
+ * @param fileOffset offset in file.
+ */
+ private void assertBufferEqualsFileContents(int bufferPosition,
+ int length,
+ int fileOffset) {
+ buffer.position(bufferPosition);
+ byte[] bufferContents = new byte[length];
+ buffer.get(bufferContents);
+ assertArrayEquals(
+ "Buffer data from [" + bufferPosition + "-" + length + "]",
+ bufferContents,
+ Arrays.copyOfRange(fileContents, fileOffset, fileOffset + length));
+ }
+
/**
* Reads half of the testFile into the {@link ByteBuffer} by specifying a
* position for the pread API that is half of the file size. Validates that
* only half of the testFile is loaded into the buffer.
*/
- private void testPositionedPreadWithByteBuffer(
- ByteBuffer buffer) throws IOException {
+ @Test
+ public void testPositionedPreadWithByteBuffer() throws IOException {
int bytesRead;
int totalBytesRead = 0;
try (FSDataInputStream in = fs.open(testFile)) {
// Start reading from halfway through the file
- while ((bytesRead = in.read(totalBytesRead + FILE_SIZE / 2,
+ while ((bytesRead = in.read(totalBytesRead + HALF_SIZE,
buffer)) > 0) {
totalBytesRead += bytesRead;
// Check that each call to read changes the position of the ByteBuffer
// correctly
- assertEquals(totalBytesRead, buffer.position());
+ assertBufferPosition(totalBytesRead);
}
// Since we starting reading halfway through the file, the buffer should
// only be half full
- assertEquals(totalBytesRead, FILE_SIZE / 2);
- assertEquals(buffer.position(), FILE_SIZE / 2);
- assertTrue(buffer.hasRemaining());
+ assertEquals("bytes read", HALF_SIZE, totalBytesRead);
+ assertBufferPosition(HALF_SIZE);
+ assertBufferIsNotFull();
// Check that the buffer contents equal the second half of the file
- buffer.position(0);
- byte[] bufferContents = new byte[FILE_SIZE / 2];
- buffer.get(bufferContents);
- assertArrayEquals(bufferContents,
- Arrays.copyOfRange(fileContents, FILE_SIZE / 2, FILE_SIZE));
+ assertBufferEqualsFileContents(0, HALF_SIZE, HALF_SIZE);
}
}
/**
* Reads the entire testFile using the preadFully API and validates that its
- * contents are properly loaded into the supplied {@link ByteBuffer}.
+ * contents are properly loaded into the {@link ByteBuffer}.
*/
- private void testPreadFullyWithByteBuffer(ByteBuffer buffer)
- throws IOException {
+ @Test
+ public void testPreadFullyWithByteBuffer() throws IOException {
int totalBytesRead = 0;
try (FSDataInputStream in = fs.open(testFile)) {
in.readFully(totalBytesRead, buffer);
// Make sure the buffer is full
- assertFalse(buffer.hasRemaining());
+ assertBufferIsFull();
// Make sure the contents of the read buffer equal the contents of the
// file
- buffer.position(0);
- byte[] bufferContents = new byte[FILE_SIZE];
- buffer.get(bufferContents);
- assertArrayEquals(bufferContents, fileContents);
+ assertBufferEqualsFileContents(0, FILE_SIZE, 0);
+ }
+ }
+
+ /**
+ * readFully past the end of the file into an empty buffer; expect this
+ * to be a no-op.
+ */
+ @Test
+ public void testPreadFullyPastEOFEmptyByteBuffer() throws IOException {
+ try (FSDataInputStream in = fs.open(testFile)) {
+ in.readFully(FILE_SIZE + 10, emptyBuffer);
+ }
+ }
+
+ /**
+ * Reads from a negative position -expects a failure.
+ * Also uses the new openFile() API to improve its coverage.
+ */
+ @Test
+ public void testPreadFullyNegativeOffset() throws Exception {
+ try (FSDataInputStream in = fs.openFile(testFile).build().get()) {
+ in.seek(QUARTER_SIZE);
+ intercept(EOFException.class, NEGATIVE_POSITION_READ,
+ () -> in.readFully(-1, buffer));
+ // the stream position has not changed.
+ assertStreamPosition(in, QUARTER_SIZE);
+ }
+ }
+
+ /**
+ * Read fully with a start position past the EOF -expects a failure.
+ */
+ @Test
+ public void testPreadFullyPositionPastEOF() throws Exception {
+ try (FSDataInputStream in = fs.openFile(testFile).build().get()) {
+ in.seek(QUARTER_SIZE);
+ intercept(EOFException.class, EOF_IN_READ_FULLY,
+ () -> in.readFully(FILE_SIZE * 2, buffer));
+ // the stream position has not changed.
+ assertStreamPosition(in, QUARTER_SIZE);
+ }
+ }
+
+ /**
+ * Read which goes past the EOF; expects a failure.
+ * The final state of the buffer is undefined; it may fail fast or fail late.
+ * Also uses the new openFile() API to improve its coverage.
+ */
+ @Test
+ public void testPreadFullySpansEOF() throws Exception {
+ try (FSDataInputStream in = fs.openFile(testFile).build().get()) {
+ intercept(EOFException.class, EOF_IN_READ_FULLY,
+ () -> in.readFully(FILE_SIZE - 10, buffer));
+ if (buffer.position() > 0) {
+ // this implementation does not do a range check before the read;
+ // it got partway through before failing.
+ // this is not an error -just inefficient.
+ LOG.warn("Buffer reads began before range checks with {}", in);
Review Comment:
i really don't remember. this is a pr from 2019.
> Add more tests for ByteBufferPositionedReadable
> ------------------------------------------------
>
> Key: HDFS-15042
> URL: https://issues.apache.org/jira/browse/HDFS-15042
> Project: Hadoop HDFS
> Issue Type: Improvement
> Components: fs, test
> Affects Versions: 3.3.0
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Major
> Labels: pull-request-available
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
> There's a few corner cases of ByteBufferPositionedReadable which need to be
> tested, mainly illegal read positions. Add them
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]