15
15
// specific language governing permissions and limitations
16
16
// under the License.
17
17
18
- use crate :: arrow:: async_reader:: AsyncFileReader ;
18
+ use crate :: arrow:: async_reader:: { AsyncFileReader , GetRange } ;
19
19
use crate :: errors:: { ParquetError , Result } ;
20
20
use crate :: file:: footer:: { decode_footer, decode_metadata} ;
21
21
use crate :: file:: metadata:: ParquetMetaData ;
@@ -25,15 +25,14 @@ use bytes::Bytes;
25
25
use futures:: future:: BoxFuture ;
26
26
use futures:: FutureExt ;
27
27
use std:: future:: Future ;
28
- use std:: ops:: Range ;
29
28
30
29
/// A data source that can be used with [`MetadataLoader`] to load [`ParquetMetaData`]
31
30
pub trait MetadataFetch {
32
- fn fetch ( & mut self , range : Range < usize > ) -> BoxFuture < ' _ , Result < Bytes > > ;
31
+ fn fetch ( & mut self , range : GetRange ) -> BoxFuture < ' _ , Result < Bytes > > ;
33
32
}
34
33
35
34
impl < ' a , T : AsyncFileReader > MetadataFetch for & ' a mut T {
36
- fn fetch ( & mut self , range : Range < usize > ) -> BoxFuture < ' _ , Result < Bytes > > {
35
+ fn fetch ( & mut self , range : GetRange ) -> BoxFuture < ' _ , Result < Bytes > > {
37
36
self . get_bytes ( range)
38
37
}
39
38
}
@@ -52,49 +51,30 @@ impl<F: MetadataFetch> MetadataLoader<F> {
52
51
/// Create a new [`MetadataLoader`] by reading the footer information
53
52
///
54
53
/// See [`fetch_parquet_metadata`] for the meaning of the individual parameters
55
- pub async fn load ( mut fetch : F , file_size : usize , prefetch : Option < usize > ) -> Result < Self > {
56
- if file_size < 8 {
57
- return Err ( ParquetError :: EOF ( format ! (
58
- "file size of {file_size} is less than footer"
59
- ) ) ) ;
60
- }
54
+ pub async fn load ( mut fetch : F , prefetch : Option < usize > ) -> Result < Self > {
61
55
62
- // If a size hint is provided, read more than the minimum size
63
- // to try and avoid a second fetch.
64
- let footer_start = if let Some ( size_hint) = prefetch {
65
- file_size. saturating_sub ( size_hint)
66
- } else {
67
- file_size - 8
68
- } ;
69
-
70
- let suffix = fetch. fetch ( footer_start..file_size) . await ?;
56
+ let suffix = fetch. fetch ( GetRange :: Suffix ( prefetch. unwrap_or ( 8 ) ) ) . await ?;
71
57
let suffix_len = suffix. len ( ) ;
72
58
73
59
let mut footer = [ 0 ; 8 ] ;
74
60
footer. copy_from_slice ( & suffix[ suffix_len - 8 ..suffix_len] ) ;
75
61
76
62
let length = decode_footer ( & footer) ?;
77
63
78
- if file_size < length + 8 {
79
- return Err ( ParquetError :: EOF ( format ! (
80
- "file size of {} is less than footer + metadata {}" ,
81
- file_size,
82
- length + 8
83
- ) ) ) ;
84
- }
85
-
86
64
// Did not fetch the entire file metadata in the initial read, need to make a second request
87
65
let ( metadata, remainder) = if length > suffix_len - 8 {
88
- let metadata_start = file_size - length - 8 ;
89
- let meta = fetch. fetch ( metadata_start..file_size - 8 ) . await ?;
90
- ( decode_metadata ( & meta) ?, None )
66
+ let metadata_offset = length + 8 ;
67
+ let meta = fetch. fetch ( GetRange :: Suffix ( metadata_offset) ) . await ?;
68
+ let slice = & meta[ 0 ..length] ;
69
+ ( decode_metadata ( & slice) ?, None )
91
70
} else {
92
- let metadata_start = file_size - length - 8 - footer_start;
71
+ let metadata_offset = length + 8 ;
72
+ let metadata_start = suffix_len - metadata_offset;
93
73
94
74
let slice = & suffix[ metadata_start..suffix_len - 8 ] ;
95
75
(
96
76
decode_metadata ( slice) ?,
97
- Some ( ( footer_start , suffix. slice ( ..metadata_start) ) ) ,
77
+ Some ( ( 0 , suffix. slice ( ..metadata_start) ) ) ,
98
78
)
99
79
} ;
100
80
@@ -105,6 +85,10 @@ impl<F: MetadataFetch> MetadataLoader<F> {
105
85
} )
106
86
}
107
87
88
+ pub async fn load_absolute ( mut fetch : F , file_size : usize , prefetch : Option < usize > ) -> Result < Self > {
89
+ todo ! ( )
90
+ }
91
+
108
92
/// Create a new [`MetadataLoader`] from an existing [`ParquetMetaData`]
109
93
pub fn new ( fetch : F , metadata : ParquetMetaData ) -> Self {
110
94
Self {
@@ -133,13 +117,15 @@ impl<F: MetadataFetch> MetadataLoader<F> {
133
117
Some ( range) => range,
134
118
} ;
135
119
120
+ let page_index_len = range. end - range. start ;
121
+ // TODO: determine if _remainder_start is needed even in the non-suffix request case
136
122
let data = match & self . remainder {
137
- Some ( ( remainder_start , remainder) ) if * remainder_start <= range . start => {
138
- let offset = range . start - * remainder_start ;
139
- remainder. slice ( offset..range . end - * remainder_start + offset )
123
+ Some ( ( _remainder_start , remainder) ) if remainder . len ( ) >= page_index_len => {
124
+ let offset = remainder . len ( ) - page_index_len ;
125
+ remainder. slice ( offset..)
140
126
}
141
127
// Note: this will potentially fetch data already in remainder, this keeps things simple
142
- _ => self . fetch . fetch ( range. start ..range. end ) . await ?,
128
+ _ => self . fetch . fetch ( ( range. start ..range. end ) . into ( ) ) . await ?,
143
129
} ;
144
130
145
131
// Sanity check
@@ -200,10 +186,10 @@ struct MetadataFetchFn<F>(F);
200
186
201
187
impl < F , Fut > MetadataFetch for MetadataFetchFn < F >
202
188
where
203
- F : FnMut ( Range < usize > ) -> Fut + Send ,
189
+ F : FnMut ( GetRange ) -> Fut + Send ,
204
190
Fut : Future < Output = Result < Bytes > > + Send ,
205
191
{
206
- fn fetch ( & mut self , range : Range < usize > ) -> BoxFuture < ' _ , Result < Bytes > > {
192
+ fn fetch ( & mut self , range : GetRange ) -> BoxFuture < ' _ , Result < Bytes > > {
207
193
async move { self . 0 ( range) . await } . boxed ( )
208
194
}
209
195
}
@@ -226,15 +212,18 @@ where
226
212
/// significantly reduce the number of `fetch` requests, and consequently latency
227
213
pub async fn fetch_parquet_metadata < F , Fut > (
228
214
fetch : F ,
229
- file_size : usize ,
215
+ file_size : Option < usize > ,
230
216
prefetch : Option < usize > ,
231
217
) -> Result < ParquetMetaData >
232
218
where
233
- F : FnMut ( Range < usize > ) -> Fut + Send ,
219
+ F : FnMut ( GetRange ) -> Fut + Send ,
234
220
Fut : Future < Output = Result < Bytes > > + Send ,
235
221
{
236
222
let fetch = MetadataFetchFn ( fetch) ;
237
- let loader = MetadataLoader :: load ( fetch, file_size, prefetch) . await ?;
223
+ let loader = match file_size {
224
+ Some ( file_size) => MetadataLoader :: load_absolute ( fetch, file_size, prefetch) . await ?,
225
+ None => MetadataLoader :: load ( fetch, prefetch) . await ?
226
+ } ;
238
227
Ok ( loader. finish ( ) )
239
228
}
240
229
@@ -247,7 +236,13 @@ mod tests {
247
236
use std:: io:: { Read , Seek , SeekFrom } ;
248
237
use std:: sync:: atomic:: { AtomicUsize , Ordering } ;
249
238
250
- fn read_range ( file : & mut File , range : Range < usize > ) -> Result < Bytes > {
239
+ fn read_range ( file : & mut File , range : GetRange ) -> Result < Bytes > {
240
+ let file_size = file. len ( ) . try_into ( ) . unwrap ( ) ;
241
+ let range = match range {
242
+ GetRange :: Bounded ( range) => range,
243
+ GetRange :: Offset ( offset) => offset..file_size,
244
+ GetRange :: Suffix ( end_offset) => ( file_size. saturating_sub ( end_offset. try_into ( ) . unwrap ( ) ) ..file_size)
245
+ } ;
251
246
file. seek ( SeekFrom :: Start ( range. start as _ ) ) ?;
252
247
let len = range. end - range. start ;
253
248
let mut buf = Vec :: with_capacity ( len) ;
@@ -269,41 +264,41 @@ mod tests {
269
264
futures:: future:: ready ( read_range ( & mut file, range) )
270
265
} ;
271
266
272
- let actual = fetch_parquet_metadata ( & mut fetch, len , None ) . await . unwrap ( ) ;
267
+ let actual = fetch_parquet_metadata ( & mut fetch, None , None ) . await . unwrap ( ) ;
273
268
assert_eq ! ( actual. file_metadata( ) . schema( ) , expected) ;
274
269
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 2 ) ;
275
270
276
271
// Metadata hint too small
277
272
fetch_count. store ( 0 , Ordering :: SeqCst ) ;
278
- let actual = fetch_parquet_metadata ( & mut fetch, len , Some ( 10 ) )
273
+ let actual = fetch_parquet_metadata ( & mut fetch, None , Some ( 10 ) )
279
274
. await
280
275
. unwrap ( ) ;
281
276
assert_eq ! ( actual. file_metadata( ) . schema( ) , expected) ;
282
277
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 2 ) ;
283
278
284
279
// Metadata hint too large
285
280
fetch_count. store ( 0 , Ordering :: SeqCst ) ;
286
- let actual = fetch_parquet_metadata ( & mut fetch, len , Some ( 500 ) )
281
+ let actual = fetch_parquet_metadata ( & mut fetch, None , Some ( 500 ) )
287
282
. await
288
283
. unwrap ( ) ;
289
284
assert_eq ! ( actual. file_metadata( ) . schema( ) , expected) ;
290
285
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
291
286
292
287
// Metadata hint exactly correct
293
288
fetch_count. store ( 0 , Ordering :: SeqCst ) ;
294
- let actual = fetch_parquet_metadata ( & mut fetch, len , Some ( 428 ) )
289
+ let actual = fetch_parquet_metadata ( & mut fetch, None , Some ( 428 ) )
295
290
. await
296
291
. unwrap ( ) ;
297
292
assert_eq ! ( actual. file_metadata( ) . schema( ) , expected) ;
298
293
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
299
294
300
- let err = fetch_parquet_metadata ( & mut fetch, 4 , None )
295
+ let err = fetch_parquet_metadata ( & mut fetch, Some ( 4 ) , None )
301
296
. await
302
297
. unwrap_err ( )
303
298
. to_string ( ) ;
304
299
assert_eq ! ( err, "EOF: file size of 4 is less than footer" ) ;
305
300
306
- let err = fetch_parquet_metadata ( & mut fetch, 20 , None )
301
+ let err = fetch_parquet_metadata ( & mut fetch, Some ( 20 ) , None )
307
302
. await
308
303
. unwrap_err ( )
309
304
. to_string ( ) ;
@@ -321,7 +316,7 @@ mod tests {
321
316
} ;
322
317
323
318
let f = MetadataFetchFn ( & mut fetch) ;
324
- let mut loader = MetadataLoader :: load ( f, len , None ) . await . unwrap ( ) ;
319
+ let mut loader = MetadataLoader :: load ( f, None ) . await . unwrap ( ) ;
325
320
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 2 ) ;
326
321
loader. load_page_index ( true , true ) . await . unwrap ( ) ;
327
322
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 3 ) ;
@@ -331,7 +326,7 @@ mod tests {
331
326
// Prefetch just footer exactly
332
327
fetch_count. store ( 0 , Ordering :: SeqCst ) ;
333
328
let f = MetadataFetchFn ( & mut fetch) ;
334
- let mut loader = MetadataLoader :: load ( f, len , Some ( 1729 ) ) . await . unwrap ( ) ;
329
+ let mut loader = MetadataLoader :: load ( f, Some ( 1729 ) ) . await . unwrap ( ) ;
335
330
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
336
331
loader. load_page_index ( true , true ) . await . unwrap ( ) ;
337
332
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 2 ) ;
@@ -341,7 +336,7 @@ mod tests {
341
336
// Prefetch more than footer but not enough
342
337
fetch_count. store ( 0 , Ordering :: SeqCst ) ;
343
338
let f = MetadataFetchFn ( & mut fetch) ;
344
- let mut loader = MetadataLoader :: load ( f, len , Some ( 130649 ) ) . await . unwrap ( ) ;
339
+ let mut loader = MetadataLoader :: load ( f, Some ( 130649 ) ) . await . unwrap ( ) ;
345
340
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
346
341
loader. load_page_index ( true , true ) . await . unwrap ( ) ;
347
342
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 2 ) ;
@@ -351,7 +346,17 @@ mod tests {
351
346
// Prefetch exactly enough
352
347
fetch_count. store ( 0 , Ordering :: SeqCst ) ;
353
348
let f = MetadataFetchFn ( & mut fetch) ;
354
- let mut loader = MetadataLoader :: load ( f, len, Some ( 130650 ) ) . await . unwrap ( ) ;
349
+ let mut loader = MetadataLoader :: load ( f, Some ( 130650 ) ) . await . unwrap ( ) ;
350
+ assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
351
+ loader. load_page_index ( true , true ) . await . unwrap ( ) ;
352
+ assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
353
+ let metadata = loader. finish ( ) ;
354
+ assert ! ( metadata. offset_index( ) . is_some( ) && metadata. column_index( ) . is_some( ) ) ;
355
+
356
+ // Prefetch more than enough
357
+ fetch_count. store ( 0 , Ordering :: SeqCst ) ;
358
+ let f = MetadataFetchFn ( & mut fetch) ;
359
+ let mut loader = MetadataLoader :: load ( f, Some ( 131651 ) ) . await . unwrap ( ) ;
355
360
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
356
361
loader. load_page_index ( true , true ) . await . unwrap ( ) ;
357
362
assert_eq ! ( fetch_count. load( Ordering :: SeqCst ) , 1 ) ;
0 commit comments