1717
1818//! Parquet file data reader
1919
20+ use arrow_schema:: SchemaRef as ArrowSchemaRef ;
2021use async_stream:: try_stream;
2122use futures:: stream:: StreamExt ;
22- use parquet:: arrow:: { ParquetRecordBatchStreamBuilder , ProjectionMask } ;
23+ use parquet:: arrow:: { ParquetRecordBatchStreamBuilder , ProjectionMask , PARQUET_FIELD_ID_META_KEY } ;
24+ use parquet:: schema:: types:: SchemaDescriptor ;
25+ use std:: collections:: HashMap ;
26+ use std:: str:: FromStr ;
2327
2428use crate :: io:: FileIO ;
25- use crate :: scan:: { ArrowRecordBatchStream , FileScanTask , FileScanTaskStream } ;
29+ use crate :: scan:: { ArrowRecordBatchStream , FileScanTaskStream } ;
2630use crate :: spec:: SchemaRef ;
2731
2832use crate :: error:: Result ;
@@ -36,6 +40,7 @@ use std::sync::Arc;
3640/// Builder to create ArrowReader
3741pub struct ArrowReaderBuilder {
3842 batch_size : Option < usize > ,
43+ field_ids : Vec < usize > ,
3944 file_io : FileIO ,
4045 schema : SchemaRef ,
4146}
@@ -45,6 +50,7 @@ impl ArrowReaderBuilder {
4550 pub fn new ( file_io : FileIO , schema : SchemaRef ) -> Self {
4651 ArrowReaderBuilder {
4752 batch_size : None ,
53+ field_ids : vec ! [ ] ,
4854 file_io,
4955 schema,
5056 }
@@ -57,10 +63,17 @@ impl ArrowReaderBuilder {
5763 self
5864 }
5965
66+ /// Sets the desired column projection with a list of field ids.
67+ pub fn with_field_ids ( mut self , field_ids : impl IntoIterator < Item = usize > ) -> Self {
68+ self . field_ids = field_ids. into_iter ( ) . collect ( ) ;
69+ self
70+ }
71+
6072 /// Build the ArrowReader.
6173 pub fn build ( self ) -> ArrowReader {
6274 ArrowReader {
6375 batch_size : self . batch_size ,
76+ field_ids : self . field_ids ,
6477 schema : self . schema ,
6578 file_io : self . file_io ,
6679 }
@@ -70,6 +83,7 @@ impl ArrowReaderBuilder {
7083/// Reads data from Parquet files
7184pub struct ArrowReader {
7285 batch_size : Option < usize > ,
86+ field_ids : Vec < usize > ,
7387 #[ allow( dead_code) ]
7488 schema : SchemaRef ,
7589 file_io : FileIO ,
@@ -83,17 +97,18 @@ impl ArrowReader {
8397
8498 Ok ( try_stream ! {
8599 while let Some ( Ok ( task) ) = tasks. next( ) . await {
86-
87- let projection_mask = self . get_arrow_projection_mask( & task) ;
88-
89100 let parquet_reader = file_io
90101 . new_input( task. data( ) . data_file( ) . file_path( ) ) ?
91102 . reader( )
92103 . await ?;
93104
94105 let mut batch_stream_builder = ParquetRecordBatchStreamBuilder :: new( parquet_reader)
95- . await ?
96- . with_projection( projection_mask) ;
106+ . await ?;
107+
108+ let parquet_schema = batch_stream_builder. parquet_schema( ) ;
109+ let arrow_schema = batch_stream_builder. schema( ) ;
110+ let projection_mask = self . get_arrow_projection_mask( parquet_schema, arrow_schema) ?;
111+ batch_stream_builder = batch_stream_builder. with_projection( projection_mask) ;
97112
98113 if let Some ( batch_size) = self . batch_size {
99114 batch_stream_builder = batch_stream_builder. with_batch_size( batch_size) ;
@@ -109,9 +124,73 @@ impl ArrowReader {
109124 . boxed ( ) )
110125 }
111126
112- fn get_arrow_projection_mask ( & self , _task : & FileScanTask ) -> ProjectionMask {
113- // TODO: full implementation
114- ProjectionMask :: all ( )
127+ fn get_arrow_projection_mask (
128+ & self ,
129+ parquet_schema : & SchemaDescriptor ,
130+ arrow_schema : & ArrowSchemaRef ,
131+ ) -> crate :: Result < ProjectionMask > {
132+ if self . field_ids . is_empty ( ) {
133+ Ok ( ProjectionMask :: all ( ) )
134+ } else {
135+ // Build the map between field id and column index in Parquet schema.
136+ let mut column_map = HashMap :: new ( ) ;
137+
138+ let fields = arrow_schema. fields ( ) ;
139+ let iceberg_schema = arrow_schema_to_schema ( arrow_schema) ?;
140+ fields. filter_leaves ( |idx, field| {
141+ let field_id = field. metadata ( ) . get ( PARQUET_FIELD_ID_META_KEY ) ;
142+ if field_id. is_none ( ) {
143+ return false ;
144+ }
145+
146+ let field_id = i32:: from_str ( field_id. unwrap ( ) ) ;
147+ if field_id. is_err ( ) {
148+ return false ;
149+ }
150+ let field_id = field_id. unwrap ( ) ;
151+
152+ if !self . field_ids . contains ( & ( field_id as usize ) ) {
153+ return false ;
154+ }
155+
156+ let iceberg_field = self . schema . field_by_id ( field_id) ;
157+ let parquet_iceberg_field = iceberg_schema. field_by_id ( field_id) ;
158+
159+ if iceberg_field. is_none ( ) || parquet_iceberg_field. is_none ( ) {
160+ return false ;
161+ }
162+
163+ if iceberg_field. unwrap ( ) . field_type != parquet_iceberg_field. unwrap ( ) . field_type {
164+ return false ;
165+ }
166+
167+ column_map. insert ( field_id, idx) ;
168+ true
169+ } ) ;
170+
171+ if column_map. len ( ) != self . field_ids . len ( ) {
172+ return Err ( Error :: new (
173+ ErrorKind :: DataInvalid ,
174+ format ! (
175+ "Parquet schema {} and Iceberg schema {} do not match." ,
176+ iceberg_schema, self . schema
177+ ) ,
178+ ) ) ;
179+ }
180+
181+ let mut indices = vec ! [ ] ;
182+ for field_id in & self . field_ids {
183+ if let Some ( col_idx) = column_map. get ( & ( * field_id as i32 ) ) {
184+ indices. push ( * col_idx) ;
185+ } else {
186+ return Err ( Error :: new (
187+ ErrorKind :: DataInvalid ,
188+ format ! ( "Field {} is not found in Parquet schema." , field_id) ,
189+ ) ) ;
190+ }
191+ }
192+ Ok ( ProjectionMask :: leaves ( parquet_schema, indices) )
193+ }
115194 }
116195}
117196
0 commit comments