4242import  static  org .apache .hadoop .fs .Options .OpenFileOptions .FS_OPTION_OPENFILE_READ_POLICY ;
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 ;
45+ import  static  org .apache .hadoop .fs .contract .ContractTestUtils .writeDataset ;
46+ import  static  org .apache .hadoop .fs .contract .ContractTestUtils .dataset ;
4547import  static  org .apache .hadoop .fs .s3a .Constants .ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX ;
4648import  static  org .apache .hadoop .fs .s3a .S3ATestUtils .enableAnalyticsAccelerator ;
4749import  static  org .apache .hadoop .fs .s3a .S3ATestUtils .removeBaseAndBucketOverrides ;
4850import  static  org .apache .hadoop .fs .s3a .test .PublicDatasetTestUtils .getExternalData ;
4951import  static  org .apache .hadoop .fs .statistics .IOStatisticAssertions .verifyStatisticCounterValue ;
52+ 
53+ import  static  org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_BYTES ;
54+ import  static  org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_OPERATIONS ;
55+ import  static  org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_ANALYTICS_GET_REQUESTS ;
56+ import  static  org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_ANALYTICS_HEAD_REQUESTS ;
5057import  static  org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_ANALYTICS_OPENED ;
5158import  static  org .apache .hadoop .fs .statistics .StreamStatisticNames .ANALYTICS_STREAM_FACTORY_CLOSED ;
59+ import  static  org .apache .hadoop .io .Sizes .S_1K ;
60+ import  static  org .apache .hadoop .io .Sizes .S_1M ;
5261import  static  org .apache .hadoop .test .LambdaTestUtils .intercept ;
5362
5463/** 
@@ -104,6 +113,13 @@ public void testConnectorFrameWorkIntegration() throws Throwable {
104113      Assertions .assertThat (objectInputStream .streamType ()).isEqualTo (InputStreamType .Analytics );
105114      Assertions .assertThat (objectInputStream .getInputPolicy ())
106115          .isEqualTo (S3AInputPolicy .Sequential );
116+ 
117+       verifyStatisticCounterValue (ioStats , STREAM_READ_BYTES , 500 );
118+       verifyStatisticCounterValue (ioStats , STREAM_READ_OPERATIONS , 1 );
119+ 
120+       long  streamBytesRead  = objectInputStream .getS3AStreamStatistics ().getBytesRead ();
121+       Assertions .assertThat (streamBytesRead ).as ("Stream statistics should track bytes read" )
122+               .isEqualTo (500 );
107123    }
108124
109125    verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
@@ -129,14 +145,24 @@ public void testMalformedParquetFooter() throws IOException {
129145
130146    byte [] buffer  = new  byte [500 ];
131147    IOStatistics  ioStats ;
148+     int  bytesRead ;
132149
133150    try  (FSDataInputStream  inputStream  = getFileSystem ().open (dest )) {
134151      ioStats  = inputStream .getIOStatistics ();
135152      inputStream .seek (5 );
136-       inputStream .read (buffer , 0 , 500 );
153+       bytesRead  = inputStream .read (buffer , 0 , 500 );
154+ 
155+       ObjectInputStream  objectInputStream  = (ObjectInputStream ) inputStream .getWrappedStream ();
156+       long  streamBytesRead  = objectInputStream .getS3AStreamStatistics ().getBytesRead ();
157+       Assertions .assertThat (streamBytesRead ).as ("Stream statistics should track bytes read" )
158+               .isEqualTo (bytesRead );
159+ 
137160    }
138161
139162    verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
163+     verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
164+     // S3A passes in the meta data on file open, we expect AAL to make no HEAD requests 
165+     verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
140166  }
141167
142168  /** 
@@ -166,15 +192,20 @@ public void testMultiRowGroupParquet() throws Throwable {
166192    }
167193
168194    verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
169- 
195+      verifyStatisticCounterValue ( ioStats ,  STREAM_READ_ANALYTICS_GET_REQUESTS ,  1 ); 
170196    try  (FSDataInputStream  inputStream  = getFileSystem ().openFile (dest )
171197        .must (FS_OPTION_OPENFILE_READ_POLICY , FS_OPTION_OPENFILE_READ_POLICY_PARQUET )
172198        .build ().get ()) {
173199      ioStats  = inputStream .getIOStatistics ();
174200      inputStream .readFully (buffer , 0 , (int ) fileStatus .getLen ());
175-     }
176201
202+       verifyStatisticCounterValue (ioStats , STREAM_READ_BYTES , (int ) fileStatus .getLen ());
203+       verifyStatisticCounterValue (ioStats , STREAM_READ_OPERATIONS , 1 );
204+     }
177205    verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
206+     // S3A passes in the meta-data(content length) on file open, 
207+     // we expect AAL to make no HEAD requests 
208+     verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
178209  }
179210
180211  @ Test 
@@ -194,4 +225,97 @@ public void testInvalidConfigurationThrows() throws Exception {
194225        () -> S3SeekableInputStreamConfiguration .fromConfiguration (connectorConfiguration ));
195226  }
196227
228+   /** 
229+    * TXT files(SEQUENTIAL format) use SequentialPrefetcher(requests the entire 10MB file). 
230+    * RangeOptimiser splits ranges larger than maxRangeSizeBytes (8MB) using partSizeBytes (8MB) 
231+    * The 10MB range gets split into: [0-8MB) and [8MB-10MB) 
232+    * Each split range becomes a separate Block, resulting in 2 GET requests: 
233+    */ 
234+   @ Test 
235+   public  void  testLargeFileMultipleGets () throws  Throwable  {
236+     describe ("Large file should trigger multiple GET requests" );
237+ 
238+     Path  dest  = path ("large-test-file.txt" );
239+     byte [] data  = dataset (10  * S_1M , 256 , 255 );
240+     writeDataset (getFileSystem (), dest , data , 10  * S_1M , 1024 , true );
241+ 
242+     byte [] buffer  = new  byte [S_1M  * 10 ];
243+     try  (FSDataInputStream  inputStream  = getFileSystem ().open (dest )) {
244+       IOStatistics  ioStats  = inputStream .getIOStatistics ();
245+       inputStream .readFully (buffer );
246+ 
247+       verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 2 );
248+       // Because S3A passes in the meta-data(content length) on file open, 
249+       // we expect AAL to make no HEAD requests 
250+       verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
251+     }
252+   }
253+ 
254+   @ Test 
255+   public  void  testSmallFileSingleGet () throws  Throwable  {
256+     describe ("Small file should trigger only one GET request" );
257+ 
258+     Path  dest  = path ("small-test-file.txt" );
259+     byte [] data  = dataset (S_1M , 256 , 255 );
260+     writeDataset (getFileSystem (), dest , data , S_1M , 1024 , true );
261+ 
262+     byte [] buffer  = new  byte [S_1M ];
263+     try  (FSDataInputStream  inputStream  = getFileSystem ().open (dest )) {
264+       IOStatistics  ioStats  = inputStream .getIOStatistics ();
265+       inputStream .readFully (buffer );
266+ 
267+       verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
268+       // Because S3A passes in the meta-data(content length) on file open, 
269+       // we expect AAL to make no HEAD requests 
270+       verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
271+     }
272+   }
273+ 
274+ 
275+   @ Test 
276+   public  void  testRandomSeekPatternGets () throws  Throwable  {
277+     describe ("Random seek pattern should optimize GET requests" );
278+ 
279+     Path  dest  = path ("seek-test.txt" );
280+     byte [] data  = dataset (5  * S_1M , 256 , 255 );
281+     writeDataset (getFileSystem (), dest , data , 5  * S_1M , 1024 , true );
282+ 
283+     byte [] buffer  = new  byte [S_1M ];
284+     try  (FSDataInputStream  inputStream  = getFileSystem ().open (dest )) {
285+       IOStatistics  ioStats  = inputStream .getIOStatistics ();
286+ 
287+       inputStream .read (buffer );
288+       inputStream .seek (2  * S_1M );
289+       inputStream .read (new  byte [512  * S_1K ]);
290+       inputStream .seek (3  * S_1M );
291+       inputStream .read (new  byte [512  * S_1K ]);
292+ 
293+       verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
294+       verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_HEAD_REQUESTS , 0 );
295+     }
296+   }
297+ 
298+ 
299+   @ Test 
300+   public  void  testSequentialStreamsNoDuplicateGets () throws  Throwable  {
301+     describe ("Sequential streams reading same object should not duplicate GETs" );
302+ 
303+     Path  dest  = path ("sequential-test.txt" );
304+     byte [] data  = dataset (S_1M , 256 , 255 );
305+     writeDataset (getFileSystem (), dest , data , S_1M , 1024 , true );
306+ 
307+     byte [] buffer  = new  byte [1024 ];
308+     try  (FSDataInputStream  stream1  = getFileSystem ().open (dest );
309+          FSDataInputStream  stream2  = getFileSystem ().open (dest )) {
310+ 
311+       stream1 .read (buffer );
312+       stream2 .read (buffer );
313+ 
314+       IOStatistics  stats1  = stream1 .getIOStatistics ();
315+       IOStatistics  stats2  = stream2 .getIOStatistics ();
316+ 
317+       verifyStatisticCounterValue (stats1 , STREAM_READ_ANALYTICS_GET_REQUESTS , 1 );
318+       verifyStatisticCounterValue (stats2 , STREAM_READ_ANALYTICS_GET_REQUESTS , 0 );
319+     }
320+   }
197321}
0 commit comments