@@ -37,6 +37,7 @@ use datafusion_common::pruning::{
37
37
} ;
38
38
use datafusion_common:: { exec_err, Result } ;
39
39
use datafusion_datasource:: PartitionedFile ;
40
+ use datafusion_physical_expr:: PhysicalExprSchemaRewriter ;
40
41
use datafusion_physical_expr_common:: physical_expr:: PhysicalExpr ;
41
42
use datafusion_physical_optimizer:: pruning:: PruningPredicate ;
42
43
use datafusion_physical_plan:: metrics:: { Count , ExecutionPlanMetricsSet , MetricBuilder } ;
@@ -117,7 +118,6 @@ impl FileOpener for ParquetOpener {
117
118
118
119
let projected_schema =
119
120
SchemaRef :: from ( self . logical_file_schema . project ( & self . projection ) ?) ;
120
- let schema_adapter_factory = Arc :: clone ( & self . schema_adapter_factory ) ;
121
121
let schema_adapter = self
122
122
. schema_adapter_factory
123
123
. create ( projected_schema, Arc :: clone ( & self . logical_file_schema ) ) ;
@@ -159,7 +159,7 @@ impl FileOpener for ParquetOpener {
159
159
if let Some ( pruning_predicate) = pruning_predicate {
160
160
// The partition column schema is the schema of the table - the schema of the file
161
161
let mut pruning = Box :: new ( PartitionPruningStatistics :: try_new (
162
- vec ! [ file. partition_values] ,
162
+ vec ! [ file. partition_values. clone ( ) ] ,
163
163
partition_fields. clone ( ) ,
164
164
) ?)
165
165
as Box < dyn PruningStatistics > ;
@@ -248,10 +248,27 @@ impl FileOpener for ParquetOpener {
248
248
}
249
249
}
250
250
251
+ // Adapt the predicate to the physical file schema.
252
+ // This evaluates missing columns and inserts any necessary casts.
253
+ let predicate = predicate
254
+ . map ( |p| {
255
+ PhysicalExprSchemaRewriter :: new (
256
+ & physical_file_schema,
257
+ & logical_file_schema,
258
+ )
259
+ . with_partition_columns (
260
+ partition_fields. to_vec ( ) ,
261
+ file. partition_values ,
262
+ )
263
+ . rewrite ( p)
264
+ . map_err ( ArrowError :: from)
265
+ } )
266
+ . transpose ( ) ?;
267
+
251
268
// Build predicates for this specific file
252
269
let ( pruning_predicate, page_pruning_predicate) = build_pruning_predicates (
253
270
predicate. as_ref ( ) ,
254
- & logical_file_schema ,
271
+ & physical_file_schema ,
255
272
& predicate_creation_errors,
256
273
) ;
257
274
@@ -288,11 +305,9 @@ impl FileOpener for ParquetOpener {
288
305
let row_filter = row_filter:: build_row_filter (
289
306
& predicate,
290
307
& physical_file_schema,
291
- & logical_file_schema,
292
308
builder. metadata ( ) ,
293
309
reorder_predicates,
294
310
& file_metrics,
295
- & schema_adapter_factory,
296
311
) ;
297
312
298
313
match row_filter {
@@ -879,4 +894,115 @@ mod test {
879
894
assert_eq ! ( num_batches, 0 ) ;
880
895
assert_eq ! ( num_rows, 0 ) ;
881
896
}
897
+
898
+ #[ tokio:: test]
899
+ async fn test_prune_on_partition_value_and_data_value ( ) {
900
+ let store = Arc :: new ( InMemory :: new ( ) ) as Arc < dyn ObjectStore > ;
901
+
902
+ // Note: number 3 is missing!
903
+ let batch = record_batch ! ( ( "a" , Int32 , vec![ Some ( 1 ) , Some ( 2 ) , Some ( 4 ) ] ) ) . unwrap ( ) ;
904
+ let data_size =
905
+ write_parquet ( Arc :: clone ( & store) , "part=1/file.parquet" , batch. clone ( ) ) . await ;
906
+
907
+ let file_schema = batch. schema ( ) ;
908
+ let mut file = PartitionedFile :: new (
909
+ "part=1/file.parquet" . to_string ( ) ,
910
+ u64:: try_from ( data_size) . unwrap ( ) ,
911
+ ) ;
912
+ file. partition_values = vec ! [ ScalarValue :: Int32 ( Some ( 1 ) ) ] ;
913
+
914
+ let table_schema = Arc :: new ( Schema :: new ( vec ! [
915
+ Field :: new( "part" , DataType :: Int32 , false ) ,
916
+ Field :: new( "a" , DataType :: Int32 , false ) ,
917
+ ] ) ) ;
918
+
919
+ let make_opener = |predicate| {
920
+ ParquetOpener {
921
+ partition_index : 0 ,
922
+ projection : Arc :: new ( [ 0 ] ) ,
923
+ batch_size : 1024 ,
924
+ limit : None ,
925
+ predicate : Some ( predicate) ,
926
+ logical_file_schema : file_schema. clone ( ) ,
927
+ metadata_size_hint : None ,
928
+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
929
+ parquet_file_reader_factory : Arc :: new (
930
+ DefaultParquetFileReaderFactory :: new ( Arc :: clone ( & store) ) ,
931
+ ) ,
932
+ partition_fields : vec ! [ Arc :: new( Field :: new(
933
+ "part" ,
934
+ DataType :: Int32 ,
935
+ false ,
936
+ ) ) ] ,
937
+ pushdown_filters : true , // note that this is true!
938
+ reorder_filters : true ,
939
+ enable_page_index : false ,
940
+ enable_bloom_filter : false ,
941
+ schema_adapter_factory : Arc :: new ( DefaultSchemaAdapterFactory ) ,
942
+ enable_row_group_stats_pruning : false , // note that this is false!
943
+ coerce_int96 : None ,
944
+ }
945
+ } ;
946
+
947
+ let make_meta = || FileMeta {
948
+ object_meta : ObjectMeta {
949
+ location : Path :: from ( "part=1/file.parquet" ) ,
950
+ last_modified : Utc :: now ( ) ,
951
+ size : u64:: try_from ( data_size) . unwrap ( ) ,
952
+ e_tag : None ,
953
+ version : None ,
954
+ } ,
955
+ range : None ,
956
+ extensions : None ,
957
+ metadata_size_hint : None ,
958
+ } ;
959
+
960
+ // Filter should match the partition value and data value
961
+ let expr = col ( "part" ) . eq ( lit ( 1 ) ) . or ( col ( "a" ) . eq ( lit ( 1 ) ) ) ;
962
+ let predicate = logical2physical ( & expr, & table_schema) ;
963
+ let opener = make_opener ( predicate) ;
964
+ let stream = opener
965
+ . open ( make_meta ( ) , file. clone ( ) )
966
+ . unwrap ( )
967
+ . await
968
+ . unwrap ( ) ;
969
+ let ( num_batches, num_rows) = count_batches_and_rows ( stream) . await ;
970
+ assert_eq ! ( num_batches, 1 ) ;
971
+ assert_eq ! ( num_rows, 3 ) ;
972
+
973
+ // Filter should match the partition value but not the data value
974
+ let expr = col ( "part" ) . eq ( lit ( 1 ) ) . or ( col ( "a" ) . eq ( lit ( 3 ) ) ) ;
975
+ let predicate = logical2physical ( & expr, & table_schema) ;
976
+ let opener = make_opener ( predicate) ;
977
+ let stream = opener
978
+ . open ( make_meta ( ) , file. clone ( ) )
979
+ . unwrap ( )
980
+ . await
981
+ . unwrap ( ) ;
982
+ let ( num_batches, num_rows) = count_batches_and_rows ( stream) . await ;
983
+ assert_eq ! ( num_batches, 1 ) ;
984
+ assert_eq ! ( num_rows, 3 ) ;
985
+
986
+ // Filter should not match the partition value but match the data value
987
+ let expr = col ( "part" ) . eq ( lit ( 2 ) ) . or ( col ( "a" ) . eq ( lit ( 1 ) ) ) ;
988
+ let predicate = logical2physical ( & expr, & table_schema) ;
989
+ let opener = make_opener ( predicate) ;
990
+ let stream = opener
991
+ . open ( make_meta ( ) , file. clone ( ) )
992
+ . unwrap ( )
993
+ . await
994
+ . unwrap ( ) ;
995
+ let ( num_batches, num_rows) = count_batches_and_rows ( stream) . await ;
996
+ assert_eq ! ( num_batches, 1 ) ;
997
+ assert_eq ! ( num_rows, 1 ) ;
998
+
999
+ // Filter should not match the partition value or the data value
1000
+ let expr = col ( "part" ) . eq ( lit ( 2 ) ) . or ( col ( "a" ) . eq ( lit ( 3 ) ) ) ;
1001
+ let predicate = logical2physical ( & expr, & table_schema) ;
1002
+ let opener = make_opener ( predicate) ;
1003
+ let stream = opener. open ( make_meta ( ) , file) . unwrap ( ) . await . unwrap ( ) ;
1004
+ let ( num_batches, num_rows) = count_batches_and_rows ( stream) . await ;
1005
+ assert_eq ! ( num_batches, 0 ) ;
1006
+ assert_eq ! ( num_rows, 0 ) ;
1007
+ }
882
1008
}
0 commit comments