ahmarsuhail commented on code in PR #7763: URL: https://github.com/apache/hadoop/pull/7763#discussion_r2177594761
########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAnalyticsAcceleratorStreamReading.java: ########## @@ -194,4 +215,127 @@ public void testInvalidConfigurationThrows() throws Exception { () -> S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); } + @Test + public void testLargeFileMultipleGets() throws Throwable { + describe("Large file should trigger multiple GET requests"); + + Path dest = writeThenReadFile("large-test-file.txt", 10 * 1024 * 1024); // 10MB + + + try (FSDataInputStream inputStream = getFileSystem().open(dest)) { + IOStatistics ioStats = inputStream.getIOStatistics(); + inputStream.readFully(new byte[(int) getFileSystem().getFileStatus(dest).getLen()]); + + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_GET_REQUESTS, 2); + } + } + + @Test + public void testSmallFileSingleGet() throws Throwable { + describe("Small file should trigger only one GET request"); + + Path dest = writeThenReadFile("small-test-file.txt", 1 * 1024 * 1024); // 1KB + + try (FSDataInputStream inputStream = getFileSystem().open(dest)) { + IOStatistics ioStats = inputStream.getIOStatistics(); + inputStream.readFully(new byte[(int) getFileSystem().getFileStatus(dest).getLen()]); + + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_GET_REQUESTS, 1); + } + } + + + @Test + public void testRandomSeekPatternGets() throws Throwable { + describe("Random seek pattern should optimize GET requests"); + + Path dest = writeThenReadFile("seek-test.txt", 100 * 1024); + + try (FSDataInputStream inputStream = getFileSystem().open(dest)) { + IOStatistics ioStats = inputStream.getIOStatistics(); + + inputStream.seek(1000); + inputStream.read(new byte[100]); + + inputStream.seek(50000); + inputStream.read(new byte[100]); + + inputStream.seek(90000); + inputStream.read(new byte[100]); + + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_GET_REQUESTS, 1); + } +} + + @Test + public void testAALNeverMakesHeadRequests() throws Throwable { + describe("Prove AAL never makes HEAD requests - S3A provides all metadata"); + + Path dest = writeThenReadFile("no-head-test.txt", 1024 * 1024); // 1MB + + try (FSDataInputStream inputStream = getFileSystem().open(dest)) { + IOStatistics ioStats = inputStream.getIOStatistics(); + inputStream.read(new byte[1024]); + + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_HEAD_REQUESTS, 0); + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_OPENED, 1); + + ObjectInputStream objectInputStream = (ObjectInputStream) inputStream.getWrappedStream(); + Assertions.assertThat(objectInputStream.streamType()).isEqualTo(InputStreamType.Analytics); + + } + } + + + @Test + public void testParquetReadingNoHeadRequests() throws Throwable { Review Comment: you don't need this test, just assert 0 head requests in the previous tests -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org