@@ -80,10 +80,10 @@ pub use store::*;
80
80
/// [`tokio::fs::File`]: https://docs.rs/tokio/latest/tokio/fs/struct.File.html
81
81
pub trait AsyncFileReader : Send {
82
82
/// Retrieve the bytes in `range`
83
- fn get_bytes ( & mut self , range : Range < usize > ) -> BoxFuture < ' _ , Result < Bytes > > ;
83
+ fn get_bytes ( & mut self , range : Range < u64 > ) -> BoxFuture < ' _ , Result < Bytes > > ;
84
84
85
85
/// Retrieve multiple byte ranges. The default implementation will call `get_bytes` sequentially
86
- fn get_byte_ranges ( & mut self , ranges : Vec < Range < usize > > ) -> BoxFuture < ' _ , Result < Vec < Bytes > > > {
86
+ fn get_byte_ranges ( & mut self , ranges : Vec < Range < u64 > > ) -> BoxFuture < ' _ , Result < Vec < Bytes > > > {
87
87
async move {
88
88
let mut result = Vec :: with_capacity ( ranges. len ( ) ) ;
89
89
@@ -121,11 +121,11 @@ pub trait AsyncFileReader: Send {
121
121
122
122
/// This allows Box<dyn AsyncFileReader + '_> to be used as an AsyncFileReader,
123
123
impl AsyncFileReader for Box < dyn AsyncFileReader + ' _ > {
124
- fn get_bytes ( & mut self , range : Range < usize > ) -> BoxFuture < ' _ , Result < Bytes > > {
124
+ fn get_bytes ( & mut self , range : Range < u64 > ) -> BoxFuture < ' _ , Result < Bytes > > {
125
125
self . as_mut ( ) . get_bytes ( range)
126
126
}
127
127
128
- fn get_byte_ranges ( & mut self , ranges : Vec < Range < usize > > ) -> BoxFuture < ' _ , Result < Vec < Bytes > > > {
128
+ fn get_byte_ranges ( & mut self , ranges : Vec < Range < u64 > > ) -> BoxFuture < ' _ , Result < Vec < Bytes > > > {
129
129
self . as_mut ( ) . get_byte_ranges ( ranges)
130
130
}
131
131
@@ -150,14 +150,14 @@ impl<T: AsyncFileReader + MetadataFetch + AsyncRead + AsyncSeek + Unpin> Metadat
150
150
}
151
151
152
152
impl < T : AsyncRead + AsyncSeek + Unpin + Send > AsyncFileReader for T {
153
- fn get_bytes ( & mut self , range : Range < usize > ) -> BoxFuture < ' _ , Result < Bytes > > {
153
+ fn get_bytes ( & mut self , range : Range < u64 > ) -> BoxFuture < ' _ , Result < Bytes > > {
154
154
async move {
155
- self . seek ( SeekFrom :: Start ( range. start as u64 ) ) . await ?;
155
+ self . seek ( SeekFrom :: Start ( range. start ) ) . await ?;
156
156
157
157
let to_read = range. end - range. start ;
158
- let mut buffer = Vec :: with_capacity ( to_read) ;
159
- let read = self . take ( to_read as u64 ) . read_to_end ( & mut buffer) . await ?;
160
- if read != to_read {
158
+ let mut buffer = Vec :: with_capacity ( to_read. try_into ( ) ? ) ;
159
+ let read = self . take ( to_read) . read_to_end ( & mut buffer) . await ?;
160
+ if read as u64 != to_read {
161
161
return Err ( eof_err ! ( "expected to read {} bytes, got {}" , to_read, read) ) ;
162
162
}
163
163
@@ -424,7 +424,7 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> {
424
424
let metadata = self . metadata . row_group ( row_group_idx) ;
425
425
let column_metadata = metadata. column ( column_idx) ;
426
426
427
- let offset: usize = if let Some ( offset) = column_metadata. bloom_filter_offset ( ) {
427
+ let offset: u64 = if let Some ( offset) = column_metadata. bloom_filter_offset ( ) {
428
428
offset
429
429
. try_into ( )
430
430
. map_err ( |_| ParquetError :: General ( "Bloom filter offset is invalid" . to_string ( ) ) ) ?
@@ -433,16 +433,16 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> {
433
433
} ;
434
434
435
435
let buffer = match column_metadata. bloom_filter_length ( ) {
436
- Some ( length) => self . input . 0 . get_bytes ( offset..offset + length as usize ) ,
436
+ Some ( length) => self . input . 0 . get_bytes ( offset..offset + length as u64 ) ,
437
437
None => self
438
438
. input
439
439
. 0
440
- . get_bytes ( offset..offset + SBBF_HEADER_SIZE_ESTIMATE ) ,
440
+ . get_bytes ( offset..offset + SBBF_HEADER_SIZE_ESTIMATE as u64 ) ,
441
441
}
442
442
. await ?;
443
443
444
444
let ( header, bitset_offset) =
445
- chunk_read_bloom_filter_header_and_offset ( offset as u64 , buffer. clone ( ) ) ?;
445
+ chunk_read_bloom_filter_header_and_offset ( offset, buffer. clone ( ) ) ?;
446
446
447
447
match header. algorithm {
448
448
BloomFilterAlgorithm :: BLOCK ( _) => {
@@ -461,14 +461,17 @@ impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> {
461
461
}
462
462
463
463
let bitset = match column_metadata. bloom_filter_length ( ) {
464
- Some ( _) => buffer. slice ( ( bitset_offset as usize - offset) ..) ,
464
+ Some ( _) => buffer. slice (
465
+ ( TryInto :: < usize > :: try_into ( bitset_offset) . unwrap ( )
466
+ - TryInto :: < usize > :: try_into ( offset) . unwrap ( ) ) ..,
467
+ ) ,
465
468
None => {
466
- let bitset_length: usize = header. num_bytes . try_into ( ) . map_err ( |_| {
469
+ let bitset_length: u64 = header. num_bytes . try_into ( ) . map_err ( |_| {
467
470
ParquetError :: General ( "Bloom filter length is invalid" . to_string ( ) )
468
471
} ) ?;
469
472
self . input
470
473
. 0
471
- . get_bytes ( bitset_offset as usize ..bitset_offset as usize + bitset_length)
474
+ . get_bytes ( bitset_offset..bitset_offset + bitset_length)
472
475
. await ?
473
476
}
474
477
} ;
@@ -880,7 +883,7 @@ impl InMemoryRowGroup<'_> {
880
883
if let Some ( ( selection, offset_index) ) = selection. zip ( self . offset_index ) {
881
884
// If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the
882
885
// `RowSelection`
883
- let mut page_start_offsets: Vec < Vec < usize > > = vec ! [ ] ;
886
+ let mut page_start_offsets: Vec < Vec < u64 > > = vec ! [ ] ;
884
887
885
888
let fetch_ranges = self
886
889
. column_chunks
@@ -893,11 +896,11 @@ impl InMemoryRowGroup<'_> {
893
896
. flat_map ( |( idx, ( _chunk, chunk_meta) ) | {
894
897
// If the first page does not start at the beginning of the column,
895
898
// then we need to also fetch a dictionary page.
896
- let mut ranges = vec ! [ ] ;
899
+ let mut ranges: Vec < Range < u64 > > = vec ! [ ] ;
897
900
let ( start, _len) = chunk_meta. byte_range ( ) ;
898
901
match offset_index[ idx] . page_locations . first ( ) {
899
902
Some ( first) if first. offset as u64 != start => {
900
- ranges. push ( start as usize ..first. offset as usize ) ;
903
+ ranges. push ( start..first. offset as u64 ) ;
901
904
}
902
905
_ => ( ) ,
903
906
}
@@ -925,7 +928,11 @@ impl InMemoryRowGroup<'_> {
925
928
926
929
* chunk = Some ( Arc :: new ( ColumnChunkData :: Sparse {
927
930
length : metadata. column ( idx) . byte_range ( ) . 1 as usize ,
928
- data : offsets. into_iter ( ) . zip ( chunks. into_iter ( ) ) . collect ( ) ,
931
+ data : offsets
932
+ . into_iter ( )
933
+ . map ( |x| x as usize )
934
+ . zip ( chunks. into_iter ( ) )
935
+ . collect ( ) ,
929
936
} ) )
930
937
}
931
938
}
@@ -938,7 +945,7 @@ impl InMemoryRowGroup<'_> {
938
945
. map ( |( idx, _chunk) | {
939
946
let column = metadata. column ( idx) ;
940
947
let ( start, length) = column. byte_range ( ) ;
941
- start as usize ..( start + length) as usize
948
+ start..( start + length)
942
949
} )
943
950
. collect ( ) ;
944
951
@@ -1108,9 +1115,16 @@ mod tests {
1108
1115
}
1109
1116
1110
1117
impl AsyncFileReader for TestReader {
1111
- fn get_bytes ( & mut self , range : Range < usize > ) -> BoxFuture < ' _ , Result < Bytes > > {
1112
- self . requests . lock ( ) . unwrap ( ) . push ( range. clone ( ) ) ;
1113
- futures:: future:: ready ( Ok ( self . data . slice ( range) ) ) . boxed ( )
1118
+ fn get_bytes ( & mut self , range : Range < u64 > ) -> BoxFuture < ' _ , Result < Bytes > > {
1119
+ let range = range. clone ( ) ;
1120
+ self . requests
1121
+ . lock ( )
1122
+ . unwrap ( )
1123
+ . push ( range. start as usize ..range. end as usize ) ;
1124
+ futures:: future:: ready ( Ok ( self
1125
+ . data
1126
+ . slice ( range. start as usize ..range. end as usize ) ) )
1127
+ . boxed ( )
1114
1128
}
1115
1129
1116
1130
fn get_metadata < ' a > (
@@ -2238,7 +2252,7 @@ mod tests {
2238
2252
let file_size = file. metadata ( ) . await . unwrap ( ) . len ( ) ;
2239
2253
let mut metadata = ParquetMetaDataReader :: new ( )
2240
2254
. with_page_indexes ( true )
2241
- . load_and_finish ( & mut file, file_size as usize )
2255
+ . load_and_finish ( & mut file, file_size)
2242
2256
. await
2243
2257
. unwrap ( ) ;
2244
2258
@@ -2263,7 +2277,7 @@ mod tests {
2263
2277
let file_size = file. metadata ( ) . await . unwrap ( ) . len ( ) ;
2264
2278
let metadata = ParquetMetaDataReader :: new ( )
2265
2279
. with_page_indexes ( true )
2266
- . load_and_finish ( & mut file, file_size as usize )
2280
+ . load_and_finish ( & mut file, file_size)
2267
2281
. await
2268
2282
. unwrap ( ) ;
2269
2283
@@ -2309,7 +2323,7 @@ mod tests {
2309
2323
let file_size = file. metadata ( ) . await . unwrap ( ) . len ( ) ;
2310
2324
let metadata = ParquetMetaDataReader :: new ( )
2311
2325
. with_page_indexes ( true )
2312
- . load_and_finish ( & mut file, file_size as usize )
2326
+ . load_and_finish ( & mut file, file_size)
2313
2327
. await
2314
2328
. unwrap ( ) ;
2315
2329
0 commit comments