@@ -34,7 +34,6 @@ use common_arrow::parquet::metadata::ColumnChunkMetaData;
3434use common_arrow:: parquet:: metadata:: FileMetaData ;
3535use common_arrow:: parquet:: metadata:: RowGroupMetaData ;
3636use common_arrow:: parquet:: read:: read_metadata;
37- use common_arrow:: read_columns_async;
3837use common_exception:: ErrorCode ;
3938use common_exception:: Result ;
4039use common_expression:: DataBlock ;
@@ -48,7 +47,9 @@ use common_settings::Settings;
4847use common_storage:: read_parquet_metas_in_parallel;
4948use common_storage:: StageFileInfo ;
5049use futures:: AsyncRead ;
50+ use futures:: AsyncReadExt ;
5151use futures:: AsyncSeek ;
52+ use futures:: AsyncSeekExt ;
5253use opendal:: Operator ;
5354use serde:: Deserializer ;
5455use serde:: Serializer ;
@@ -174,16 +175,9 @@ impl InputFormatPipe for ParquetFormatPipe {
174175 ) -> Result < Self :: RowBatch > {
175176 let meta = Self :: get_split_meta ( & split_info) . expect ( "must success" ) ;
176177 let op = ctx. source . get_operator ( ) ?;
177- let mut reader = op. reader ( & split_info. file . path ) . await ?;
178178 let input_fields = Arc :: new ( get_used_fields ( & meta. file . fields , & ctx. schema ) ?) ;
179179
180- RowGroupInMemory :: read_async (
181- split_info. to_string ( ) ,
182- & mut reader,
183- meta. meta . clone ( ) ,
184- input_fields,
185- )
186- . await
180+ RowGroupInMemory :: read_async ( split_info. clone ( ) , op, meta. meta . clone ( ) , input_fields) . await
187181 }
188182}
189183
@@ -269,25 +263,48 @@ impl RowGroupInMemory {
269263 } )
270264 }
271265
272- async fn read_async < R : AsyncRead + AsyncSeek + Send + Unpin > (
273- split_info : String ,
274- reader : & mut R ,
266+ async fn read_field_async (
267+ op : Operator ,
268+ path : String ,
269+ col_metas : Vec < ColumnChunkMetaData > ,
270+ index : usize ,
271+ ) -> Result < ( usize , Vec < Vec < u8 > > ) > {
272+ let mut cols = Vec :: with_capacity ( col_metas. len ( ) ) ;
273+ let mut reader = op. reader ( & path) . await ?;
274+ for meta in & col_metas {
275+ cols. push ( read_single_column_async ( & mut reader, meta) . await ?)
276+ }
277+ Ok ( ( index, cols) )
278+ }
279+
280+ async fn read_async (
281+ split_info : Arc < SplitInfo > ,
282+ operator : Operator ,
275283 meta : RowGroupMetaData ,
276284 fields : Arc < Vec < Field > > ,
277285 ) -> Result < Self > {
278286 let field_names = fields. iter ( ) . map ( |x| x. name . as_str ( ) ) . collect :: < Vec < _ > > ( ) ;
279287 let field_meta_indexes = split_column_metas_by_field ( meta. columns ( ) , & field_names) ;
280- let mut filed_arrays = vec ! [ ] ;
281- for field_name in field_names {
282- let meta_data = read_columns_async ( reader, meta. columns ( ) , field_name) . await ?;
283- let data = meta_data. into_iter ( ) . map ( |t| t. 1 ) . collect :: < Vec < _ > > ( ) ;
284- filed_arrays. push ( data)
288+ let mut join_handlers = Vec :: with_capacity ( field_names. len ( ) ) ;
289+ for ( i, field_name) in field_names. iter ( ) . enumerate ( ) {
290+ let col_metas = get_field_columns ( meta. columns ( ) , field_name)
291+ . into_iter ( )
292+ . cloned ( )
293+ . collect :: < Vec < _ > > ( ) ;
294+ let op = operator. clone ( ) ;
295+ let path = split_info. file . path . clone ( ) ;
296+ join_handlers. push ( async move { Self :: read_field_async ( op, path, col_metas, i) . await } ) ;
285297 }
298+
299+ let mut field_arrays = futures:: future:: try_join_all ( join_handlers) . await ?;
300+ field_arrays. sort ( ) ;
301+ let field_arrays = field_arrays. into_iter ( ) . map ( |t| t. 1 ) . collect :: < Vec < _ > > ( ) ;
302+
286303 Ok ( Self {
287- split_info,
304+ split_info : split_info . to_string ( ) ,
288305 meta,
289306 field_meta_indexes,
290- field_arrays : filed_arrays ,
307+ field_arrays,
291308 fields_to_read : fields,
292309 } )
293310 }
@@ -489,3 +506,27 @@ pub fn split_column_metas_by_field(
489506 } ) ;
490507 r
491508}
509+
510+ fn get_field_columns < ' a > (
511+ columns : & ' a [ ColumnChunkMetaData ] ,
512+ field_name : & str ,
513+ ) -> Vec < & ' a ColumnChunkMetaData > {
514+ columns
515+ . iter ( )
516+ . filter ( |x| x. descriptor ( ) . path_in_schema [ 0 ] == field_name)
517+ . collect ( )
518+ }
519+
520+ async fn read_single_column_async < R > (
521+ reader : & mut R ,
522+ meta : & ColumnChunkMetaData ,
523+ ) -> Result < Vec < u8 > >
524+ where
525+ R : AsyncRead + AsyncSeek + Send + Unpin ,
526+ {
527+ let ( start, len) = meta. byte_range ( ) ;
528+ reader. seek ( std:: io:: SeekFrom :: Start ( start) ) . await ?;
529+ let mut chunk = vec ! [ 0 ; len as usize ] ;
530+ reader. read_exact ( & mut chunk) . await ?;
531+ Ok ( chunk)
532+ }
0 commit comments