5252import static org .apache .hadoop .fs .statistics .IOStatisticAssertions .verifyStatisticCounterValue ;
5353import static org .apache .hadoop .fs .statistics .StoreStatisticNames .ACTION_HTTP_GET_REQUEST ;
5454import static org .apache .hadoop .fs .statistics .StoreStatisticNames .ACTION_HTTP_HEAD_REQUEST ;
55- import static org .apache .hadoop .fs .statistics .StreamStatisticNames .*;
55+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .ANALYTICS_STREAM_FACTORY_CLOSED ;
56+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_ANALYTICS_OPENED ;
57+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_BYTES ;
58+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_CACHE_HIT ;
59+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_OPERATIONS ;
60+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_PARQUET_FOOTER_PARSING_FAILED ;
61+ import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_PREFETCHED_BYTES ;
5662import static org .apache .hadoop .io .Sizes .S_1K ;
5763import static org .apache .hadoop .io .Sizes .S_1M ;
5864import static org .apache .hadoop .test .LambdaTestUtils .intercept ;
@@ -95,6 +101,12 @@ public void testConnectorFrameWorkIntegration() throws Throwable {
95101
96102 S3AFileSystem fs =
97103 (S3AFileSystem ) FileSystem .get (externalTestFile .toUri (), getConfiguration ());
104+
105+ long fileLength = fs .getFileStatus (externalTestFile ).getLen ();
106+
107+ // Head request for the file length.
108+ verifyStatisticCounterValue (fs .getIOStatistics (), AUDIT_REQUEST_EXECUTION , 1 );
109+
98110 byte [] buffer = new byte [500 ];
99111 IOStatistics ioStats ;
100112
@@ -123,9 +135,9 @@ public void testConnectorFrameWorkIntegration() throws Throwable {
123135
124136 verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
125137
126- // Total file size is: 21511173, and read starts from pos 5. Since policy is WHOLE_FILE, the whole file starts
127- // getting prefetched as soon as the stream to it is opened. So prefetched bytes is 21511173 - 5 = 21511168
128- verifyStatisticCounterValue (ioStats , STREAM_READ_PREFETCHED_BYTES , 21511168 );
138+ // Since policy is WHOLE_FILE, the whole file starts getting prefetched as soon as the stream to it is opened.
139+ // So prefetched bytes is fileLen - 5
140+ verifyStatisticCounterValue (ioStats , STREAM_READ_PREFETCHED_BYTES , fileLength - 5 );
129141
130142 fs .close ();
131143 verifyStatisticCounterValue (fs .getIOStatistics (), ANALYTICS_STREAM_FACTORY_CLOSED , 1 );
@@ -134,7 +146,7 @@ public void testConnectorFrameWorkIntegration() throws Throwable {
134146 // in which case, AAL will start prefetching till EoF on file open in 8MB chunks. The file read here
135147 // s3://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz, has a size of ~21MB, resulting in 3 GETS:
136148 // [0-8388607, 8388608-16777215, 16777216-21511173].
137- verifyStatisticCounterValue (fs .getIOStatistics (), AUDIT_REQUEST_EXECUTION , 4 );
149+ verifyStatisticCounterValue (fs .getIOStatistics (), AUDIT_REQUEST_EXECUTION , 5 );
138150 }
139151
140152 @ Test
@@ -146,13 +158,15 @@ public void testSequentialPrefetching() throws IOException {
146158 // While this works well when running on EC2, for local testing, it can take more than 1s to download large chunks
147159 // of data. Set this value to higher for testing to prevent early cache evictions.
148160 conf .setInt (ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX +
149- "." + "physicalio.cache.timeout" , 10000 );
161+ "." + AAL_CACHE_TIMEOUT , 10000 );
150162
151163 S3AFileSystem fs =
152164 (S3AFileSystem ) FileSystem .get (externalTestFile .toUri (), getConfiguration ());
153165 byte [] buffer = new byte [10 * ONE_MB ];
154166 IOStatistics ioStats ;
155167
168+ long fileLength = fs .getFileStatus (externalTestFile ).getLen ();
169+
156170 // Here we read through the 21MB external test file, but do not pass in the WHOLE_FILE policy. Instead, we rely
157171 // on AAL detecting a sequential pattern being read, and then prefetching bytes in a geometrical progression.
158172 // AAL's sequential prefetching starts prefetching in increments 4MB, 8MB, 16MB etc. depending on how many
@@ -181,20 +195,19 @@ public void testSequentialPrefetching() throws IOException {
181195 verifyStatisticCounterValue (ioStats , STREAM_READ_CACHE_HIT , 2 );
182196 // A total of 10MB is prefetched - 3MB and then 7MB.
183197 verifyStatisticCounterValue (ioStats , STREAM_READ_PREFETCHED_BYTES , 10 * ONE_MB );
184-
198+ long bytesRemainingForPrefetch = fileLength - ( inputStream . getPos () + 10 * ONE_MB );
185199 inputStream .readFully (buffer , 0 , 10 * ONE_MB );
186200
187- // Though the next GP should prefetch 16MB, since the file is ~23MB, only the bytes till EoF are prefetched:
188- // 6291456 remaining bytes.
189- verifyStatisticCounterValue (ioStats , STREAM_READ_PREFETCHED_BYTES , 10 * ONE_MB + 6291456 );
201+
202+ // Though the next GP should prefetch 16MB, since the file is ~23MB, only the bytes till EoF are prefetched .
203+ verifyStatisticCounterValue (ioStats , STREAM_READ_PREFETCHED_BYTES , 10 * ONE_MB + bytesRemainingForPrefetch );
190204 inputStream .readFully (buffer , 0 , 3 * ONE_MB );
191205 verifyStatisticCounterValue (ioStats , STREAM_READ_CACHE_HIT , 3 );
192206 }
193207
194208 // verify all AAL stats are passed to the FS.
195209 verifyStatisticCounterValue (fs .getIOStatistics (), STREAM_READ_CACHE_HIT , 3 );
196210 verifyStatisticCounterValue (fs .getIOStatistics (), STREAM_READ_PARQUET_FOOTER_PARSING_FAILED , 0 );
197- verifyStatisticCounterValue (fs .getIOStatistics (), STREAM_READ_PREFETCHED_BYTES , 10 * ONE_MB + 6291456 );
198211 }
199212
200213 @ Test
@@ -213,6 +226,8 @@ public void testMalformedParquetFooter() throws IOException {
213226 Path sourcePath = new Path (file .toURI ().getPath ());
214227 getFileSystem ().copyFromLocalFile (false , true , sourcePath , dest );
215228
229+ long fileLength = getFileSystem ().getFileStatus (dest ).getLen ();
230+
216231 byte [] buffer = new byte [500 ];
217232 IOStatistics ioStats ;
218233 int bytesRead ;
@@ -232,9 +247,9 @@ public void testMalformedParquetFooter() throws IOException {
232247 verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
233248 verifyStatisticCounterValue (ioStats , ACTION_HTTP_GET_REQUEST , 1 );
234249 verifyStatisticCounterValue (ioStats , ACTION_HTTP_HEAD_REQUEST , 0 );
235- // This file has a content length of 450 . Since it's a parquet file, AAL will prefetch the footer bytes (last 32KB),
250+ // This file has a content length of 451 . Since it's a parquet file, AAL will prefetch the footer bytes (last 32KB),
236251 // as soon as the file is opened, but because the file is < 32KB, the whole file is prefetched.
237- verifyStatisticCounterValue (ioStats , STREAM_READ_PREFETCHED_BYTES , 450 );
252+ verifyStatisticCounterValue (ioStats , STREAM_READ_PREFETCHED_BYTES , fileLength );
238253
239254 // The footer is only prefetched once, but parsing is attempted on each stream open.
240255 verifyStatisticCounterValue (ioStats , STREAM_READ_PARQUET_FOOTER_PARSING_FAILED , 1 );
@@ -367,8 +382,10 @@ public void testSequentialStreamsNoDuplicateGets() throws Throwable {
367382 describe ("Sequential streams reading same object should not duplicate GETs" );
368383
369384 Path dest = path ("sequential-test.txt" );
370- byte [] data = dataset (S_1M , 256 , 255 );
371- writeDataset (getFileSystem (), dest , data , S_1M , 1024 , true );
385+ int fileLen = S_1M ;
386+
387+ byte [] data = dataset (fileLen , 256 , 255 );
388+ writeDataset (getFileSystem (), dest , data , fileLen , 1024 , true );
372389
373390 byte [] buffer = new byte [ONE_MB ];
374391 try (FSDataInputStream stream1 = getFileSystem ().open (dest );
@@ -386,14 +403,14 @@ public void testSequentialStreamsNoDuplicateGets() throws Throwable {
386403
387404 // Since it's a small file (ALL will prefetch the whole file for size < 8MB), the whole file is prefetched
388405 // on the first read.
389- verifyStatisticCounterValue (stats1 , STREAM_READ_PREFETCHED_BYTES , 1048575 );
406+ verifyStatisticCounterValue (stats1 , STREAM_READ_PREFETCHED_BYTES , fileLen );
390407
391408 // The second stream will not prefetch any bytes, as they have already been prefetched by stream 1.
392409 verifyStatisticCounterValue (stats2 , STREAM_READ_PREFETCHED_BYTES , 0 );
393410 }
394411
395412 // verify value is passed up to the FS
396- verifyStatisticCounterValue (getFileSystem ().getIOStatistics (), STREAM_READ_PREFETCHED_BYTES , 1048575 );
413+ verifyStatisticCounterValue (getFileSystem ().getIOStatistics (), STREAM_READ_PREFETCHED_BYTES , fileLen );
397414
398415 // We did 3 reads, all of them were served from the small object cache. In this case, the whole object was
399416 // downloaded as soon as the stream to it was opened.
0 commit comments