@@ -33,8 +33,9 @@ use tokio::sync::{
3333 mpsc:: { self } ,
3434 RwLock , RwLockWriteGuard ,
3535} ;
36- use tracing:: instrument;
36+ use tracing:: { instrument, Instrument } ;
3737
38+ use crate :: statistics:: statistics_from_datafiles;
3839use crate :: {
3940 error:: Error as DataFusionIcebergError ,
4041 pruning_statistics:: { transform_predicate, PruneDataFiles , PruneManifests } ,
@@ -73,7 +74,6 @@ use datafusion::{
7374 projection:: ProjectionExec ,
7475 union:: UnionExec ,
7576 DisplayAs , DisplayFormatType , ExecutionPlan , PhysicalExpr , SendableRecordBatchStream ,
76- Statistics ,
7777 } ,
7878 prelude:: Expr ,
7979 scalar:: ScalarValue ,
@@ -259,16 +259,11 @@ impl TableProvider for DataFusionTable {
259259 }
260260 Tabular :: Table ( table) => {
261261 let schema = self . schema ( ) ;
262- let statistics = self
263- . statistics ( )
264- . await
265- . map_err ( DataFusionIcebergError :: from) ?;
266262 table_scan (
267263 table,
268264 & self . snapshot_range ,
269265 schema,
270266 self . config . as_ref ( ) ,
271- statistics,
272267 session_state,
273268 projection,
274269 filters,
@@ -282,16 +277,11 @@ impl TableProvider for DataFusionTable {
282277 . await
283278 . map_err ( DataFusionIcebergError :: from) ?;
284279 let schema = self . schema ( ) ;
285- let statistics = self
286- . statistics ( )
287- . await
288- . map_err ( DataFusionIcebergError :: from) ?;
289280 table_scan (
290281 & table,
291282 & self . snapshot_range ,
292283 schema,
293284 self . config . as_ref ( ) ,
294- statistics,
295285 session_state,
296286 projection,
297287 filters,
@@ -350,7 +340,7 @@ fn fake_object_store_url(table_location_url: &str) -> ObjectStoreUrl {
350340}
351341
352342#[ allow( clippy:: too_many_arguments) ]
353- #[ instrument( name = "datafusion_iceberg::table_scan" , level = "debug" , skip( arrow_schema, statistics , session, filters) , fields(
343+ #[ instrument( name = "datafusion_iceberg::table_scan" , level = "debug" , skip( arrow_schema, session, filters) , fields(
354344 table_identifier = %table. identifier( ) ,
355345 snapshot_range = ?snapshot_range,
356346 projection = ?projection,
@@ -362,7 +352,6 @@ async fn table_scan(
362352 snapshot_range : & ( Option < i64 > , Option < i64 > ) ,
363353 arrow_schema : SchemaRef ,
364354 config : Option < & DataFusionTableConfig > ,
365- statistics : Statistics ,
366355 session : & SessionState ,
367356 projection : Option < & Vec < usize > > ,
368357 filters : & [ Expr ] ,
@@ -445,7 +434,7 @@ async fn table_scan(
445434 HashMap :: new ( ) ;
446435
447436 // Prune data & delete file and insert them into the according map
448- if let Some ( physical_predicate) = physical_predicate. clone ( ) {
437+ let statistics = if let Some ( physical_predicate) = physical_predicate. clone ( ) {
449438 let partition_schema = Arc :: new ( ArrowSchema :: new ( table_partition_cols. clone ( ) ) ) ;
450439 let partition_column_names = partition_fields
451440 . iter ( )
@@ -492,15 +481,13 @@ async fn table_scan(
492481 . await
493482 . map_err ( DataFusionIcebergError :: from) ?
494483 . try_collect ( )
495- . await
496484 . map_err ( DataFusionIcebergError :: from) ?
497485 } else {
498486 table
499487 . datafiles ( & manifests, None , sequence_number_range)
500488 . await
501489 . map_err ( DataFusionIcebergError :: from) ?
502490 . try_collect ( )
503- . await
504491 . map_err ( DataFusionIcebergError :: from) ?
505492 } ;
506493
@@ -513,6 +500,8 @@ async fn table_scan(
513500 & data_files,
514501 ) ) ?;
515502
503+ let statistics = statistics_from_datafiles ( & schema, & data_files) ;
504+
516505 data_files
517506 . into_iter ( )
518507 . zip ( files_to_prune. into_iter ( ) )
@@ -537,18 +526,21 @@ async fn table_scan(
537526 }
538527 } ;
539528 } ) ;
529+ statistics
540530 } else {
541531 let manifests = table
542532 . manifests ( snapshot_range. 0 , snapshot_range. 1 )
543533 . await
544534 . map_err ( DataFusionIcebergError :: from) ?;
545- let data_files: Vec < ( ManifestPath , ManifestEntry ) > = table
535+ let data_files: Vec < _ > = table
546536 . datafiles ( & manifests, None , sequence_number_range)
547537 . await
548538 . map_err ( DataFusionIcebergError :: from) ?
549539 . try_collect ( )
550- . await
551540 . map_err ( DataFusionIcebergError :: from) ?;
541+
542+ let statistics = statistics_from_datafiles ( & schema, & data_files) ;
543+
552544 data_files. into_iter ( ) . for_each ( |manifest| {
553545 if * manifest. 1 . status ( ) != Status :: Deleted {
554546 match manifest. 1 . data_file ( ) . content ( ) {
@@ -570,6 +562,7 @@ async fn table_scan(
570562 }
571563 }
572564 } ) ;
565+ statistics
573566 } ;
574567
575568 let file_source = Arc :: new (
@@ -865,6 +858,9 @@ async fn table_scan(
865858
866859 let other_plan = ParquetFormat :: default ( )
867860 . create_physical_plan ( session, file_scan_config)
861+ . instrument ( tracing:: debug_span!(
862+ "datafusion_iceberg::create_physical_plan_scan_data_files"
863+ ) )
868864 . await ?;
869865
870866 plans. push ( other_plan) ;
0 commit comments