[ https://issues.apache.org/jira/browse/HADOOP-18189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17619018#comment-17619018 ]
ASF GitHub Bot commented on HADOOP-18189: ----------------------------------------- steveloughran commented on code in PR #5036: URL: https://github.com/apache/hadoop/pull/5036#discussion_r997325307 ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java: ########## @@ -240,4 +242,56 @@ public void testRandomReadSmallFile() throws Throwable { } } + @Test + public void testStatusProbesAfterClosingStream() throws Throwable { + describe("When the underlying input stream is closed, the prefetch input stream" + + " should still support some status probes"); + + byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26); + Path smallFile = path("testStatusProbesAfterClosingStream"); Review Comment: you can just use `path()` and one is built up from the method name. this is better as it avoids cut and paste bugs *and* if a build is parameterized, the parameterized string is used in the path (though tests fail if that string isn't a valid path any more) ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java: ########## @@ -240,4 +242,56 @@ public void testRandomReadSmallFile() throws Throwable { } } + @Test + public void testStatusProbesAfterClosingStream() throws Throwable { + describe("When the underlying input stream is closed, the prefetch input stream" + + " should still support some status probes"); + + byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26); + Path smallFile = path("testStatusProbesAfterClosingStream"); + ContractTestUtils.writeDataset(getFileSystem(), smallFile, data, data.length, 16, true); + + FSDataInputStream in = getFileSystem().open(smallFile); + + byte[] buffer = new byte[SMALL_FILE_SIZE]; + in.read(buffer, 0, S_1K * 4); + in.seek(S_1K * 12); + in.read(buffer, 0, S_1K * 4); + + long pos = in.getPos(); + IOStatistics ioStats = in.getIOStatistics(); + S3AInputStreamStatistics inputStreamStatistics = + ((S3APrefetchingInputStream) (in.getWrappedStream())).getS3AStreamStatistics(); + + assertNotNull("Prefetching input IO stats should not be null", ioStats); + assertNotNull("Prefetching input stream stats should not be null", inputStreamStatistics); + assertNotEquals("Position retrieved from prefetching input stream should be greater than 0", 0, + pos); + + in.close(); + + // status probes after closing the input stream + long newPos = in.getPos(); + IOStatistics newIoStats = in.getIOStatistics(); + S3AInputStreamStatistics newInputStreamStatistics = + ((S3APrefetchingInputStream) (in.getWrappedStream())).getS3AStreamStatistics(); + + assertNotNull("Prefetching input IO stats should not be null", newIoStats); + assertNotNull("Prefetching input stream stats should not be null", newInputStreamStatistics); + assertNotEquals("Position retrieved from prefetching input stream should be greater than 0", 0, + newPos); + + // compare status probes after closing of the stream with status probes done before + // closing the stream + assertEquals("Position retrieved through stream before and after closing should match", pos, + newPos); + assertEquals("IO stats retrieved through stream before and after closing should match", ioStats, + newIoStats); + assertEquals("Stream stats retrieved through stream before and after closing should match", + inputStreamStatistics, newInputStreamStatistics); + + assertFalse("Not supported with prefetch", in.seekToNewSource(10)); Review Comment: nit: can you have the error message include "seekToNewSource()". > S3PrefetchingInputStream to support status probes when closed > ------------------------------------------------------------- > > Key: HADOOP-18189 > URL: https://issues.apache.org/jira/browse/HADOOP-18189 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 > Affects Versions: 3.4.0 > Reporter: Steve Loughran > Assignee: Viraj Jasani > Priority: Minor > Labels: pull-request-available > > S3PrefetchingInputStream is a bit over aggressive on raising > exceptions/downgrading responses after a stream is closed > * MUST: getPos() to return last read location, or a least 0 (maybe we should > add this to filesystem spec) > * MUST: getIOStatistics(). critical for collecting stats in processes > * MUST: seekToNewSource() (it's a no op anyway) > * MAY S3AInputStreamStatistics and getStreamStatistics() though that is only > in used in testing...ioStatistics have given us a stable stats api. we may > want to tag the method @VisibleForTesting to discourage use. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org