1515// specific language governing permissions and limitations
1616// under the License.
1717
18- use std:: collections:: HashMap ;
1918use std:: pin:: Pin ;
2019use std:: sync:: Arc ;
2120
2221use arrow_array:: { RecordBatch , UInt64Array } ;
23- use arrow_schema:: { DataType , Field , Schema as ArrowSchema } ;
22+ use arrow_schema:: Schema as ArrowSchema ;
2423use futures:: channel:: mpsc:: channel;
2524use futures:: stream:: select;
2625use futures:: { SinkExt , Stream , StreamExt , TryStreamExt } ;
27- use parquet:: arrow:: PARQUET_FIELD_ID_META_KEY ;
2826
2927use crate :: arrow:: record_batch_transformer:: RecordBatchTransformerBuilder ;
30- use crate :: arrow:: {
31- ArrowReader , RESERVED_COL_NAME_FILE_PATH , RESERVED_COL_NAME_POS , RESERVED_FIELD_ID_FILE_PATH ,
32- RESERVED_FIELD_ID_POS , StreamsInto ,
33- } ;
28+ use crate :: arrow:: { ArrowReader , StreamsInto } ;
3429use crate :: delete_vector:: DeleteVector ;
3530use crate :: io:: FileIO ;
3631use crate :: runtime:: spawn;
@@ -43,19 +38,6 @@ use crate::{Error, ErrorKind, Result};
4338/// Default batch size for incremental delete operations.
4439const DEFAULT_BATCH_SIZE : usize = 1024 ;
4540
46- /// Creates the schema for positional delete records containing the "pos" column.
47- /// The pos field includes the reserved field ID as metadata.
48- fn create_pos_delete_schema ( ) -> Arc < ArrowSchema > {
49- let pos_field =
50- Field :: new ( RESERVED_COL_NAME_POS , DataType :: UInt64 , false ) . with_metadata ( HashMap :: from ( [
51- (
52- PARQUET_FIELD_ID_META_KEY . to_string ( ) ,
53- RESERVED_FIELD_ID_POS . to_string ( ) ,
54- ) ,
55- ] ) ) ;
56- Arc :: new ( ArrowSchema :: new ( vec ! [ pos_field] ) )
57- }
58-
5941/// The type of incremental batch: appended data or deleted records.
6042#[ derive( Debug , Clone , Copy , PartialEq , Eq ) ]
6143pub enum IncrementalBatchType {
@@ -254,10 +236,15 @@ async fn process_incremental_append_task(
254236 record_batch_stream_builder = record_batch_stream_builder. with_projection ( projection_mask) ;
255237
256238 // RecordBatchTransformer performs any transformations required on the RecordBatches
257- // that come back from the file, such as type promotion, default column insertion
258- // and column re-ordering
239+ // that come back from the file, such as type promotion, default column insertion,
240+ // column re-ordering, and virtual field addition (like _file)
259241 let mut record_batch_transformer =
260- RecordBatchTransformerBuilder :: new ( task. schema_ref ( ) , & task. base . project_field_ids ) . build ( ) ;
242+ RecordBatchTransformerBuilder :: new ( task. schema_ref ( ) , & task. base . project_field_ids )
243+ . with_constant (
244+ crate :: metadata_columns:: RESERVED_FIELD_ID_FILE ,
245+ crate :: spec:: PrimitiveLiteral :: String ( task. base . data_file_path . clone ( ) ) ,
246+ ) ?
247+ . build ( ) ;
261248
262249 if let Some ( batch_size) = batch_size {
263250 record_batch_stream_builder = record_batch_stream_builder. with_batch_size ( batch_size) ;
@@ -295,7 +282,9 @@ fn process_incremental_delete_task(
295282 delete_vector : DeleteVector ,
296283 batch_size : Option < usize > ,
297284) -> Result < ArrowRecordBatchStream > {
298- let schema = create_pos_delete_schema ( ) ;
285+ let schema = Arc :: new ( ArrowSchema :: new ( vec ! [ Arc :: clone(
286+ crate :: metadata_columns:: pos_field( ) ,
287+ ) ] ) ) ;
299288
300289 let batch_size = batch_size. unwrap_or ( DEFAULT_BATCH_SIZE ) ;
301290
@@ -315,14 +304,7 @@ fn process_incremental_delete_task(
315304 "Failed to create RecordBatch for DeleteVector" ,
316305 )
317306 } )
318- . and_then ( |batch| {
319- ArrowReader :: add_file_path_column (
320- batch,
321- & file_path,
322- RESERVED_COL_NAME_FILE_PATH ,
323- RESERVED_FIELD_ID_FILE_PATH ,
324- )
325- } )
307+ . and_then ( |batch| ArrowReader :: add_file_path_column ( batch, & file_path) )
326308 } ) ;
327309
328310 Ok ( Box :: pin ( stream) as ArrowRecordBatchStream )
@@ -333,7 +315,9 @@ fn process_incremental_deleted_file_task(
333315 total_records : u64 ,
334316 batch_size : Option < usize > ,
335317) -> Result < ArrowRecordBatchStream > {
336- let schema = create_pos_delete_schema ( ) ;
318+ let schema = Arc :: new ( ArrowSchema :: new ( vec ! [ Arc :: clone(
319+ crate :: metadata_columns:: pos_field( ) ,
320+ ) ] ) ) ;
337321
338322 let batch_size = batch_size. unwrap_or ( DEFAULT_BATCH_SIZE ) ;
339323
@@ -352,14 +336,7 @@ fn process_incremental_deleted_file_task(
352336 "Failed to create RecordBatch for deleted file" ,
353337 )
354338 } )
355- . and_then ( |batch| {
356- ArrowReader :: add_file_path_column (
357- batch,
358- & file_path,
359- RESERVED_COL_NAME_FILE_PATH ,
360- RESERVED_FIELD_ID_FILE_PATH ,
361- )
362- } )
339+ . and_then ( |batch| ArrowReader :: add_file_path_column ( batch, & file_path) )
363340 } ) ;
364341
365342 Ok ( Box :: pin ( stream) as ArrowRecordBatchStream )
0 commit comments