4343import static org .apache .hadoop .fs .Options .OpenFileOptions .FS_OPTION_OPENFILE_READ_POLICY_PARQUET ;
4444import static org .apache .hadoop .fs .Options .OpenFileOptions .FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE ;
4545import static org .apache .hadoop .fs .audit .AuditStatisticNames .AUDIT_REQUEST_EXECUTION ;
46+ import static org .apache .hadoop .fs .contract .ContractTestUtils .writeDataset ;
47+ import static org .apache .hadoop .fs .contract .ContractTestUtils .dataset ;
4648import static org .apache .hadoop .fs .s3a .Constants .ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX ;
4749import static org .apache .hadoop .fs .s3a .S3ATestUtils .enableAnalyticsAccelerator ;
4850import static org .apache .hadoop .fs .s3a .S3ATestUtils .removeBaseAndBucketOverrides ;
4951import static org .apache .hadoop .fs .s3a .test .PublicDatasetTestUtils .getExternalData ;
5052import static org .apache .hadoop .fs .statistics .IOStatisticAssertions .verifyStatisticCounterValue ;
53+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_BYTES ;
54+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_OPERATIONS ;
5155import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_ANALYTICS_OPENED ;
5256import static org .apache .hadoop .fs .statistics .StreamStatisticNames .ANALYTICS_STREAM_FACTORY_CLOSED ;
57+ import static org .apache .hadoop .io .Sizes .S_1K ;
58+ import static org .apache .hadoop .io .Sizes .S_1M ;
5359import static org .apache .hadoop .test .LambdaTestUtils .intercept ;
5460
5561/**
@@ -105,6 +111,13 @@ public void testConnectorFrameWorkIntegration() throws Throwable {
105111 Assertions .assertThat (objectInputStream .streamType ()).isEqualTo (InputStreamType .Analytics );
106112 Assertions .assertThat (objectInputStream .getInputPolicy ())
107113 .isEqualTo (S3AInputPolicy .Sequential );
114+
115+ verifyStatisticCounterValue (ioStats , STREAM_READ_BYTES , 500 );
116+ verifyStatisticCounterValue (ioStats , STREAM_READ_OPERATIONS , 1 );
117+
118+ long streamBytesRead = objectInputStream .getS3AStreamStatistics ().getBytesRead ();
119+ Assertions .assertThat (streamBytesRead ).as ("Stream statistics should track bytes read" )
120+ .isEqualTo (500 );
108121 }
109122
110123 verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
@@ -136,14 +149,24 @@ public void testMalformedParquetFooter() throws IOException {
136149
137150 byte [] buffer = new byte [500 ];
138151 IOStatistics ioStats ;
152+ int bytesRead ;
139153
140154 try (FSDataInputStream inputStream = getFileSystem ().open (dest )) {
141155 ioStats = inputStream .getIOStatistics ();
142156 inputStream .seek (5 );
143- inputStream .read (buffer , 0 , 500 );
157+ bytesRead = inputStream .read (buffer , 0 , 500 );
158+
159+ ObjectInputStream objectInputStream = (ObjectInputStream ) inputStream .getWrappedStream ();
160+ long streamBytesRead = objectInputStream .getS3AStreamStatistics ().getBytesRead ();
161+ Assertions .assertThat (streamBytesRead ).as ("Stream statistics should track bytes read" )
162+ .isEqualTo (bytesRead );
163+
144164 }
145165
146166 verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
167+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
168+ // S3A passes in the meta data on file open, we expect AAL to make no HEAD requests
169+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
147170 }
148171
149172 /**
@@ -173,17 +196,23 @@ public void testMultiRowGroupParquet() throws Throwable {
173196 }
174197
175198 verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
176-
199+ verifyStatisticCounterValue ( ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
177200 try (FSDataInputStream inputStream = getFileSystem ().openFile (dest )
178201 .must (FS_OPTION_OPENFILE_READ_POLICY , FS_OPTION_OPENFILE_READ_POLICY_PARQUET )
179202 .build ().get ()) {
180203 ioStats = inputStream .getIOStatistics ();
181204 inputStream .readFully (buffer , 0 , (int ) fileStatus .getLen ());
182- }
183205
206+ verifyStatisticCounterValue (ioStats , STREAM_READ_BYTES , (int ) fileStatus .getLen ());
207+ verifyStatisticCounterValue (ioStats , STREAM_READ_OPERATIONS , 1 );
208+ }
184209 verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
185210
186211 verifyStatisticCounterValue (getFileSystem ().getIOStatistics (), AUDIT_REQUEST_EXECUTION , 4 );
212+
213+ // S3A passes in the meta-data(content length) on file open,
214+ // we expect AAL to make no HEAD requests
215+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
187216 }
188217
189218 @ Test
@@ -203,4 +232,97 @@ public void testInvalidConfigurationThrows() throws Exception {
203232 () -> S3SeekableInputStreamConfiguration .fromConfiguration (connectorConfiguration ));
204233 }
205234
235+ /**
236+ * TXT files(SEQUENTIAL format) use SequentialPrefetcher(requests the entire 10MB file).
237+ * RangeOptimiser splits ranges larger than maxRangeSizeBytes (8MB) using partSizeBytes (8MB)
238+ * The 10MB range gets split into: [0-8MB) and [8MB-10MB)
239+ * Each split range becomes a separate Block, resulting in 2 GET requests:
240+ */
241+ @ Test
242+ public void testLargeFileMultipleGets () throws Throwable {
243+ describe ("Large file should trigger multiple GET requests" );
244+
245+ Path dest = path ("large-test-file.txt" );
246+ byte [] data = dataset (10 * S_1M , 256 , 255 );
247+ writeDataset (getFileSystem (), dest , data , 10 * S_1M , 1024 , true );
248+
249+ byte [] buffer = new byte [S_1M * 10 ];
250+ try (FSDataInputStream inputStream = getFileSystem ().open (dest )) {
251+ IOStatistics ioStats = inputStream .getIOStatistics ();
252+ inputStream .readFully (buffer );
253+
254+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 2 );
255+ // Because S3A passes in the meta-data(content length) on file open,
256+ // we expect AAL to make no HEAD requests
257+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
258+ }
259+ }
260+
261+ @ Test
262+ public void testSmallFileSingleGet () throws Throwable {
263+ describe ("Small file should trigger only one GET request" );
264+
265+ Path dest = path ("small-test-file.txt" );
266+ byte [] data = dataset (S_1M , 256 , 255 );
267+ writeDataset (getFileSystem (), dest , data , S_1M , 1024 , true );
268+
269+ byte [] buffer = new byte [S_1M ];
270+ try (FSDataInputStream inputStream = getFileSystem ().open (dest )) {
271+ IOStatistics ioStats = inputStream .getIOStatistics ();
272+ inputStream .readFully (buffer );
273+
274+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
275+ // Because S3A passes in the meta-data(content length) on file open,
276+ // we expect AAL to make no HEAD requests
277+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
278+ }
279+ }
280+
281+
282+ @ Test
283+ public void testRandomSeekPatternGets () throws Throwable {
284+ describe ("Random seek pattern should optimize GET requests" );
285+
286+ Path dest = path ("seek-test.txt" );
287+ byte [] data = dataset (5 * S_1M , 256 , 255 );
288+ writeDataset (getFileSystem (), dest , data , 5 * S_1M , 1024 , true );
289+
290+ byte [] buffer = new byte [S_1M ];
291+ try (FSDataInputStream inputStream = getFileSystem ().open (dest )) {
292+ IOStatistics ioStats = inputStream .getIOStatistics ();
293+
294+ inputStream .read (buffer );
295+ inputStream .seek (2 * S_1M );
296+ inputStream .read (new byte [512 * S_1K ]);
297+ inputStream .seek (3 * S_1M );
298+ inputStream .read (new byte [512 * S_1K ]);
299+
300+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
301+ verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
302+ }
303+ }
304+
305+
306+ @ Test
307+ public void testSequentialStreamsNoDuplicateGets () throws Throwable {
308+ describe ("Sequential streams reading same object should not duplicate GETs" );
309+
310+ Path dest = path ("sequential-test.txt" );
311+ byte [] data = dataset (S_1M , 256 , 255 );
312+ writeDataset (getFileSystem (), dest , data , S_1M , 1024 , true );
313+
314+ byte [] buffer = new byte [1024 ];
315+ try (FSDataInputStream stream1 = getFileSystem ().open (dest );
316+ FSDataInputStream stream2 = getFileSystem ().open (dest )) {
317+
318+ stream1 .read (buffer );
319+ stream2 .read (buffer );
320+
321+ IOStatistics stats1 = stream1 .getIOStatistics ();
322+ IOStatistics stats2 = stream2 .getIOStatistics ();
323+
324+ verifyStatisticCounterValue (stats1 , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
325+ verifyStatisticCounterValue (stats2 , STREAM_READ_ANALYTICS_GET_REQUESTS , 0 );
326+ }
327+ }
206328}
0 commit comments