@@ -26,6 +26,7 @@ use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
2626use datafusion:: physical_plan:: stream:: RecordBatchStreamAdapter ;
2727use datafusion:: physical_plan:: { DisplayAs , ExecutionPlan , Partitioning , PlanProperties } ;
2828use futures:: { StreamExt , TryStreamExt } ;
29+ use paimon:: spec:: Predicate ;
2930use paimon:: table:: Table ;
3031use paimon:: DataSplit ;
3132
@@ -41,6 +42,9 @@ pub struct PaimonTableScan {
4142 table : Table ,
4243 /// Projected column names (if None, reads all columns).
4344 projected_columns : Option < Vec < String > > ,
45+ /// Filter translated from DataFusion expressions and reused during execute()
46+ /// so reader-side pruning reaches the actual read path.
47+ pushed_predicate : Option < Predicate > ,
4448 /// Pre-planned partition assignments: `planned_partitions[i]` contains the
4549 /// Paimon splits that DataFusion partition `i` will read.
4650 /// Wrapped in `Arc` to avoid deep-cloning `DataSplit` metadata in `execute()`.
@@ -55,6 +59,7 @@ impl PaimonTableScan {
5559 schema : ArrowSchemaRef ,
5660 table : Table ,
5761 projected_columns : Option < Vec < String > > ,
62+ pushed_predicate : Option < Predicate > ,
5863 planned_partitions : Vec < Arc < [ DataSplit ] > > ,
5964 limit : Option < usize > ,
6065 ) -> Self {
@@ -67,6 +72,7 @@ impl PaimonTableScan {
6772 Self {
6873 table,
6974 projected_columns,
75+ pushed_predicate,
7076 planned_partitions,
7177 plan_properties,
7278 limit,
@@ -82,6 +88,11 @@ impl PaimonTableScan {
8288 & self . planned_partitions
8389 }
8490
91+ #[ cfg( test) ]
92+ pub ( crate ) fn pushed_predicate ( & self ) -> Option < & Predicate > {
93+ self . pushed_predicate . as_ref ( )
94+ }
95+
8596 pub fn limit ( & self ) -> Option < usize > {
8697 self . limit
8798 }
@@ -126,6 +137,7 @@ impl ExecutionPlan for PaimonTableScan {
126137 let table = self . table . clone ( ) ;
127138 let schema = self . schema ( ) ;
128139 let projected_columns = self . projected_columns . clone ( ) ;
140+ let pushed_predicate = self . pushed_predicate . clone ( ) ;
129141
130142 let fut = async move {
131143 let mut read_builder = table. new_read_builder ( ) ;
@@ -134,6 +146,9 @@ impl ExecutionPlan for PaimonTableScan {
134146 let col_refs: Vec < & str > = columns. iter ( ) . map ( |s| s. as_str ( ) ) . collect ( ) ;
135147 read_builder. with_projection ( & col_refs) ;
136148 }
149+ if let Some ( filter) = pushed_predicate {
150+ read_builder. with_filter ( filter) ;
151+ }
137152
138153 let read = read_builder. new_read ( ) . map_err ( to_datafusion_error) ?;
139154 let stream = read. to_arrow ( & splits) . map_err ( to_datafusion_error) ?;
@@ -173,11 +188,26 @@ impl DisplayAs for PaimonTableScan {
173188#[ cfg( test) ]
174189mod tests {
175190 use super :: * ;
176- use datafusion:: arrow:: datatypes:: { DataType as ArrowDataType , Field , Schema } ;
191+ mod test_utils {
192+ include ! ( concat!( env!( "CARGO_MANIFEST_DIR" ) , "/../../test_utils.rs" ) ) ;
193+ }
194+
195+ use datafusion:: arrow:: array:: Int32Array ;
196+ use datafusion:: arrow:: datatypes:: { DataType as ArrowDataType , Field , Schema as ArrowSchema } ;
177197 use datafusion:: physical_plan:: ExecutionPlan ;
198+ use datafusion:: prelude:: SessionContext ;
199+ use futures:: TryStreamExt ;
200+ use paimon:: catalog:: Identifier ;
201+ use paimon:: io:: FileIOBuilder ;
202+ use paimon:: spec:: {
203+ BinaryRow , DataType , Datum , IntType , PredicateBuilder , Schema as PaimonSchema , TableSchema ,
204+ } ;
205+ use std:: fs;
206+ use tempfile:: tempdir;
207+ use test_utils:: { local_file_path, test_data_file, write_int_parquet_file} ;
178208
179209 fn test_schema ( ) -> ArrowSchemaRef {
180- Arc :: new ( Schema :: new ( vec ! [ Field :: new(
210+ Arc :: new ( ArrowSchema :: new ( vec ! [ Field :: new(
181211 "id" ,
182212 ArrowDataType :: Int32 ,
183213 false ,
@@ -191,6 +221,7 @@ mod tests {
191221 schema,
192222 dummy_table ( ) ,
193223 None ,
224+ None ,
194225 vec ! [ Arc :: from( Vec :: new( ) ) ] ,
195226 None ,
196227 ) ;
@@ -205,19 +236,16 @@ mod tests {
205236 Arc :: from( Vec :: new( ) ) ,
206237 Arc :: from( Vec :: new( ) ) ,
207238 ] ;
208- let scan = PaimonTableScan :: new ( schema, dummy_table ( ) , None , planned_partitions, None ) ;
239+ let scan =
240+ PaimonTableScan :: new ( schema, dummy_table ( ) , None , None , planned_partitions, None ) ;
209241 assert_eq ! ( scan. properties( ) . output_partitioning( ) . partition_count( ) , 3 ) ;
210242 }
211243
212244 /// Constructs a minimal Table for testing (no real files needed since we
213245 /// only test PlanProperties, not actual reads).
214246 fn dummy_table ( ) -> Table {
215- use paimon:: catalog:: Identifier ;
216- use paimon:: io:: FileIOBuilder ;
217- use paimon:: spec:: { Schema , TableSchema } ;
218-
219247 let file_io = FileIOBuilder :: new ( "file" ) . build ( ) . unwrap ( ) ;
220- let schema = Schema :: builder ( ) . build ( ) . unwrap ( ) ;
248+ let schema = PaimonSchema :: builder ( ) . build ( ) . unwrap ( ) ;
221249 let table_schema = TableSchema :: new ( 0 , & schema) ;
222250 Table :: new (
223251 file_io,
@@ -226,4 +254,83 @@ mod tests {
226254 table_schema,
227255 )
228256 }
257+
258+ #[ tokio:: test]
259+ async fn test_execute_applies_pushed_filter_during_read ( ) {
260+ let tempdir = tempdir ( ) . unwrap ( ) ;
261+ let table_path = local_file_path ( tempdir. path ( ) ) ;
262+ let bucket_dir = tempdir. path ( ) . join ( "bucket-0" ) ;
263+ fs:: create_dir_all ( & bucket_dir) . unwrap ( ) ;
264+
265+ write_int_parquet_file (
266+ & bucket_dir. join ( "data.parquet" ) ,
267+ vec ! [ ( "id" , vec![ 1 , 2 , 3 , 4 ] ) , ( "value" , vec![ 5 , 20 , 30 , 40 ] ) ] ,
268+ Some ( 2 ) ,
269+ ) ;
270+
271+ let file_io = FileIOBuilder :: new ( "file" ) . build ( ) . unwrap ( ) ;
272+ let table_schema = TableSchema :: new (
273+ 0 ,
274+ & paimon:: spec:: Schema :: builder ( )
275+ . column ( "id" , DataType :: Int ( IntType :: new ( ) ) )
276+ . column ( "value" , DataType :: Int ( IntType :: new ( ) ) )
277+ . build ( )
278+ . unwrap ( ) ,
279+ ) ;
280+ let table = Table :: new (
281+ file_io,
282+ Identifier :: new ( "default" , "t" ) ,
283+ table_path,
284+ table_schema,
285+ ) ;
286+
287+ let split = paimon:: DataSplitBuilder :: new ( )
288+ . with_snapshot ( 1 )
289+ . with_partition ( BinaryRow :: new ( 0 ) )
290+ . with_bucket ( 0 )
291+ . with_bucket_path ( local_file_path ( & bucket_dir) )
292+ . with_total_buckets ( 1 )
293+ . with_data_files ( vec ! [ test_data_file( "data.parquet" , 4 ) ] )
294+ . with_raw_convertible ( true )
295+ . build ( )
296+ . unwrap ( ) ;
297+
298+ let pushed_predicate = PredicateBuilder :: new ( table. schema ( ) . fields ( ) )
299+ . greater_or_equal ( "value" , Datum :: Int ( 10 ) )
300+ . unwrap ( ) ;
301+
302+ let schema = Arc :: new ( ArrowSchema :: new ( vec ! [ Field :: new(
303+ "id" ,
304+ ArrowDataType :: Int32 ,
305+ false ,
306+ ) ] ) ) ;
307+ let scan = PaimonTableScan :: new (
308+ schema,
309+ table,
310+ Some ( vec ! [ "id" . to_string( ) ] ) ,
311+ Some ( pushed_predicate) ,
312+ vec ! [ Arc :: from( vec![ split] ) ] ,
313+ None ,
314+ ) ;
315+
316+ let ctx = SessionContext :: new ( ) ;
317+ let stream = scan
318+ . execute ( 0 , ctx. task_ctx ( ) )
319+ . expect ( "execute should succeed" ) ;
320+ let batches = stream. try_collect :: < Vec < _ > > ( ) . await . unwrap ( ) ;
321+
322+ let actual_ids: Vec < i32 > = batches
323+ . iter ( )
324+ . flat_map ( |batch| {
325+ let ids = batch
326+ . column ( 0 )
327+ . as_any ( )
328+ . downcast_ref :: < Int32Array > ( )
329+ . expect ( "id column should be Int32Array" ) ;
330+ ( 0 ..ids. len ( ) ) . map ( |idx| ids. value ( idx) ) . collect :: < Vec < _ > > ( )
331+ } )
332+ . collect ( ) ;
333+
334+ assert_eq ! ( actual_ids, vec![ 2 , 3 , 4 ] ) ;
335+ }
229336}
0 commit comments