@@ -27,14 +27,17 @@ use datafusion::physical_plan::execution_plan::{
2727 Boundedness , CardinalityEffect , EmissionType , PlanProperties ,
2828} ;
2929use datafusion:: physical_plan:: filter_pushdown:: { FilterDescription , FilterPushdownPhase } ;
30- use datafusion:: physical_plan:: metrics:: { BaselineMetrics , ExecutionPlanMetricsSet , MetricsSet } ;
30+ use datafusion:: physical_plan:: metrics:: {
31+ BaselineMetrics , Count , ExecutionPlanMetricsSet , MetricBuilder , MetricsSet ,
32+ } ;
3133use datafusion:: physical_plan:: {
3234 DisplayAs , DisplayFormatType , ExecutionPlan , Partitioning , PhysicalExpr , Statistics ,
3335} ;
3436use delta_kernel:: schema:: { Schema as KernelSchema , SchemaRef as KernelSchemaRef } ;
3537use delta_kernel:: { EvaluationHandler , ExpressionRef } ;
3638use futures:: stream:: Stream ;
3739use itertools:: Itertools as _;
40+ use tracing:: debug;
3841
3942use crate :: delta_datafusion:: table_provider:: next:: KernelScanPlan ;
4043use crate :: kernel:: ARROW_HANDLER ;
@@ -191,9 +194,12 @@ impl ExecutionPlan for DeltaScanMetaExec {
191194 scan_plan : Arc :: clone ( & self . scan_plan ) ,
192195 input : self . input [ partition] . clone ( ) ,
193196 baseline_metrics : BaselineMetrics :: new ( & self . metrics , partition) ,
197+ dv_short_mask_padded_files_total : MetricBuilder :: new ( & self . metrics )
198+ . counter ( "dv_short_mask_padded_files_total" , partition) ,
194199 transforms : Arc :: clone ( & self . transforms ) ,
195200 selection_vectors : Arc :: clone ( & self . selection_vectors ) ,
196201 file_id_field : self . file_id_field . clone ( ) ,
202+ schema_adapter : super :: SchemaAdapter :: new ( Arc :: clone ( & self . scan_plan . result_schema ) ) ,
197203 } ) )
198204 }
199205
@@ -258,17 +264,21 @@ struct DeltaScanMetaStream {
258264 input : VecDeque < ( String , usize ) > ,
259265 /// Execution metrics
260266 baseline_metrics : BaselineMetrics ,
267+ /// Count of file batches where short deletion-vector masks required padding.
268+ dv_short_mask_padded_files_total : Count ,
261269 /// Transforms to be applied to data read from individual files
262270 transforms : Arc < HashMap < String , ExpressionRef > > ,
263271 /// Selection vectors to be applied to data read from individual files
264272 selection_vectors : Arc < DashMap < String , Vec < bool > > > ,
265273 /// Column name for the file id
266274 file_id_field : Option < FieldRef > ,
275+ /// Cached schema adapter for efficient batch adaptation across batches
276+ schema_adapter : super :: SchemaAdapter ,
267277}
268278
269279impl DeltaScanMetaStream {
270280 /// Apply the per-file transformation to a RecordBatch.
271- fn batch_project ( & self , file_id : String , row_count : usize ) -> Result < RecordBatch > {
281+ fn batch_project ( & mut self , file_id : String , row_count : usize ) -> Result < RecordBatch > {
272282 static EMPTY_SCHEMA : LazyLock < SchemaRef > =
273283 LazyLock :: new ( || Arc :: new ( Schema :: new ( Fields :: empty ( ) ) ) ) ;
274284 static EMPTY_KERNEL_SCHEMA : LazyLock < KernelSchemaRef > =
@@ -283,17 +293,19 @@ impl DeltaScanMetaStream {
283293 ) ?;
284294
285295 let batch = if let Some ( selection) = self . selection_vectors . get ( & file_id) {
286- if selection. len ( ) != batch. num_rows ( ) {
287- return Err ( internal_datafusion_err ! (
288- "Selection vector length ({}) does not match row count ({}) for file '{}'. \
289- This indicates a bug in deletion vector processing.",
290- selection. len( ) ,
291- batch. num_rows( ) ,
292- file_id
293- ) ) ;
296+ let mask_len = selection. len ( ) ;
297+ let ( batch, padded_rows) = apply_selection_vector ( batch, selection. value ( ) , & file_id) ?;
298+ if padded_rows > 0 {
299+ self . dv_short_mask_padded_files_total . add ( 1 ) ;
300+ debug ! (
301+ file_id = file_id. as_str( ) ,
302+ mask_len,
303+ row_count,
304+ padded_rows,
305+ "Padded short deletion-vector keep-mask in metadata scan"
306+ ) ;
294307 }
295- let filter = BooleanArray :: from_iter ( selection. iter ( ) ) ;
296- filter_record_batch ( & batch, & filter) ?
308+ batch
297309 } else {
298310 batch
299311 } ;
@@ -343,13 +355,46 @@ impl DeltaScanMetaStream {
343355 result,
344356 & self . scan_plan ,
345357 Some ( ( Arc :: new ( file_id_array) , file_id_field. clone ( ) ) ) ,
358+ & mut self . schema_adapter ,
346359 )
347360 } else {
348- super :: finalize_transformed_batch ( result, & self . scan_plan , None )
361+ super :: finalize_transformed_batch (
362+ result,
363+ & self . scan_plan ,
364+ None ,
365+ & mut self . schema_adapter ,
366+ )
349367 }
350368 }
351369}
352370
371+ fn apply_selection_vector (
372+ batch : RecordBatch ,
373+ selection : & [ bool ] ,
374+ file_id : & str ,
375+ ) -> Result < ( RecordBatch , usize ) > {
376+ if selection. len ( ) > batch. num_rows ( ) {
377+ return Err ( internal_datafusion_err ! (
378+ "Selection vector length ({}) exceeds row count ({}) for file '{}'. \
379+ This indicates a bug in deletion vector processing.",
380+ selection. len( ) ,
381+ batch. num_rows( ) ,
382+ file_id
383+ ) ) ;
384+ }
385+
386+ let n_rows_to_pad = batch. num_rows ( ) - selection. len ( ) ;
387+ // Delta Kernel may emit short keep-masks; missing trailing entries are
388+ // implicitly `true` (row is kept).
389+ let filter = BooleanArray :: from_iter (
390+ selection
391+ . iter ( )
392+ . copied ( )
393+ . chain ( std:: iter:: repeat_n ( true , n_rows_to_pad) ) ,
394+ ) ;
395+ Ok ( ( filter_record_batch ( & batch, & filter) ?, n_rows_to_pad) )
396+ }
397+
353398impl Stream for DeltaScanMetaStream {
354399 type Item = Result < RecordBatch > ;
355400
@@ -377,6 +422,9 @@ impl RecordBatchStream for DeltaScanMetaStream {
377422mod tests {
378423 use std:: sync:: Arc ;
379424
425+ use arrow:: array:: RecordBatch ;
426+ use arrow_array:: RecordBatchOptions ;
427+ use arrow_schema:: { Fields , Schema } ;
380428 use datafusion:: {
381429 physical_plan:: collect_partitioned,
382430 prelude:: { col, lit} ,
@@ -389,6 +437,54 @@ mod tests {
389437 test_utils:: { TestResult , open_fs_path} ,
390438 } ;
391439
440+ #[ test]
441+ fn test_apply_selection_vector_short_mask_pads_with_true ( ) {
442+ let batch = RecordBatch :: try_new_with_options (
443+ Arc :: new ( Schema :: new ( Fields :: empty ( ) ) ) ,
444+ vec ! [ ] ,
445+ & RecordBatchOptions :: new ( ) . with_row_count ( Some ( 4 ) ) ,
446+ )
447+ . unwrap ( ) ;
448+
449+ // Short masks are valid: missing trailing entries are implicitly true.
450+ let ( filtered, padded_rows) =
451+ apply_selection_vector ( batch, & [ false , true ] , "file:///f.parquet" ) . unwrap ( ) ;
452+ assert_eq ! ( filtered. num_rows( ) , 3 ) ;
453+ assert_eq ! ( padded_rows, 2 ) ;
454+ }
455+
456+ #[ test]
457+ fn test_apply_selection_vector_no_padding_when_lengths_match ( ) {
458+ let batch = RecordBatch :: try_new_with_options (
459+ Arc :: new ( Schema :: new ( Fields :: empty ( ) ) ) ,
460+ vec ! [ ] ,
461+ & RecordBatchOptions :: new ( ) . with_row_count ( Some ( 2 ) ) ,
462+ )
463+ . unwrap ( ) ;
464+
465+ let ( filtered, padded_rows) =
466+ apply_selection_vector ( batch, & [ true , false ] , "file:///f.parquet" ) . unwrap ( ) ;
467+ assert_eq ! ( filtered. num_rows( ) , 1 ) ;
468+ assert_eq ! ( padded_rows, 0 ) ;
469+ }
470+
471+ #[ test]
472+ fn test_apply_selection_vector_longer_than_batch_errors ( ) {
473+ let batch = RecordBatch :: try_new_with_options (
474+ Arc :: new ( Schema :: new ( Fields :: empty ( ) ) ) ,
475+ vec ! [ ] ,
476+ & RecordBatchOptions :: new ( ) . with_row_count ( Some ( 2 ) ) ,
477+ )
478+ . unwrap ( ) ;
479+
480+ let err = apply_selection_vector ( batch, & [ true , true , true ] , "file:///f.parquet" )
481+ . expect_err ( "selection vector longer than row count must error" ) ;
482+ assert ! (
483+ err. to_string( ) . contains( "Selection vector length" ) ,
484+ "unexpected error: {err}"
485+ ) ;
486+ }
487+
392488 #[ tokio:: test]
393489 async fn test_meta_only_scan ( ) -> TestResult {
394490 let table = open_fs_path ( "../../dat/v0.0.3/reader_tests/generated/multi_partitioned/delta" ) ;
0 commit comments