From b9819b4a8d3cf03a46203d683c816193e1055003 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Tue, 31 Mar 2026 13:24:56 +0200 Subject: [PATCH 01/16] feat(datafusion): add IcebergPartitionedTableProvider and IcebergPartitionedScan for parallel file scanning --- crates/integrations/datafusion/src/lib.rs | 2 + .../datafusion/src/physical_plan/mod.rs | 2 + .../src/physical_plan/partitioned_scan.rs | 174 ++++++++++ .../integrations/datafusion/src/table/mod.rs | 7 +- .../datafusion/src/table/partitioned.rs | 315 ++++++++++++++++++ 5 files changed, 499 insertions(+), 1 deletion(-) create mode 100644 crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs create mode 100644 crates/integrations/datafusion/src/table/partitioned.rs diff --git a/crates/integrations/datafusion/src/lib.rs b/crates/integrations/datafusion/src/lib.rs index 4b0ea8606d..9a84832d88 100644 --- a/crates/integrations/datafusion/src/lib.rs +++ b/crates/integrations/datafusion/src/lib.rs @@ -24,6 +24,8 @@ pub use error::*; pub mod physical_plan; mod schema; pub mod table; +pub use physical_plan::IcebergPartitionedScan; +pub use table::partitioned::IcebergPartitionedTableProvider; pub use table::table_provider_factory::IcebergTableProviderFactory; pub use table::*; diff --git a/crates/integrations/datafusion/src/physical_plan/mod.rs b/crates/integrations/datafusion/src/physical_plan/mod.rs index aeac30de32..a257fe9e20 100644 --- a/crates/integrations/datafusion/src/physical_plan/mod.rs +++ b/crates/integrations/datafusion/src/physical_plan/mod.rs @@ -18,6 +18,7 @@ pub(crate) mod commit; pub(crate) mod expr_to_predicate; pub(crate) mod metadata_scan; +pub(crate) mod partitioned_scan; pub(crate) mod project; pub(crate) mod repartition; pub(crate) mod scan; @@ -27,5 +28,6 @@ pub(crate) mod write; pub(crate) const DATA_FILES_COL_NAME: &str = "data_files"; pub use expr_to_predicate::convert_filters_to_predicate; +pub use partitioned_scan::IcebergPartitionedScan; pub use project::project_with_partition; pub use scan::IcebergTableScan; diff --git a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs new file mode 100644 index 0000000000..69d12b872b --- /dev/null +++ b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs @@ -0,0 +1,174 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::any::Any; +use std::sync::Arc; + +use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; +use datafusion::error::Result as DFResult; +use datafusion::execution::{SendableRecordBatchStream, TaskContext}; +use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{DisplayAs, ExecutionPlan, Partitioning, PlanProperties}; +use futures::TryStreamExt; +use iceberg::arrow::ArrowReaderBuilder; +use iceberg::io::FileIO; +use iceberg::scan::FileScanTask; + +use crate::to_datafusion_error; + +/// A DataFusion [`ExecutionPlan`] that reads one [`FileScanTask`] per partition. +/// +/// Display information (projection, predicate) is derived at runtime from the output schema and +/// the tasks rather than stored as dedicated struct fields. This keeps the node self-contained: +/// all state is already serializable via `FileScanTask`, which simplifies the DataFusion +/// distributed codec, adding dedicated fields would require encoding them separately in the +/// protobuf round-trip. +#[derive(Debug, Clone)] +pub struct IcebergPartitionedScan { + tasks: Vec, + file_io: FileIO, + plan_properties: Arc, +} + +impl IcebergPartitionedScan { + pub fn new(tasks: Vec, file_io: FileIO, schema: ArrowSchemaRef) -> Self { + let n_partitions = tasks.len(); + let plan_properties = Self::compute_properties(schema, n_partitions); + Self { + tasks, + file_io, + plan_properties, + } + } + + pub fn tasks(&self) -> &[FileScanTask] { + &self.tasks + } + + pub fn file_io(&self) -> &FileIO { + &self.file_io + } + + fn compute_properties(schema: ArrowSchemaRef, n_partitions: usize) -> Arc { + Arc::new(PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(n_partitions), + EmissionType::Incremental, + Boundedness::Bounded, + )) + } +} + +impl ExecutionPlan for IcebergPartitionedScan { + fn name(&self) -> &str { + "IcebergPartitionedScan" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _children: Vec>, + ) -> DFResult> { + Ok(self) + } + + fn properties(&self) -> &Arc { + &self.plan_properties + } + + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> DFResult { + let task = self.tasks.get(partition).cloned().ok_or_else(|| { + datafusion::error::DataFusionError::Internal(format!( + "{}: partition index {partition} is out of bounds \ + (total tasks: {})", + self.name(), + self.tasks.len() + )) + })?; + + let file_io = self.file_io.clone(); + + let fut = async move { + let task_stream = futures::stream::once(futures::future::ready(Ok(task))); + let record_batch_stream = ArrowReaderBuilder::new(file_io) + .build() + .read(Box::pin(task_stream)) + .map_err(to_datafusion_error)? + .map_err(to_datafusion_error); + Ok::<_, datafusion::error::DataFusionError>(record_batch_stream) + }; + + let stream = futures::stream::once(fut).try_flatten(); + + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + stream, + ))) + } +} + +impl DisplayAs for IcebergPartitionedScan { + fn fmt_as( + &self, + _t: datafusion::physical_plan::DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + let projection = self + .schema() + .fields() + .iter() + .map(|f| f.name().as_str()) + .collect::>() + .join(","); + // All tasks share the same predicate (they come from a single scan plan build), + // so reading it from the first task is sufficient. + let predicate = self + .tasks + .first() + .and_then(|t| t.predicate()) + .map_or(String::new(), |p| format!("{p}")); + let file_count = self.tasks.len(); + write!( + f, + "{} projection:[{projection}] predicate:[{predicate}] file_count:[{file_count}]", + self.name() + )?; + if self.tasks.len() <= 5 { + let files = self + .tasks + .iter() + .map(|t| t.data_file_path()) + .collect::>() + .join(", "); + write!(f, " files:[{files}]")?; + } + Ok(()) + } +} diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index 75b7988d8d..5ae41b86c1 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -17,15 +17,20 @@ //! Iceberg table providers for DataFusion. //! -//! This module provides two table provider implementations: +//! This module provides three table provider implementations: //! //! - [`IcebergTableProvider`]: Catalog-backed provider with automatic metadata refresh. //! Use for write operations and when you need to see the latest table state. //! //! - [`IcebergStaticTableProvider`]: Static provider for read-only access to a specific //! table snapshot. Use for consistent analytical queries or time-travel scenarios. +//! +//! - [`IcebergPartitionedTableProvider`]: Catalog-backed provider that assigns one +//! DataFusion partition per data file, enabling parallel file-level scanning. +//! Read-only; use [`IcebergTableProvider`] for write operations. pub mod metadata_table; +pub mod partitioned; pub mod table_provider_factory; use std::any::Any; diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs new file mode 100644 index 0000000000..2e9aa22628 --- /dev/null +++ b/crates/integrations/datafusion/src/table/partitioned.rs @@ -0,0 +1,315 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::any::Any; +use std::sync::Arc; + +use async_trait::async_trait; +use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; +use datafusion::catalog::Session; +use datafusion::common::DataFusionError; +use datafusion::datasource::{TableProvider, TableType}; +use datafusion::error::Result as DFResult; +use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; +use datafusion::physical_plan::ExecutionPlan; +use futures::TryStreamExt; +use iceberg::arrow::schema_to_arrow_schema; +use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; + +use crate::error::to_datafusion_error; +use crate::physical_plan::expr_to_predicate::convert_filters_to_predicate; +use crate::physical_plan::partitioned_scan::IcebergPartitionedScan; + +/// Catalog-backed table provider that scans each data file in a separate DataFusion partition. +/// +/// This provider reloads table metadata from the catalog on every [`scan`][Self::scan] call +/// to guarantee freshness, then issues one DataFusion partition per data file so that +/// DataFusion's scheduler can execute file reads in parallel. +/// +/// Write operations are not supported. Use [`IcebergTableProvider`] for write access. +/// +/// For consistent read-only access to a fixed snapshot without per-scan catalog overhead, +/// use [`IcebergStaticTableProvider`] instead. +#[derive(Debug, Clone)] +pub struct IcebergPartitionedTableProvider { + catalog: Arc, + table_ident: TableIdent, + schema: ArrowSchemaRef, +} + +impl IcebergPartitionedTableProvider { + pub async fn try_new( + catalog: Arc, + namespace: NamespaceIdent, + name: impl Into, + ) -> Result { + let table_ident = TableIdent::new(namespace, name.into()); + // First load: used only to snapshot the Arrow schema for DataFusion planning. + // A second load_table is issued at scan time to guarantee the freshest snapshot. + let table = catalog.load_table(&table_ident).await?; + let schema = Arc::new(schema_to_arrow_schema(table.metadata().current_schema())?); + Ok(Self { + catalog, + table_ident, + schema, + }) + } +} + +#[async_trait] +impl TableProvider for IcebergPartitionedTableProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> ArrowSchemaRef { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + _state: &dyn Session, + projection: Option<&Vec>, + filters: &[Expr], + _limit: Option, + ) -> DFResult> { + // Per-partition row limits are not yet implemented for IcebergPartitionedScan. + // DataFusion will apply a GlobalLimitExec on top of this node when needed. + + // Second load: fetch the latest snapshot so scans always reflect current table state. + let table = self + .catalog + .load_table(&self.table_ident) + .await + .map_err(to_datafusion_error)?; + + // Projection indices are resolved against self.schema (captured at try_new time), + // same as IcebergTableProvider / IcebergTableScan. + let col_names = projection.map(|indices| { + indices + .iter() + .map(|&i| self.schema.field(i).name().clone()) + .collect::>() + }); + + let predicate = convert_filters_to_predicate(filters); + + let mut builder = table.scan(); + builder = match col_names { + Some(names) => builder.select(names), + None => builder.select_all(), + }; + if let Some(pred) = predicate { + builder = builder.with_filter(pred); + } + + let tasks = builder + .build() + .map_err(to_datafusion_error)? + .plan_files() + .await + .map_err(to_datafusion_error)? + .try_collect::>() + .await + .map_err(to_datafusion_error)?; + + let output_schema = match projection { + None => self.schema.clone(), + Some(indices) => Arc::new(self.schema.project(indices).map_err(|e| { + DataFusionError::Internal(format!("schema projection failed: {e}")) + })?), + }; + + Ok(Arc::new(IcebergPartitionedScan::new( + tasks, + table.file_io().clone(), + output_schema, + ))) + } + + fn supports_filters_pushdown( + &self, + filters: &[&Expr], + ) -> DFResult> { + Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) + } + + async fn insert_into( + &self, + _state: &dyn Session, + _input: Arc, + _insert_op: datafusion::logical_expr::dml::InsertOp, + ) -> DFResult> { + Err(to_datafusion_error(Error::new( + ErrorKind::FeatureUnsupported, + "IcebergPartitionedTableProvider does not support writes; \ + use IcebergTableProvider instead", + ))) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use datafusion::prelude::SessionContext; + use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; + use iceberg::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, Type, + }; + use iceberg::transaction::{ApplyTransactionAction, Transaction}; + use iceberg::{Catalog, CatalogBuilder, NamespaceIdent, TableCreation, TableIdent}; + use tempfile::TempDir; + + use super::*; + + async fn make_catalog_and_table() -> (Arc, NamespaceIdent, String, TempDir) { + let temp_dir = TempDir::new().unwrap(); + let warehouse = temp_dir.path().to_str().unwrap().to_string(); + + let catalog = Arc::new( + MemoryCatalogBuilder::default() + .load( + "memory", + HashMap::from([(MEMORY_CATALOG_WAREHOUSE.to_string(), warehouse.clone())]), + ) + .await + .unwrap(), + ); + + let namespace = NamespaceIdent::new("ns".to_string()); + catalog + .create_namespace(&namespace, HashMap::new()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + catalog + .create_table( + &namespace, + TableCreation::builder() + .name("t".to_string()) + .location(format!("{warehouse}/t")) + .schema(schema) + .properties(HashMap::new()) + .build(), + ) + .await + .unwrap(); + + (catalog, namespace, "t".to_string(), temp_dir) + } + + /// Registers `n` synthetic data files in the table metadata via the iceberg + /// transaction API. No actual parquet files are written, only the metadata + /// entries that `plan_files()` reads are created. + async fn append_fake_data_files( + catalog: &Arc, + namespace: &NamespaceIdent, + table_name: &str, + n: usize, + ) { + let table = catalog + .load_table(&TableIdent::new(namespace.clone(), table_name.to_string())) + .await + .unwrap(); + + let data_files = (0..n) + .map(|i| { + DataFileBuilder::default() + .content(DataContentType::Data) + .file_path(format!( + "{}/data/fake_{i}.parquet", + table.metadata().location() + )) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(128) + .record_count(1) + .partition_spec_id(table.metadata().default_partition_spec_id()) + .build() + .unwrap() + }) + .collect::>(); + + let tx = Transaction::new(&table); + let action = tx.fast_append().add_data_files(data_files); + action + .apply(tx) + .unwrap() + .commit(catalog.as_ref()) + .await + .unwrap(); + } + + /// An empty table must produce a zero-partition scan so DataFusion never calls + /// execute(0), which would otherwise return an out-of-bounds error. + #[tokio::test] + async fn test_empty_table_zero_partitions() { + let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; + // no files appended + let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&SessionContext::new().state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(scan.tasks().len(), 0); + assert_eq!(scan.properties().partitioning.partition_count(), 0); + } + + /// Each data file in the table must become exactly one DataFusion partition + /// in IcebergPartitionedScan, enabling parallel file reads. + #[tokio::test] + async fn test_one_partition_per_file() { + let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; + append_fake_data_files(&catalog, &namespace, &table_name, 3).await; + + let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&SessionContext::new().state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(scan.tasks().len(), 3); + assert_eq!(scan.properties().partitioning.partition_count(), 3); + } +} From c362b0fb28b7b939fe1944797900c2dfe7ca2488 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Tue, 31 Mar 2026 16:40:57 +0200 Subject: [PATCH 02/16] docs(datafusion): update comment in IcebergPartitionedScan --- .../datafusion/src/physical_plan/partitioned_scan.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs index 69d12b872b..5083f068b2 100644 --- a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs @@ -35,10 +35,7 @@ use crate::to_datafusion_error; /// A DataFusion [`ExecutionPlan`] that reads one [`FileScanTask`] per partition. /// /// Display information (projection, predicate) is derived at runtime from the output schema and -/// the tasks rather than stored as dedicated struct fields. This keeps the node self-contained: -/// all state is already serializable via `FileScanTask`, which simplifies the DataFusion -/// distributed codec, adding dedicated fields would require encoding them separately in the -/// protobuf round-trip. +/// the tasks rather than stored as dedicated struct fields. #[derive(Debug, Clone)] pub struct IcebergPartitionedScan { tasks: Vec, From ec1bd371cbea6620c182341802d886ea6108d4bd Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Mon, 20 Apr 2026 14:45:58 +0200 Subject: [PATCH 03/16] Update crates/integrations/datafusion/src/table/mod.rs Co-authored-by: Tim Saucer --- crates/integrations/datafusion/src/table/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index 5ae41b86c1..e2c9ca6efa 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -17,7 +17,7 @@ //! Iceberg table providers for DataFusion. //! -//! This module provides three table provider implementations: +//! This module provides various table provider implementations: //! //! - [`IcebergTableProvider`]: Catalog-backed provider with automatic metadata refresh. //! Use for write operations and when you need to see the latest table state. From 5e53cb8a6ae9a8e523dffff4c39c86ee4d189cc3 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Mon, 20 Apr 2026 15:25:59 +0200 Subject: [PATCH 04/16] fix(datafusion): reject non-empty children in IcebergPartitionedScan::with_new_children --- .../datafusion/src/physical_plan/partitioned_scan.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs index 5083f068b2..e0e2fd272c 100644 --- a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs @@ -87,8 +87,15 @@ impl ExecutionPlan for IcebergPartitionedScan { fn with_new_children( self: Arc, - _children: Vec>, + children: Vec>, ) -> DFResult> { + if !children.is_empty() { + return Err(datafusion::error::DataFusionError::Internal(format!( + "{} is a leaf node and expects no children, but {} were provided", + self.name(), + children.len() + ))); + } Ok(self) } From cc6a833016239aa94065860de4e91ef671457236 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Tue, 21 Apr 2026 11:37:28 +0200 Subject: [PATCH 05/16] fix(datafusion): use ArrowReaderBuilder existing configuration path --- crates/iceberg/src/scan/mod.rs | 31 +++- .../src/physical_plan/partitioned_scan.rs | 140 +++++++++++++----- .../datafusion/src/physical_plan/scan.rs | 2 +- .../datafusion/src/table/partitioned.rs | 19 +-- 4 files changed, 136 insertions(+), 56 deletions(-) diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 27f479183a..c46378a20e 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -433,6 +433,32 @@ impl TableScan { /// Returns an [`ArrowRecordBatchStream`]. pub async fn to_arrow(&self) -> Result { + self.to_arrow_with_tasks(self.plan_files().await?) + } + + /// Consumes an externally-planned [`FileScanTask`] stream and returns an + /// [`ArrowRecordBatchStream`] using this scan's [`ArrowReaderBuilder`] + /// configuration (row-group filtering, row selection, data-file + /// concurrency limit, batch size). + /// + /// Equivalent to [`TableScan::to_arrow`] — which delegates to this method + /// after awaiting [`TableScan::plan_files`] — but lets the caller supply + /// a pre-computed task stream. This decouples planning from reading, so + /// external executors (e.g. a DataFusion partitioned scan) can plan once, + /// distribute tasks across workers, and replay them here without + /// re-running `plan_files()`. + /// + /// # Correctness + /// + /// The tasks passed in must have been produced by a [`TableScan`] whose + /// projection and filter match `self`: filters are already baked into + /// each [`FileScanTask::predicate`] at planning time and are not + /// re-applied here. Using tasks from a scan with a different projection + /// or filter yields undefined behavior. + pub fn to_arrow_with_tasks( + &self, + tasks: FileScanTaskStream, + ) -> Result { let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) .with_data_file_concurrency_limit(self.concurrency_limit_data_files) .with_row_group_filtering_enabled(self.row_group_filtering_enabled) @@ -442,10 +468,7 @@ impl TableScan { arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); } - arrow_reader_builder - .build() - .read(self.plan_files().await?) - .map(|result| result.stream()) + arrow_reader_builder.build().read(tasks) } /// Returns a reference to the column names of the table scan. diff --git a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs index e0e2fd272c..bf6a1d3465 100644 --- a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs @@ -19,47 +19,99 @@ use std::any::Any; use std::sync::Arc; use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; -use datafusion::error::Result as DFResult; +use datafusion::error::{DataFusionError, Result as DFResult}; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{DisplayAs, ExecutionPlan, Partitioning, PlanProperties}; +use datafusion::prelude::Expr; use futures::TryStreamExt; -use iceberg::arrow::ArrowReaderBuilder; -use iceberg::io::FileIO; +use iceberg::expr::Predicate; use iceberg::scan::FileScanTask; +use iceberg::table::Table; +use super::expr_to_predicate::convert_filters_to_predicate; +use super::scan::get_column_names; use crate::to_datafusion_error; /// A DataFusion [`ExecutionPlan`] that reads one [`FileScanTask`] per partition. /// -/// Display information (projection, predicate) is derived at runtime from the output schema and -/// the tasks rather than stored as dedicated struct fields. -#[derive(Debug, Clone)] +/// Arrow reader configuration (row-group filtering, row selection, concurrency +/// limit, batch size) matches [`IcebergTableScan`][super::scan::IcebergTableScan]: +/// it is sourced from the underlying [`TableScan`][iceberg::scan::TableScan] +/// rebuilt in [`execute`](ExecutionPlan::execute) and applied via +/// [`TableScan::to_arrow_with_tasks`][iceberg::scan::TableScan::to_arrow_with_tasks]. +/// +/// Note: the `TableScan` is rebuilt on every `execute(partition)` call rather +/// than cached as an `Arc` on the struct. Caching would avoid +/// redundant schema resolution and predicate binding per partition, but +/// `TableScan` carries a `PlanContext` with `Arc`-shared evaluator caches +/// which is awkward to serialize if this plan ever needs to be shipped across +/// workers. The per-build cost is bounded (no I/O), so the rebuild is kept +/// for now; revisit once the cross-worker story is clearer. +#[derive(Debug)] pub struct IcebergPartitionedScan { - tasks: Vec, - file_io: FileIO, + /// A table in the catalog. + table: Table, + /// Snapshot of the table to scan. + snapshot_id: Option, + /// Stores certain, often expensive to compute, + /// plan properties used in query optimization. plan_properties: Arc, + /// Projection column names, None means all columns. + projection: Option>, + /// Filters to apply to the table scan. + predicates: Option, + /// Pre-planned file scan tasks, one per DataFusion partition. + tasks: Vec, } impl IcebergPartitionedScan { - pub fn new(tasks: Vec, file_io: FileIO, schema: ArrowSchemaRef) -> Self { - let n_partitions = tasks.len(); - let plan_properties = Self::compute_properties(schema, n_partitions); + pub(crate) fn new( + table: Table, + snapshot_id: Option, + schema: ArrowSchemaRef, + projection: Option<&Vec>, + filters: &[Expr], + tasks: Vec, + ) -> Self { + let output_schema = match projection { + None => schema.clone(), + Some(projection) => Arc::new(schema.project(projection).unwrap()), + }; + let plan_properties = Self::compute_properties(output_schema, tasks.len()); + let projection = get_column_names(schema, projection); + let predicates = convert_filters_to_predicate(filters); + Self { - tasks, - file_io, + table, + snapshot_id, plan_properties, + projection, + predicates, + tasks, } } - pub fn tasks(&self) -> &[FileScanTask] { - &self.tasks + pub fn table(&self) -> &Table { + &self.table } - pub fn file_io(&self) -> &FileIO { - &self.file_io + pub fn snapshot_id(&self) -> Option { + self.snapshot_id + } + + pub fn projection(&self) -> Option<&[String]> { + self.projection.as_deref() + } + + pub fn predicates(&self) -> Option<&Predicate> { + self.predicates.as_ref() + } + + pub fn tasks(&self) -> &[FileScanTask] { + &self.tasks } fn compute_properties(schema: ArrowSchemaRef, n_partitions: usize) -> Arc { @@ -90,7 +142,7 @@ impl ExecutionPlan for IcebergPartitionedScan { children: Vec>, ) -> DFResult> { if !children.is_empty() { - return Err(datafusion::error::DataFusionError::Internal(format!( + return Err(DataFusionError::Internal(format!( "{} is a leaf node and expects no children, but {} were provided", self.name(), children.len() @@ -109,24 +161,40 @@ impl ExecutionPlan for IcebergPartitionedScan { _context: Arc, ) -> DFResult { let task = self.tasks.get(partition).cloned().ok_or_else(|| { - datafusion::error::DataFusionError::Internal(format!( - "{}: partition index {partition} is out of bounds \ - (total tasks: {})", + DataFusionError::Internal(format!( + "{}: partition index {partition} is out of bounds (total tasks: {})", self.name(), self.tasks.len() )) })?; - let file_io = self.file_io.clone(); + let table = self.table.clone(); + let snapshot_id = self.snapshot_id; + let column_names = self.projection.clone(); + let predicates = self.predicates.clone(); let fut = async move { - let task_stream = futures::stream::once(futures::future::ready(Ok(task))); - let record_batch_stream = ArrowReaderBuilder::new(file_io) - .build() - .read(Box::pin(task_stream)) + // Rebuild a TableScan mirroring IcebergTableScan::get_batch_stream so we + // inherit the same defaults (row-group filtering, batch size, concurrency, ...). + let scan_builder = match snapshot_id { + Some(id) => table.scan().snapshot_id(id), + None => table.scan(), + }; + let mut scan_builder = match column_names { + Some(names) => scan_builder.select(names), + None => scan_builder.select_all(), + }; + if let Some(pred) = predicates { + scan_builder = scan_builder.with_filter(pred); + } + let table_scan = scan_builder.build().map_err(to_datafusion_error)?; + + let task_stream = Box::pin(futures::stream::once(futures::future::ready(Ok(task)))); + let record_batch_stream = table_scan + .to_arrow_with_tasks(task_stream) .map_err(to_datafusion_error)? .map_err(to_datafusion_error); - Ok::<_, datafusion::error::DataFusionError>(record_batch_stream) + Ok::<_, DataFusionError>(record_batch_stream) }; let stream = futures::stream::once(fut).try_flatten(); @@ -145,18 +213,12 @@ impl DisplayAs for IcebergPartitionedScan { f: &mut std::fmt::Formatter, ) -> std::fmt::Result { let projection = self - .schema() - .fields() - .iter() - .map(|f| f.name().as_str()) - .collect::>() - .join(","); - // All tasks share the same predicate (they come from a single scan plan build), - // so reading it from the first task is sufficient. + .projection + .clone() + .map_or(String::new(), |v| v.join(",")); let predicate = self - .tasks - .first() - .and_then(|t| t.predicate()) + .predicates + .clone() .map_or(String::new(), |p| format!("{p}")); let file_count = self.tasks.len(); write!( @@ -164,7 +226,7 @@ impl DisplayAs for IcebergPartitionedScan { "{} projection:[{projection}] predicate:[{predicate}] file_count:[{file_count}]", self.name() )?; - if self.tasks.len() <= 5 { + if file_count <= 5 { let files = self .tasks .iter() diff --git a/crates/integrations/datafusion/src/physical_plan/scan.rs b/crates/integrations/datafusion/src/physical_plan/scan.rs index 36539ae503..c5a892abeb 100644 --- a/crates/integrations/datafusion/src/physical_plan/scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/scan.rs @@ -237,7 +237,7 @@ async fn get_batch_stream( Ok(Box::pin(stream)) } -fn get_column_names( +pub(super) fn get_column_names( schema: ArrowSchemaRef, projection: Option<&Vec>, ) -> Option> { diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs index 2e9aa22628..00e08efb94 100644 --- a/crates/integrations/datafusion/src/table/partitioned.rs +++ b/crates/integrations/datafusion/src/table/partitioned.rs @@ -21,7 +21,6 @@ use std::sync::Arc; use async_trait::async_trait; use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; use datafusion::catalog::Session; -use datafusion::common::DataFusionError; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::Result as DFResult; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; @@ -101,8 +100,8 @@ impl TableProvider for IcebergPartitionedTableProvider { .await .map_err(to_datafusion_error)?; - // Projection indices are resolved against self.schema (captured at try_new time), - // same as IcebergTableProvider / IcebergTableScan. + // Build a TableScan mirroring the inputs we'll hand to IcebergPartitionedScan, + // so plan_files() uses the same projection/filters the scan will replay in execute(). let col_names = projection.map(|indices| { indices .iter() @@ -131,17 +130,13 @@ impl TableProvider for IcebergPartitionedTableProvider { .await .map_err(to_datafusion_error)?; - let output_schema = match projection { - None => self.schema.clone(), - Some(indices) => Arc::new(self.schema.project(indices).map_err(|e| { - DataFusionError::Internal(format!("schema projection failed: {e}")) - })?), - }; - Ok(Arc::new(IcebergPartitionedScan::new( + table, + None, // Always use current snapshot for catalog-backed provider + self.schema.clone(), + projection, + filters, tasks, - table.file_io().clone(), - output_schema, ))) } From 75e521dac2aa00f3b590f617f76be45fcd59ca56 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Tue, 21 Apr 2026 11:44:12 +0200 Subject: [PATCH 06/16] format --- crates/iceberg/src/scan/mod.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index c46378a20e..eff1bce3ff 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -455,10 +455,7 @@ impl TableScan { /// each [`FileScanTask::predicate`] at planning time and are not /// re-applied here. Using tasks from a scan with a different projection /// or filter yields undefined behavior. - pub fn to_arrow_with_tasks( - &self, - tasks: FileScanTaskStream, - ) -> Result { + pub fn to_arrow_with_tasks(&self, tasks: FileScanTaskStream) -> Result { let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) .with_data_file_concurrency_limit(self.concurrency_limit_data_files) .with_row_group_filtering_enabled(self.row_group_filtering_enabled) From 9edd54a71c01e9a596d1997e4ebf0784a2c31e3e Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 24 Apr 2026 19:24:09 -0400 Subject: [PATCH 07/16] feat(datafusion): bucket FileScanTasks across target_partitions with identity-hash partitioning Replace the one-task-per-partition layout in IcebergPartitionedScan with N buckets sized from the session's target_partitions. When the table's default spec exposes identity-transform columns and every task carries the corresponding partition values, tasks are bucketed by hashing those values via DataFusion's REPARTITION_RANDOM_STATE so the resulting partitioning matches what RepartitionExec would produce. The scan then declares Partitioning::Hash(exprs, N), letting downstream joins and aggregates skip an extra repartition. Hash declaration is conservative and only stands when: - the table has a single partition spec (no spec evolution) - every identity source column is present in the output projection - every column type is supported by literal_to_array - every task supplied a full identity key Any miss collapses to UnknownPartitioning(N) while bucketing falls back to a hash of data_file_path so partitions still distribute. IcebergPartitionedScan now stores Vec> and execute(i) streams every task in buckets[i] through to_arrow_with_tasks. Bucket count is capped at min(target_partitions, num_files), and an empty table still yields zero partitions to avoid out-of-bounds execute calls. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../src/physical_plan/partitioned_scan.rs | 59 ++-- .../datafusion/src/table/partitioned.rs | 291 +++++++++++++++++- 2 files changed, 312 insertions(+), 38 deletions(-) diff --git a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs index bf6a1d3465..159665dddd 100644 --- a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs @@ -35,7 +35,14 @@ use super::expr_to_predicate::convert_filters_to_predicate; use super::scan::get_column_names; use crate::to_datafusion_error; -/// A DataFusion [`ExecutionPlan`] that reads one [`FileScanTask`] per partition. +/// A DataFusion [`ExecutionPlan`] that reads a bucket of [`FileScanTask`]s per partition. +/// +/// Each DataFusion partition `i` streams every [`FileScanTask`] in `buckets[i]`, +/// concatenated into a single Arrow record-batch stream. The caller decides how +/// tasks are assigned to buckets and supplies the resulting [`Partitioning`] +/// (typically [`Partitioning::Hash`] when files are bucketed by identity-partition +/// values matching DataFusion's repartition hash, otherwise +/// [`Partitioning::UnknownPartitioning`]). /// /// Arrow reader configuration (row-group filtering, row selection, concurrency /// limit, batch size) matches [`IcebergTableScan`][super::scan::IcebergTableScan]: @@ -63,8 +70,9 @@ pub struct IcebergPartitionedScan { projection: Option>, /// Filters to apply to the table scan. predicates: Option, - /// Pre-planned file scan tasks, one per DataFusion partition. - tasks: Vec, + /// Pre-planned file scan tasks grouped by output DataFusion partition. + /// `buckets[i]` holds every task that `execute(i)` will read. + buckets: Vec>, } impl IcebergPartitionedScan { @@ -74,13 +82,19 @@ impl IcebergPartitionedScan { schema: ArrowSchemaRef, projection: Option<&Vec>, filters: &[Expr], - tasks: Vec, + buckets: Vec>, + partitioning: Partitioning, ) -> Self { let output_schema = match projection { None => schema.clone(), Some(projection) => Arc::new(schema.project(projection).unwrap()), }; - let plan_properties = Self::compute_properties(output_schema, tasks.len()); + let plan_properties = Arc::new(PlanProperties::new( + EquivalenceProperties::new(output_schema), + partitioning, + EmissionType::Incremental, + Boundedness::Bounded, + )); let projection = get_column_names(schema, projection); let predicates = convert_filters_to_predicate(filters); @@ -90,7 +104,7 @@ impl IcebergPartitionedScan { plan_properties, projection, predicates, - tasks, + buckets, } } @@ -110,17 +124,12 @@ impl IcebergPartitionedScan { self.predicates.as_ref() } - pub fn tasks(&self) -> &[FileScanTask] { - &self.tasks + pub fn buckets(&self) -> &[Vec] { + &self.buckets } - fn compute_properties(schema: ArrowSchemaRef, n_partitions: usize) -> Arc { - Arc::new(PlanProperties::new( - EquivalenceProperties::new(schema), - Partitioning::UnknownPartitioning(n_partitions), - EmissionType::Incremental, - Boundedness::Bounded, - )) + fn total_file_count(&self) -> usize { + self.buckets.iter().map(|b| b.len()).sum() } } @@ -160,11 +169,11 @@ impl ExecutionPlan for IcebergPartitionedScan { partition: usize, _context: Arc, ) -> DFResult { - let task = self.tasks.get(partition).cloned().ok_or_else(|| { + let bucket = self.buckets.get(partition).cloned().ok_or_else(|| { DataFusionError::Internal(format!( - "{}: partition index {partition} is out of bounds (total tasks: {})", + "{}: partition index {partition} is out of bounds (total buckets: {})", self.name(), - self.tasks.len() + self.buckets.len() )) })?; @@ -189,7 +198,9 @@ impl ExecutionPlan for IcebergPartitionedScan { } let table_scan = scan_builder.build().map_err(to_datafusion_error)?; - let task_stream = Box::pin(futures::stream::once(futures::future::ready(Ok(task)))); + let task_stream = Box::pin(futures::stream::iter( + bucket.into_iter().map(Ok::<_, iceberg::Error>), + )); let record_batch_stream = table_scan .to_arrow_with_tasks(task_stream) .map_err(to_datafusion_error)? @@ -220,17 +231,19 @@ impl DisplayAs for IcebergPartitionedScan { .predicates .clone() .map_or(String::new(), |p| format!("{p}")); - let file_count = self.tasks.len(); + let file_count = self.total_file_count(); + let bucket_count = self.buckets.len(); write!( f, - "{} projection:[{projection}] predicate:[{predicate}] file_count:[{file_count}]", + "{} projection:[{projection}] predicate:[{predicate}] \ + buckets:[{bucket_count}] file_count:[{file_count}]", self.name() )?; if file_count <= 5 { let files = self - .tasks + .buckets .iter() - .map(|t| t.data_file_path()) + .flat_map(|b| b.iter().map(|t| t.data_file_path())) .collect::>() .join(", "); write!(f, " files:[{files}]")?; diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs index 00e08efb94..580cf68a47 100644 --- a/crates/integrations/datafusion/src/table/partitioned.rs +++ b/crates/integrations/datafusion/src/table/partitioned.rs @@ -19,14 +19,25 @@ use std::any::Any; use std::sync::Arc; use async_trait::async_trait; -use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; +use datafusion::arrow::array::{ + ArrayRef, BooleanArray, Date32Array, Float32Array, Float64Array, Int32Array, Int64Array, + StringArray, +}; +use datafusion::arrow::datatypes::{DataType, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use datafusion::catalog::Session; +use datafusion::common::hash_utils::create_hashes; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::Result as DFResult; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; -use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_expr::expressions::Column; +use datafusion::physical_plan::repartition::REPARTITION_RANDOM_STATE; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; use futures::TryStreamExt; use iceberg::arrow::schema_to_arrow_schema; +use iceberg::scan::FileScanTask; +use iceberg::spec::{Literal, PrimitiveLiteral, Transform}; +use iceberg::table::Table; use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; use crate::error::to_datafusion_error; @@ -85,7 +96,7 @@ impl TableProvider for IcebergPartitionedTableProvider { async fn scan( &self, - _state: &dyn Session, + state: &dyn Session, projection: Option<&Vec>, filters: &[Expr], _limit: Option, @@ -120,7 +131,7 @@ impl TableProvider for IcebergPartitionedTableProvider { builder = builder.with_filter(pred); } - let tasks = builder + let tasks: Vec = builder .build() .map_err(to_datafusion_error)? .plan_files() @@ -130,13 +141,51 @@ impl TableProvider for IcebergPartitionedTableProvider { .await .map_err(to_datafusion_error)?; + // Output schema after projection: column indices in `Hash` exprs and any + // Arrow array we hash must reference this schema, not the full table schema. + let output_schema = match projection { + None => self.schema.clone(), + Some(p) => Arc::new(self.schema.project(p).map_err(|e| { + to_datafusion_error(Error::new(ErrorKind::DataInvalid, e.to_string())) + })?), + }; + + let target_partitions = state.config().target_partitions(); + let n_partitions = if tasks.is_empty() { + 0 + } else { + target_partitions.min(tasks.len()).max(1) + }; + + // identity_cols is Some(non-empty) iff every condition for declaring + // Partitioning::Hash is met: the table's default spec has identity-transform + // fields, every such source column is present in the output projection, and + // every column type is supported by literal_to_array. Any miss collapses to + // None, which forces UnknownPartitioning regardless of bucketing strategy. + let identity_cols = compute_identity_cols(&table, &output_schema); + + let (buckets, all_had_full_key) = + bucket_tasks(tasks, n_partitions, identity_cols.as_deref()); + + let partitioning = match identity_cols { + Some(cols) if !cols.is_empty() && all_had_full_key && n_partitions > 0 => { + let exprs: Vec> = cols + .iter() + .map(|c| Arc::new(Column::new(&c.name, c.output_idx)) as Arc) + .collect(); + Partitioning::Hash(exprs, n_partitions) + } + _ => Partitioning::UnknownPartitioning(n_partitions), + }; + Ok(Arc::new(IcebergPartitionedScan::new( table, None, // Always use current snapshot for catalog-backed provider self.schema.clone(), projection, filters, - tasks, + buckets, + partitioning, ))) } @@ -161,12 +210,167 @@ impl TableProvider for IcebergPartitionedTableProvider { } } +/// Identity-partitioned column that is also present in the output projection +/// and whose Arrow type can be reconstructed from a `Literal` for hashing. +struct IdentityCol { + name: String, + /// Position of this column in the *output* schema (after projection). + output_idx: usize, + /// Position of this column inside the partition spec's `fields()` slice, + /// matching the slot order of `FileScanTask::partition`. + spec_field_idx: usize, + output_dtype: DataType, +} + +/// Inspect the table's default partition spec and return the list of identity +/// columns that can support a [`Partitioning::Hash`] declaration. Returns +/// `None` if any condition is violated: +/// - the source column for an identity field is not in the output projection +/// - the source column's Arrow type is not currently supported by +/// [`literal_to_array`] +/// - the table has spec evolution (>1 historical specs), since older files +/// may carry a partition tuple that does not align with the default spec +/// +/// Returning `None` forces the scan to declare `UnknownPartitioning` even if +/// bucketing succeeds. +fn compute_identity_cols(table: &Table, output_schema: &ArrowSchema) -> Option> { + let metadata = table.metadata(); + if metadata.partition_specs_iter().len() > 1 { + return None; + } + let spec = metadata.default_partition_spec(); + let table_schema = metadata.current_schema(); + + let mut cols = Vec::new(); + for (spec_field_idx, pf) in spec.fields().iter().enumerate() { + if pf.transform != Transform::Identity { + continue; + } + let source_field = table_schema.field_by_id(pf.source_id)?; + let output_idx = output_schema.index_of(source_field.name.as_str()).ok()?; + let output_dtype = output_schema.field(output_idx).data_type().clone(); + if !is_supported_dtype(&output_dtype) { + return None; + } + cols.push(IdentityCol { + name: source_field.name.clone(), + output_idx, + spec_field_idx, + output_dtype, + }); + } + Some(cols) +} + +fn is_supported_dtype(dt: &DataType) -> bool { + matches!( + dt, + DataType::Boolean + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Utf8 + | DataType::Date32 + ) +} + +/// Distribute `tasks` across `n_partitions` buckets. When `identity_cols` +/// describes a non-empty, hashable identity key, each task is hashed on +/// that key using DataFusion's repartition hash so the resulting partitioning +/// matches what `RepartitionExec` would produce on the same data. Tasks +/// missing partition data fall back to hashing `data_file_path`, which still +/// distributes evenly but breaks the `Hash` contract — the second tuple +/// element flags whether every task supplied a full identity key. +fn bucket_tasks( + tasks: Vec, + n_partitions: usize, + identity_cols: Option<&[IdentityCol]>, +) -> (Vec>, bool) { + if n_partitions == 0 { + return (Vec::new(), tasks.is_empty()); + } + let mut buckets: Vec> = (0..n_partitions).map(|_| Vec::new()).collect(); + let mut all_full_key = true; + let cols = identity_cols.unwrap_or(&[]); + + for task in tasks { + let bucket_idx = match identity_hash(&task, cols) { + Some(h) => (h % n_partitions as u64) as usize, + None => { + all_full_key = false; + fallback_hash(&task) as usize % n_partitions + } + }; + buckets[bucket_idx].push(task); + } + (buckets, all_full_key) +} + +/// Hash the identity-partition values of `task` using +/// [`REPARTITION_RANDOM_STATE`] so the bucket assignment matches DataFusion's +/// hash-repartition convention. Returns `None` if the task lacks partition +/// data or any required slot is null/unsupported. +fn identity_hash(task: &FileScanTask, cols: &[IdentityCol]) -> Option { + if cols.is_empty() { + return None; + } + let partition = task.partition.as_ref()?; + let mut arrays: Vec = Vec::with_capacity(cols.len()); + for col in cols { + let lit = partition.fields().get(col.spec_field_idx)?.as_ref()?; + arrays.push(literal_to_array(lit, &col.output_dtype)?); + } + let mut hashes = vec![0u64; 1]; + create_hashes( + &arrays, + REPARTITION_RANDOM_STATE.random_state(), + &mut hashes, + ) + .ok()?; + Some(hashes[0]) +} + +/// Deterministic per-file fallback used when `identity_hash` cannot produce a +/// bucket. The hash function does not need to match DataFusion's because any +/// task taking this path causes the scan to drop to `UnknownPartitioning`. +fn fallback_hash(task: &FileScanTask) -> u64 { + use std::collections::hash_map::DefaultHasher; + use std::hash::{Hash, Hasher}; + let mut hasher = DefaultHasher::new(); + task.data_file_path.hash(&mut hasher); + hasher.finish() +} + +/// Materialize a single-element Arrow array of `dt` holding the value of +/// `lit`. The Arrow type must match what DataFusion will see for this column +/// at scan time, otherwise `create_hashes` would dispatch on a different type +/// and produce a hash that disagrees with DataFusion's row-wise hashing. +fn literal_to_array(lit: &Literal, dt: &DataType) -> Option { + let prim = match lit { + Literal::Primitive(p) => p, + _ => return None, + }; + Some(match (prim, dt) { + (PrimitiveLiteral::Boolean(v), DataType::Boolean) => Arc::new(BooleanArray::from(vec![*v])), + (PrimitiveLiteral::Int(v), DataType::Int32) => Arc::new(Int32Array::from(vec![*v])), + (PrimitiveLiteral::Int(v), DataType::Date32) => Arc::new(Date32Array::from(vec![*v])), + (PrimitiveLiteral::Long(v), DataType::Int64) => Arc::new(Int64Array::from(vec![*v])), + (PrimitiveLiteral::Float(v), DataType::Float32) => Arc::new(Float32Array::from(vec![v.0])), + (PrimitiveLiteral::Double(v), DataType::Float64) => Arc::new(Float64Array::from(vec![v.0])), + (PrimitiveLiteral::String(v), DataType::Utf8) => { + Arc::new(StringArray::from(vec![v.as_str()])) + } + _ => return None, + }) +} + #[cfg(test)] mod tests { use std::collections::HashMap; use std::sync::Arc; - use datafusion::prelude::SessionContext; + use datafusion::prelude::{SessionConfig, SessionContext}; use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; use iceberg::spec::{ DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, Type, @@ -263,6 +467,10 @@ mod tests { .unwrap(); } + fn ctx_with_target_partitions(n: usize) -> SessionContext { + SessionContext::new_with_config(SessionConfig::new().with_target_partitions(n)) + } + /// An empty table must produce a zero-partition scan so DataFusion never calls /// execute(0), which would otherwise return an out-of-bounds error. #[tokio::test] @@ -273,7 +481,7 @@ mod tests { .await .unwrap(); let plan = provider - .scan(&SessionContext::new().state(), None, &[], None) + .scan(&ctx_with_target_partitions(8).state(), None, &[], None) .await .unwrap(); let scan = plan @@ -281,22 +489,75 @@ mod tests { .downcast_ref::() .unwrap(); - assert_eq!(scan.tasks().len(), 0); + assert_eq!(scan.buckets().len(), 0); assert_eq!(scan.properties().partitioning.partition_count(), 0); } - /// Each data file in the table must become exactly one DataFusion partition - /// in IcebergPartitionedScan, enabling parallel file reads. + /// When the table has no identity-partition columns, every task takes the + /// fallback (file_path) bucket path, so the declaration must drop to + /// `UnknownPartitioning`. The bucket count should still equal + /// min(target_partitions, num_files). + #[tokio::test] + async fn test_unpartitioned_falls_back_to_unknown() { + let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; + append_fake_data_files(&catalog, &namespace, &table_name, 5).await; + + let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&ctx_with_target_partitions(3).state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + let total_files: usize = scan.buckets().iter().map(|b| b.len()).sum(); + assert_eq!(total_files, 5); + assert_eq!(scan.buckets().len(), 3); + assert!(matches!( + scan.properties().partitioning, + Partitioning::UnknownPartitioning(3) + )); + } + + /// Bucket count must be capped at the number of files: spinning up more + /// DataFusion partitions than there are tasks would just leave empty + /// streams, wasting scheduler slots. + #[tokio::test] + async fn test_bucket_count_capped_at_file_count() { + let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; + append_fake_data_files(&catalog, &namespace, &table_name, 2).await; + + let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&ctx_with_target_partitions(16).state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(scan.buckets().len(), 2); + } + + /// target_partitions = 1 collapses every task into a single bucket, giving + /// the same execution profile as `IcebergTableScan`. #[tokio::test] - async fn test_one_partition_per_file() { + async fn test_single_target_partition_single_bucket() { let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; - append_fake_data_files(&catalog, &namespace, &table_name, 3).await; + append_fake_data_files(&catalog, &namespace, &table_name, 4).await; let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) .await .unwrap(); let plan = provider - .scan(&SessionContext::new().state(), None, &[], None) + .scan(&ctx_with_target_partitions(1).state(), None, &[], None) .await .unwrap(); let scan = plan @@ -304,7 +565,7 @@ mod tests { .downcast_ref::() .unwrap(); - assert_eq!(scan.tasks().len(), 3); - assert_eq!(scan.properties().partitioning.partition_count(), 3); + assert_eq!(scan.buckets().len(), 1); + assert_eq!(scan.buckets()[0].len(), 4); } } From add5e35c5488fb71d828de59393ee1c171ed541c Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 24 Apr 2026 19:37:22 -0400 Subject: [PATCH 08/16] feat(datafusion): mark identity-partition filters as Exact pushdown `IcebergPartitionedTableProvider::supports_filters_pushdown` previously returned `Inexact` for every filter, forcing DataFusion to re-evaluate even filters that Iceberg's manifest-level pruning has fully resolved. Per-filter the provider now returns `Exact` when both: - the iceberg conversion can represent the filter, so manifest pruning will remove every row that fails it, and - every leaf is a comparison or null check against an identity- partition column with a literal RHS. Identity-partitioned column names are cached at `try_new` from the table's default spec; tables with spec evolution (>1 historical specs) fall back to an empty set so all filters stay `Inexact`. Supported shapes: =, !=, <, <=, >, >=, IS NULL, IS NOT NULL, IN/NOT IN, plus AND/OR/NOT compositions of the above. Every other shape is `Inexact`. `convert_filter_to_predicate` is promoted to `pub(crate)` so the provider can probe convertibility per filter without rebuilding the whole AND-collapsed predicate. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../src/physical_plan/expr_to_predicate.rs | 2 +- .../datafusion/src/table/partitioned.rs | 249 +++++++++++++++++- 2 files changed, 246 insertions(+), 5 deletions(-) diff --git a/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs b/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs index 17c9416d54..b426b1228f 100644 --- a/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs +++ b/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs @@ -49,7 +49,7 @@ pub fn convert_filters_to_predicate(filters: &[Expr]) -> Option { .reduce(Predicate::and) } -fn convert_filter_to_predicate(expr: &Expr) -> Option { +pub(crate) fn convert_filter_to_predicate(expr: &Expr) -> Option { match to_iceberg_predicate(expr) { TransformedResult::Predicate(predicate) => Some(predicate), TransformedResult::Column(column) => { diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs index 580cf68a47..34b8cf18f6 100644 --- a/crates/integrations/datafusion/src/table/partitioned.rs +++ b/crates/integrations/datafusion/src/table/partitioned.rs @@ -16,6 +16,7 @@ // under the License. use std::any::Any; +use std::collections::HashSet; use std::sync::Arc; use async_trait::async_trait; @@ -28,7 +29,7 @@ use datafusion::catalog::Session; use datafusion::common::hash_utils::create_hashes; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::Result as DFResult; -use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; +use datafusion::logical_expr::{Expr, Operator, TableProviderFilterPushDown}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_expr::expressions::Column; use datafusion::physical_plan::repartition::REPARTITION_RANDOM_STATE; @@ -41,7 +42,9 @@ use iceberg::table::Table; use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; use crate::error::to_datafusion_error; -use crate::physical_plan::expr_to_predicate::convert_filters_to_predicate; +use crate::physical_plan::expr_to_predicate::{ + convert_filter_to_predicate, convert_filters_to_predicate, +}; use crate::physical_plan::partitioned_scan::IcebergPartitionedScan; /// Catalog-backed table provider that scans each data file in a separate DataFusion partition. @@ -59,6 +62,17 @@ pub struct IcebergPartitionedTableProvider { catalog: Arc, table_ident: TableIdent, schema: ArrowSchemaRef, + /// Source-column names that are identity-partitioned in the table's + /// default spec, captured at construction. Used by + /// `supports_filters_pushdown` to mark filters as `Exact` when they + /// only reference identity-partition columns. Empty when the table + /// has spec evolution (>1 historical specs) or no identity transforms, + /// which forces every filter to `Inexact`. + /// + /// This is a snapshot: if the table's default spec changes between + /// `try_new` and a later scan, the cached set may be stale. Spec + /// evolution is rare in practice and the next `try_new` will refresh. + identity_partition_cols: HashSet, } impl IcebergPartitionedTableProvider { @@ -72,10 +86,12 @@ impl IcebergPartitionedTableProvider { // A second load_table is issued at scan time to guarantee the freshest snapshot. let table = catalog.load_table(&table_ident).await?; let schema = Arc::new(schema_to_arrow_schema(table.metadata().current_schema())?); + let identity_partition_cols = identity_partition_col_names(&table); Ok(Self { catalog, table_ident, schema, + identity_partition_cols, }) } } @@ -193,7 +209,24 @@ impl TableProvider for IcebergPartitionedTableProvider { &self, filters: &[&Expr], ) -> DFResult> { - Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) + Ok(filters + .iter() + .map(|f| { + // `Exact` is only safe when (1) the filter touches nothing but + // identity-partition columns and operators preserved by the + // identity transform, and (2) the iceberg conversion can + // actually represent the filter, so manifest pruning will + // remove every row that fails it. Either miss falls back to + // `Inexact` and DataFusion adds a FilterExec on top. + if convert_filter_to_predicate(f).is_some() + && is_exact_on_identity(f, &self.identity_partition_cols) + { + TableProviderFilterPushDown::Exact + } else { + TableProviderFilterPushDown::Inexact + } + }) + .collect()) } async fn insert_into( @@ -342,6 +375,76 @@ fn fallback_hash(task: &FileScanTask) -> u64 { hasher.finish() } +/// Source-column names of every identity-transform field in the table's +/// default partition spec. Returns the empty set when the table has spec +/// evolution (>1 historical specs) — older files may carry partition tuples +/// whose identity status differs from the current spec, so the safe choice +/// is to refuse all `Exact` pushdowns until each task carries its own spec. +fn identity_partition_col_names(table: &Table) -> HashSet { + let metadata = table.metadata(); + if metadata.partition_specs_iter().len() > 1 { + return HashSet::new(); + } + let spec = metadata.default_partition_spec(); + let table_schema = metadata.current_schema(); + let mut names = HashSet::new(); + for pf in spec.fields() { + if pf.transform != Transform::Identity { + continue; + } + if let Some(field) = table_schema.field_by_id(pf.source_id) { + names.insert(field.name.clone()); + } + } + names +} + +/// Returns `true` when every leaf of `expr` is a comparison or null check +/// against an identity-partition column. Such filters are fully resolvable +/// by manifest-level partition pruning, so DataFusion does not need to +/// re-apply them post-scan. +/// +/// Safe operators: `=`, `!=`, `<`, `<=`, `>`, `>=`, `IS NULL`, `IS NOT NULL`, +/// `IN (..)`, `NOT IN (..)`, plus `AND` / `OR` / `NOT` of any of those. Every +/// other shape returns `false` (caller falls back to `Inexact`). +fn is_exact_on_identity(expr: &Expr, cols: &HashSet) -> bool { + if cols.is_empty() { + return false; + } + match expr { + Expr::BinaryExpr(b) => match b.op { + Operator::And | Operator::Or => { + is_exact_on_identity(&b.left, cols) && is_exact_on_identity(&b.right, cols) + } + Operator::Eq + | Operator::NotEq + | Operator::Lt + | Operator::LtEq + | Operator::Gt + | Operator::GtEq => is_simple_compare_on_identity(&b.left, &b.right, cols), + _ => false, + }, + Expr::Not(inner) => is_exact_on_identity(inner, cols), + Expr::IsNull(inner) | Expr::IsNotNull(inner) => is_identity_col(inner, cols), + Expr::InList(l) => { + is_identity_col(&l.expr, cols) && l.list.iter().all(|e| matches!(e, Expr::Literal(..))) + } + _ => false, + } +} + +fn is_simple_compare_on_identity(l: &Expr, r: &Expr, cols: &HashSet) -> bool { + let l_col = is_identity_col(l, cols); + let r_col = is_identity_col(r, cols); + let l_lit = matches!(l, Expr::Literal(..)); + let r_lit = matches!(r, Expr::Literal(..)); + (l_col && r_lit) || (r_col && l_lit) +} + +fn is_identity_col(e: &Expr, cols: &HashSet) -> bool { + matches!(e, Expr::Column(c) if cols.contains(&c.name)) +} + /// Materialize a single-element Arrow array of `dt` holding the value of /// `lit`. The Arrow type must match what DataFusion will see for this column /// at scan time, otherwise `create_hashes` would dispatch on a different type @@ -370,10 +473,12 @@ mod tests { use std::collections::HashMap; use std::sync::Arc; + use datafusion::logical_expr::{col, lit}; use datafusion::prelude::{SessionConfig, SessionContext}; use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; use iceberg::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, Type, + DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, + Transform, Type, UnboundPartitionSpec, }; use iceberg::transaction::{ApplyTransactionAction, Transaction}; use iceberg::{Catalog, CatalogBuilder, NamespaceIdent, TableCreation, TableIdent}; @@ -381,6 +486,58 @@ mod tests { use super::*; + async fn make_catalog_and_partitioned_table( + partition_spec: Option, + ) -> (Arc, NamespaceIdent, String, TempDir) { + let temp_dir = TempDir::new().unwrap(); + let warehouse = temp_dir.path().to_str().unwrap().to_string(); + + let catalog = Arc::new( + MemoryCatalogBuilder::default() + .load( + "memory", + HashMap::from([(MEMORY_CATALOG_WAREHOUSE.to_string(), warehouse.clone())]), + ) + .await + .unwrap(), + ); + + let namespace = NamespaceIdent::new("ns".to_string()); + catalog + .create_namespace(&namespace, HashMap::new()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + let creation = match partition_spec { + Some(spec) => TableCreation::builder() + .name("t".to_string()) + .location(format!("{warehouse}/t")) + .schema(schema) + .partition_spec(spec) + .properties(HashMap::new()) + .build(), + None => TableCreation::builder() + .name("t".to_string()) + .location(format!("{warehouse}/t")) + .schema(schema) + .properties(HashMap::new()) + .build(), + }; + + catalog.create_table(&namespace, creation).await.unwrap(); + + (catalog, namespace, "t".to_string(), temp_dir) + } + async fn make_catalog_and_table() -> (Arc, NamespaceIdent, String, TempDir) { let temp_dir = TempDir::new().unwrap(); let warehouse = temp_dir.path().to_str().unwrap().to_string(); @@ -546,6 +703,90 @@ mod tests { assert_eq!(scan.buckets().len(), 2); } + /// Filters that only touch identity-partition columns with literal RHS + /// can be marked `Exact` because Iceberg's manifest-level pruning already + /// removes every file whose partition value fails the predicate. + #[tokio::test] + async fn test_pushdown_exact_on_identity_column() { + let spec = UnboundPartitionSpec::builder() + .add_partition_field(1, "id_part", Transform::Identity) + .unwrap() + .build(); + let (catalog, namespace, table_name, _temp_dir) = + make_catalog_and_partitioned_table(Some(spec)).await; + let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + + let f_eq = col("id").eq(lit(5_i32)); + let f_neq = col("id").not_eq(lit(5_i32)); + let f_isnull = col("id").is_null(); + let f_and = col("id").eq(lit(5_i32)).and(col("id").lt(lit(10_i32))); + + let supports = provider + .supports_filters_pushdown(&[&f_eq, &f_neq, &f_isnull, &f_and]) + .unwrap(); + for (i, s) in supports.iter().enumerate() { + assert!( + matches!(s, TableProviderFilterPushDown::Exact), + "filter index {i} should be Exact, got {s:?}" + ); + } + } + + /// Filters touching non-partition columns or columns with non-identity + /// transforms must remain `Inexact`: the partition value is either + /// missing or lossy (bucket/truncate/etc.), so DataFusion still needs to + /// re-apply the filter against actual row values. + #[tokio::test] + async fn test_pushdown_inexact_on_non_identity_column() { + let spec = UnboundPartitionSpec::builder() + .add_partition_field(1, "id_part", Transform::Identity) + .unwrap() + .build(); + let (catalog, namespace, table_name, _temp_dir) = + make_catalog_and_partitioned_table(Some(spec)).await; + let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + + // `name` is not partitioned — manifest pruning cannot eliminate files + // by it, so the filter must re-execute post-scan. + let f_name = col("name").eq(lit("alice")); + // Mixed AND: even though `id` is identity-partitioned, the `name` arm + // is not exact, so the whole expression is Inexact. + let f_mixed = col("id").eq(lit(5_i32)).and(col("name").eq(lit("alice"))); + + let supports = provider + .supports_filters_pushdown(&[&f_name, &f_mixed]) + .unwrap(); + for (i, s) in supports.iter().enumerate() { + assert!( + matches!(s, TableProviderFilterPushDown::Inexact), + "filter index {i} should be Inexact, got {s:?}" + ); + } + } + + /// Unpartitioned tables must mark every filter `Inexact` regardless of + /// shape: there is no partition pruning that could make the scan + /// authoritative. + #[tokio::test] + async fn test_pushdown_unpartitioned_table_all_inexact() { + let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; + let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + + let f_id = col("id").eq(lit(5_i32)); + let f_name = col("name").eq(lit("alice")); + let supports = provider + .supports_filters_pushdown(&[&f_id, &f_name]) + .unwrap(); + assert!(matches!(supports[0], TableProviderFilterPushDown::Inexact)); + assert!(matches!(supports[1], TableProviderFilterPushDown::Inexact)); + } + /// target_partitions = 1 collapses every task into a single bucket, giving /// the same execution profile as `IcebergTableScan`. #[tokio::test] From e8771e45fa16522b77cc4299a40a99964900cced Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 24 Apr 2026 19:46:27 -0400 Subject: [PATCH 09/16] feat(datafusion): allow Exact pushdown across spec evolution via per-column intersection Previously identity_partition_col_names returned an empty set whenever the table had more than one historical partition spec, forcing every filter back to Inexact under spec evolution. This was overly conservative: Iceberg evaluates partition predicates against each manifest's own spec, so a column that is identity-partitioned in every spec is fully prunable across the entire table regardless of which spec a given file was written under. Replace the multi-spec gate with an intersection across every spec's identity-source set. A column survives only if every spec includes it with Transform::Identity; columns that appear with non-identity transforms in some spec, or are missing from a spec entirely, are dropped. The result remains an honest set of columns for which Exact pushdown is provably safe across all surviving files. Hash bucketing (compute_identity_cols) keeps its single-spec gate because slot-order alignment with the table's default spec depends on each task carrying its own spec id, which the native plan flow does not yet do. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../datafusion/src/table/partitioned.rs | 202 +++++++++++++++--- 1 file changed, 173 insertions(+), 29 deletions(-) diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs index 34b8cf18f6..ba2ee2b615 100644 --- a/crates/integrations/datafusion/src/table/partitioned.rs +++ b/crates/integrations/datafusion/src/table/partitioned.rs @@ -37,7 +37,7 @@ use datafusion::physical_plan::{ExecutionPlan, Partitioning}; use futures::TryStreamExt; use iceberg::arrow::schema_to_arrow_schema; use iceberg::scan::FileScanTask; -use iceberg::spec::{Literal, PrimitiveLiteral, Transform}; +use iceberg::spec::{Literal, PartitionSpec, PrimitiveLiteral, Transform}; use iceberg::table::Table; use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; @@ -62,16 +62,21 @@ pub struct IcebergPartitionedTableProvider { catalog: Arc, table_ident: TableIdent, schema: ArrowSchemaRef, - /// Source-column names that are identity-partitioned in the table's - /// default spec, captured at construction. Used by - /// `supports_filters_pushdown` to mark filters as `Exact` when they - /// only reference identity-partition columns. Empty when the table - /// has spec evolution (>1 historical specs) or no identity transforms, - /// which forces every filter to `Inexact`. + /// Source-column names that are identity-partitioned in *every* historical + /// partition spec of the table, captured at construction. Used by + /// `supports_filters_pushdown` to mark filters as `Exact` when they only + /// reference these columns: Iceberg evaluates partition predicates against + /// each manifest's own spec, so a column that is identity-partitioned in + /// every spec is fully prunable across the full table regardless of which + /// spec a given file was written under. /// - /// This is a snapshot: if the table's default spec changes between - /// `try_new` and a later scan, the cached set may be stale. Spec - /// evolution is rare in practice and the next `try_new` will refresh. + /// Columns that appear in some specs with non-identity transforms + /// (`bucket`, `truncate`, `year`/`month`/etc.), or that are missing from + /// any spec entirely, are dropped from the set — those files cannot be + /// pruned exactly, so DataFusion must keep its FilterExec. + /// + /// This is a snapshot: if the table's specs change between `try_new` and + /// a later scan, the cached set may be stale. The next `try_new` refreshes. identity_partition_cols: HashSet, } @@ -375,28 +380,44 @@ fn fallback_hash(task: &FileScanTask) -> u64 { hasher.finish() } -/// Source-column names of every identity-transform field in the table's -/// default partition spec. Returns the empty set when the table has spec -/// evolution (>1 historical specs) — older files may carry partition tuples -/// whose identity status differs from the current spec, so the safe choice -/// is to refuse all `Exact` pushdowns until each task carries its own spec. +/// Intersection of identity-partitioned source-column names across every +/// historical partition spec. A column is included only if every spec +/// includes that column with `Transform::Identity`; any spec where the column +/// is absent or has a non-identity transform drops it from the result. +/// +/// Why intersection: Iceberg evaluates partition predicates against each +/// manifest's own spec. A file written under spec A can only be exactly +/// pruned by columns identity-partitioned in spec A. To guarantee Exact +/// pushdown for *every* file in the table, the column must be identity in +/// *every* spec. Otherwise some surviving files would still need DataFusion's +/// FilterExec to enforce the predicate. fn identity_partition_col_names(table: &Table) -> HashSet { let metadata = table.metadata(); - if metadata.partition_specs_iter().len() > 1 { - return HashSet::new(); - } - let spec = metadata.default_partition_spec(); let table_schema = metadata.current_schema(); - let mut names = HashSet::new(); - for pf in spec.fields() { - if pf.transform != Transform::Identity { - continue; - } - if let Some(field) = table_schema.field_by_id(pf.source_id) { - names.insert(field.name.clone()); + let identity_set = |spec: &PartitionSpec| -> HashSet { + spec.fields() + .iter() + .filter(|pf| pf.transform == Transform::Identity) + .filter_map(|pf| { + table_schema + .field_by_id(pf.source_id) + .map(|f| f.name.clone()) + }) + .collect() + }; + + let mut iter = metadata.partition_specs_iter(); + let Some(first) = iter.next() else { + return HashSet::new(); + }; + let mut acc = identity_set(first); + for spec in iter { + if acc.is_empty() { + break; } + acc = acc.intersection(&identity_set(spec)).cloned().collect(); } - names + acc } /// Returns `true` when every leaf of `expr` is a comparison or null check @@ -475,11 +496,14 @@ mod tests { use datafusion::logical_expr::{col, lit}; use datafusion::prelude::{SessionConfig, SessionContext}; + use iceberg::io::FileIO; use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; use iceberg::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, - Transform, Type, UnboundPartitionSpec, + DataContentType, DataFileBuilder, DataFileFormat, FormatVersion, NestedField, + PrimitiveType, Schema, SortOrder, TableMetadataBuilder, Transform, Type, + UnboundPartitionSpec, }; + use iceberg::table::Table; use iceberg::transaction::{ApplyTransactionAction, Transaction}; use iceberg::{Catalog, CatalogBuilder, NamespaceIdent, TableCreation, TableIdent}; use tempfile::TempDir; @@ -628,6 +652,126 @@ mod tests { SessionContext::new_with_config(SessionConfig::new().with_target_partitions(n)) } + /// Build a `Table` carrying `specs.len()` historical partition specs. The + /// first spec is the table's initial spec; each subsequent spec is added + /// via `into_builder().add_partition_spec(...)`. No catalog round-trip, + /// no real I/O — `FileIO::new_with_memory()` is sufficient because the + /// helper under test only reads metadata. + fn build_table_with_specs(specs: Vec) -> Table { + assert!(!specs.is_empty(), "need at least one spec"); + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + let mut iter = specs.into_iter(); + let first = iter.next().unwrap(); + let mut metadata = TableMetadataBuilder::new( + schema, + first, + SortOrder::unsorted_order(), + "memory:///t".to_string(), + FormatVersion::V2, + HashMap::new(), + ) + .unwrap() + .build() + .unwrap() + .metadata; + + for spec in iter { + metadata = metadata + .into_builder(None) + .add_partition_spec(spec) + .unwrap() + .build() + .unwrap() + .metadata; + } + + Table::builder() + .file_io(FileIO::new_with_memory()) + .metadata(Arc::new(metadata)) + .identifier(TableIdent::new( + NamespaceIdent::new("ns".to_string()), + "t".to_string(), + )) + .build() + .unwrap() + } + + /// Multi-spec table where every historical spec keeps `id` as identity: + /// the column survives the intersection and remains Exact-pushdown safe. + /// `name` is never identity-partitioned, so it is excluded. + #[test] + fn test_identity_cols_preserved_across_compatible_specs() { + let spec_v0 = UnboundPartitionSpec::builder() + .add_partition_field(1, "id_part", Transform::Identity) + .unwrap() + .build(); + // Evolved spec: still identity on `id`, plus a non-identity transform + // on `name`. The latter must not pollute the result. + let spec_v1 = UnboundPartitionSpec::builder() + .add_partition_field(1, "id_part", Transform::Identity) + .unwrap() + .add_partition_field(2, "name_bucket", Transform::Bucket(8)) + .unwrap() + .build(); + let table = build_table_with_specs(vec![spec_v0, spec_v1]); + + let cols = identity_partition_col_names(&table); + assert_eq!(cols, HashSet::from(["id".to_string()])); + } + + /// Multi-spec table where the evolved spec replaces `identity(id)` with + /// `bucket(id)`. Files written under the evolved spec cannot be exactly + /// pruned on `id`, so `id` must be dropped from the Exact-safe set. + #[test] + fn test_identity_cols_dropped_when_transform_changes() { + let spec_v0 = UnboundPartitionSpec::builder() + .add_partition_field(1, "id_part", Transform::Identity) + .unwrap() + .build(); + let spec_v1 = UnboundPartitionSpec::builder() + .add_partition_field(1, "id_bucket", Transform::Bucket(8)) + .unwrap() + .build(); + let table = build_table_with_specs(vec![spec_v0, spec_v1]); + + let cols = identity_partition_col_names(&table); + assert!( + cols.is_empty(), + "expected empty set after non-identity replacement, got {cols:?}" + ); + } + + /// Multi-spec table where the second spec omits `id` from partitioning + /// entirely. Files under that spec carry no `id` partition tuple, so + /// pruning is a no-op for them — `id` must be dropped from the + /// Exact-safe set. + #[test] + fn test_identity_cols_dropped_when_column_missing_from_some_spec() { + let spec_v0 = UnboundPartitionSpec::builder() + .add_partition_field(1, "id_part", Transform::Identity) + .unwrap() + .build(); + // Evolved spec only partitions on `name`, omitting `id`. + let spec_v1 = UnboundPartitionSpec::builder() + .add_partition_field(2, "name_part", Transform::Identity) + .unwrap() + .build(); + let table = build_table_with_specs(vec![spec_v0, spec_v1]); + + let cols = identity_partition_col_names(&table); + // Neither column survives the intersection: `id` missing from v1, + // `name` missing from v0. + assert!(cols.is_empty(), "got {cols:?}"); + } + /// An empty table must produce a zero-partition scan so DataFusion never calls /// execute(0), which would otherwise return an out-of-bounds error. #[tokio::test] From aba45235b10d4115898123bb4ebd8bb8f84e4011 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Tue, 28 Apr 2026 16:40:25 +0200 Subject: [PATCH 10/16] Revert "feat(datafusion): allow Exact pushdown across spec evolution via per-column intersection" This reverts commit b2613e347ac3367f95220f7d55701d3e9db8b02f. (cherry picked from commit 826f054e368e6e6d7543c1f1f7cbc781f3bf6da1) --- .../datafusion/src/table/partitioned.rs | 202 +++--------------- 1 file changed, 29 insertions(+), 173 deletions(-) diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs index ba2ee2b615..34b8cf18f6 100644 --- a/crates/integrations/datafusion/src/table/partitioned.rs +++ b/crates/integrations/datafusion/src/table/partitioned.rs @@ -37,7 +37,7 @@ use datafusion::physical_plan::{ExecutionPlan, Partitioning}; use futures::TryStreamExt; use iceberg::arrow::schema_to_arrow_schema; use iceberg::scan::FileScanTask; -use iceberg::spec::{Literal, PartitionSpec, PrimitiveLiteral, Transform}; +use iceberg::spec::{Literal, PrimitiveLiteral, Transform}; use iceberg::table::Table; use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; @@ -62,21 +62,16 @@ pub struct IcebergPartitionedTableProvider { catalog: Arc, table_ident: TableIdent, schema: ArrowSchemaRef, - /// Source-column names that are identity-partitioned in *every* historical - /// partition spec of the table, captured at construction. Used by - /// `supports_filters_pushdown` to mark filters as `Exact` when they only - /// reference these columns: Iceberg evaluates partition predicates against - /// each manifest's own spec, so a column that is identity-partitioned in - /// every spec is fully prunable across the full table regardless of which - /// spec a given file was written under. + /// Source-column names that are identity-partitioned in the table's + /// default spec, captured at construction. Used by + /// `supports_filters_pushdown` to mark filters as `Exact` when they + /// only reference identity-partition columns. Empty when the table + /// has spec evolution (>1 historical specs) or no identity transforms, + /// which forces every filter to `Inexact`. /// - /// Columns that appear in some specs with non-identity transforms - /// (`bucket`, `truncate`, `year`/`month`/etc.), or that are missing from - /// any spec entirely, are dropped from the set — those files cannot be - /// pruned exactly, so DataFusion must keep its FilterExec. - /// - /// This is a snapshot: if the table's specs change between `try_new` and - /// a later scan, the cached set may be stale. The next `try_new` refreshes. + /// This is a snapshot: if the table's default spec changes between + /// `try_new` and a later scan, the cached set may be stale. Spec + /// evolution is rare in practice and the next `try_new` will refresh. identity_partition_cols: HashSet, } @@ -380,44 +375,28 @@ fn fallback_hash(task: &FileScanTask) -> u64 { hasher.finish() } -/// Intersection of identity-partitioned source-column names across every -/// historical partition spec. A column is included only if every spec -/// includes that column with `Transform::Identity`; any spec where the column -/// is absent or has a non-identity transform drops it from the result. -/// -/// Why intersection: Iceberg evaluates partition predicates against each -/// manifest's own spec. A file written under spec A can only be exactly -/// pruned by columns identity-partitioned in spec A. To guarantee Exact -/// pushdown for *every* file in the table, the column must be identity in -/// *every* spec. Otherwise some surviving files would still need DataFusion's -/// FilterExec to enforce the predicate. +/// Source-column names of every identity-transform field in the table's +/// default partition spec. Returns the empty set when the table has spec +/// evolution (>1 historical specs) — older files may carry partition tuples +/// whose identity status differs from the current spec, so the safe choice +/// is to refuse all `Exact` pushdowns until each task carries its own spec. fn identity_partition_col_names(table: &Table) -> HashSet { let metadata = table.metadata(); - let table_schema = metadata.current_schema(); - let identity_set = |spec: &PartitionSpec| -> HashSet { - spec.fields() - .iter() - .filter(|pf| pf.transform == Transform::Identity) - .filter_map(|pf| { - table_schema - .field_by_id(pf.source_id) - .map(|f| f.name.clone()) - }) - .collect() - }; - - let mut iter = metadata.partition_specs_iter(); - let Some(first) = iter.next() else { + if metadata.partition_specs_iter().len() > 1 { return HashSet::new(); - }; - let mut acc = identity_set(first); - for spec in iter { - if acc.is_empty() { - break; + } + let spec = metadata.default_partition_spec(); + let table_schema = metadata.current_schema(); + let mut names = HashSet::new(); + for pf in spec.fields() { + if pf.transform != Transform::Identity { + continue; + } + if let Some(field) = table_schema.field_by_id(pf.source_id) { + names.insert(field.name.clone()); } - acc = acc.intersection(&identity_set(spec)).cloned().collect(); } - acc + names } /// Returns `true` when every leaf of `expr` is a comparison or null check @@ -496,14 +475,11 @@ mod tests { use datafusion::logical_expr::{col, lit}; use datafusion::prelude::{SessionConfig, SessionContext}; - use iceberg::io::FileIO; use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; use iceberg::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, FormatVersion, NestedField, - PrimitiveType, Schema, SortOrder, TableMetadataBuilder, Transform, Type, - UnboundPartitionSpec, + DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, + Transform, Type, UnboundPartitionSpec, }; - use iceberg::table::Table; use iceberg::transaction::{ApplyTransactionAction, Transaction}; use iceberg::{Catalog, CatalogBuilder, NamespaceIdent, TableCreation, TableIdent}; use tempfile::TempDir; @@ -652,126 +628,6 @@ mod tests { SessionContext::new_with_config(SessionConfig::new().with_target_partitions(n)) } - /// Build a `Table` carrying `specs.len()` historical partition specs. The - /// first spec is the table's initial spec; each subsequent spec is added - /// via `into_builder().add_partition_spec(...)`. No catalog round-trip, - /// no real I/O — `FileIO::new_with_memory()` is sufficient because the - /// helper under test only reads metadata. - fn build_table_with_specs(specs: Vec) -> Table { - assert!(!specs.is_empty(), "need at least one spec"); - let schema = Schema::builder() - .with_schema_id(0) - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), - ]) - .build() - .unwrap(); - - let mut iter = specs.into_iter(); - let first = iter.next().unwrap(); - let mut metadata = TableMetadataBuilder::new( - schema, - first, - SortOrder::unsorted_order(), - "memory:///t".to_string(), - FormatVersion::V2, - HashMap::new(), - ) - .unwrap() - .build() - .unwrap() - .metadata; - - for spec in iter { - metadata = metadata - .into_builder(None) - .add_partition_spec(spec) - .unwrap() - .build() - .unwrap() - .metadata; - } - - Table::builder() - .file_io(FileIO::new_with_memory()) - .metadata(Arc::new(metadata)) - .identifier(TableIdent::new( - NamespaceIdent::new("ns".to_string()), - "t".to_string(), - )) - .build() - .unwrap() - } - - /// Multi-spec table where every historical spec keeps `id` as identity: - /// the column survives the intersection and remains Exact-pushdown safe. - /// `name` is never identity-partitioned, so it is excluded. - #[test] - fn test_identity_cols_preserved_across_compatible_specs() { - let spec_v0 = UnboundPartitionSpec::builder() - .add_partition_field(1, "id_part", Transform::Identity) - .unwrap() - .build(); - // Evolved spec: still identity on `id`, plus a non-identity transform - // on `name`. The latter must not pollute the result. - let spec_v1 = UnboundPartitionSpec::builder() - .add_partition_field(1, "id_part", Transform::Identity) - .unwrap() - .add_partition_field(2, "name_bucket", Transform::Bucket(8)) - .unwrap() - .build(); - let table = build_table_with_specs(vec![spec_v0, spec_v1]); - - let cols = identity_partition_col_names(&table); - assert_eq!(cols, HashSet::from(["id".to_string()])); - } - - /// Multi-spec table where the evolved spec replaces `identity(id)` with - /// `bucket(id)`. Files written under the evolved spec cannot be exactly - /// pruned on `id`, so `id` must be dropped from the Exact-safe set. - #[test] - fn test_identity_cols_dropped_when_transform_changes() { - let spec_v0 = UnboundPartitionSpec::builder() - .add_partition_field(1, "id_part", Transform::Identity) - .unwrap() - .build(); - let spec_v1 = UnboundPartitionSpec::builder() - .add_partition_field(1, "id_bucket", Transform::Bucket(8)) - .unwrap() - .build(); - let table = build_table_with_specs(vec![spec_v0, spec_v1]); - - let cols = identity_partition_col_names(&table); - assert!( - cols.is_empty(), - "expected empty set after non-identity replacement, got {cols:?}" - ); - } - - /// Multi-spec table where the second spec omits `id` from partitioning - /// entirely. Files under that spec carry no `id` partition tuple, so - /// pruning is a no-op for them — `id` must be dropped from the - /// Exact-safe set. - #[test] - fn test_identity_cols_dropped_when_column_missing_from_some_spec() { - let spec_v0 = UnboundPartitionSpec::builder() - .add_partition_field(1, "id_part", Transform::Identity) - .unwrap() - .build(); - // Evolved spec only partitions on `name`, omitting `id`. - let spec_v1 = UnboundPartitionSpec::builder() - .add_partition_field(2, "name_part", Transform::Identity) - .unwrap() - .build(); - let table = build_table_with_specs(vec![spec_v0, spec_v1]); - - let cols = identity_partition_col_names(&table); - // Neither column survives the intersection: `id` missing from v1, - // `name` missing from v0. - assert!(cols.is_empty(), "got {cols:?}"); - } - /// An empty table must produce a zero-partition scan so DataFusion never calls /// execute(0), which would otherwise return an out-of-bounds error. #[tokio::test] From 598c5de3ba93fe69f339f38c1a5687e852bc205b Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Tue, 28 Apr 2026 16:40:32 +0200 Subject: [PATCH 11/16] Revert "feat(datafusion): mark identity-partition filters as Exact pushdown" This reverts commit 6d0ed4c7c5ba4a0d53de6bca5c1321e744edc8fa. (cherry picked from commit 4381f004dc0d2aff5616e4cdd474f595d4ef1f8c) --- .../src/physical_plan/expr_to_predicate.rs | 2 +- .../datafusion/src/table/partitioned.rs | 249 +----------------- 2 files changed, 5 insertions(+), 246 deletions(-) diff --git a/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs b/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs index b426b1228f..17c9416d54 100644 --- a/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs +++ b/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs @@ -49,7 +49,7 @@ pub fn convert_filters_to_predicate(filters: &[Expr]) -> Option { .reduce(Predicate::and) } -pub(crate) fn convert_filter_to_predicate(expr: &Expr) -> Option { +fn convert_filter_to_predicate(expr: &Expr) -> Option { match to_iceberg_predicate(expr) { TransformedResult::Predicate(predicate) => Some(predicate), TransformedResult::Column(column) => { diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs index 34b8cf18f6..580cf68a47 100644 --- a/crates/integrations/datafusion/src/table/partitioned.rs +++ b/crates/integrations/datafusion/src/table/partitioned.rs @@ -16,7 +16,6 @@ // under the License. use std::any::Any; -use std::collections::HashSet; use std::sync::Arc; use async_trait::async_trait; @@ -29,7 +28,7 @@ use datafusion::catalog::Session; use datafusion::common::hash_utils::create_hashes; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::Result as DFResult; -use datafusion::logical_expr::{Expr, Operator, TableProviderFilterPushDown}; +use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_expr::expressions::Column; use datafusion::physical_plan::repartition::REPARTITION_RANDOM_STATE; @@ -42,9 +41,7 @@ use iceberg::table::Table; use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; use crate::error::to_datafusion_error; -use crate::physical_plan::expr_to_predicate::{ - convert_filter_to_predicate, convert_filters_to_predicate, -}; +use crate::physical_plan::expr_to_predicate::convert_filters_to_predicate; use crate::physical_plan::partitioned_scan::IcebergPartitionedScan; /// Catalog-backed table provider that scans each data file in a separate DataFusion partition. @@ -62,17 +59,6 @@ pub struct IcebergPartitionedTableProvider { catalog: Arc, table_ident: TableIdent, schema: ArrowSchemaRef, - /// Source-column names that are identity-partitioned in the table's - /// default spec, captured at construction. Used by - /// `supports_filters_pushdown` to mark filters as `Exact` when they - /// only reference identity-partition columns. Empty when the table - /// has spec evolution (>1 historical specs) or no identity transforms, - /// which forces every filter to `Inexact`. - /// - /// This is a snapshot: if the table's default spec changes between - /// `try_new` and a later scan, the cached set may be stale. Spec - /// evolution is rare in practice and the next `try_new` will refresh. - identity_partition_cols: HashSet, } impl IcebergPartitionedTableProvider { @@ -86,12 +72,10 @@ impl IcebergPartitionedTableProvider { // A second load_table is issued at scan time to guarantee the freshest snapshot. let table = catalog.load_table(&table_ident).await?; let schema = Arc::new(schema_to_arrow_schema(table.metadata().current_schema())?); - let identity_partition_cols = identity_partition_col_names(&table); Ok(Self { catalog, table_ident, schema, - identity_partition_cols, }) } } @@ -209,24 +193,7 @@ impl TableProvider for IcebergPartitionedTableProvider { &self, filters: &[&Expr], ) -> DFResult> { - Ok(filters - .iter() - .map(|f| { - // `Exact` is only safe when (1) the filter touches nothing but - // identity-partition columns and operators preserved by the - // identity transform, and (2) the iceberg conversion can - // actually represent the filter, so manifest pruning will - // remove every row that fails it. Either miss falls back to - // `Inexact` and DataFusion adds a FilterExec on top. - if convert_filter_to_predicate(f).is_some() - && is_exact_on_identity(f, &self.identity_partition_cols) - { - TableProviderFilterPushDown::Exact - } else { - TableProviderFilterPushDown::Inexact - } - }) - .collect()) + Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) } async fn insert_into( @@ -375,76 +342,6 @@ fn fallback_hash(task: &FileScanTask) -> u64 { hasher.finish() } -/// Source-column names of every identity-transform field in the table's -/// default partition spec. Returns the empty set when the table has spec -/// evolution (>1 historical specs) — older files may carry partition tuples -/// whose identity status differs from the current spec, so the safe choice -/// is to refuse all `Exact` pushdowns until each task carries its own spec. -fn identity_partition_col_names(table: &Table) -> HashSet { - let metadata = table.metadata(); - if metadata.partition_specs_iter().len() > 1 { - return HashSet::new(); - } - let spec = metadata.default_partition_spec(); - let table_schema = metadata.current_schema(); - let mut names = HashSet::new(); - for pf in spec.fields() { - if pf.transform != Transform::Identity { - continue; - } - if let Some(field) = table_schema.field_by_id(pf.source_id) { - names.insert(field.name.clone()); - } - } - names -} - -/// Returns `true` when every leaf of `expr` is a comparison or null check -/// against an identity-partition column. Such filters are fully resolvable -/// by manifest-level partition pruning, so DataFusion does not need to -/// re-apply them post-scan. -/// -/// Safe operators: `=`, `!=`, `<`, `<=`, `>`, `>=`, `IS NULL`, `IS NOT NULL`, -/// `IN (..)`, `NOT IN (..)`, plus `AND` / `OR` / `NOT` of any of those. Every -/// other shape returns `false` (caller falls back to `Inexact`). -fn is_exact_on_identity(expr: &Expr, cols: &HashSet) -> bool { - if cols.is_empty() { - return false; - } - match expr { - Expr::BinaryExpr(b) => match b.op { - Operator::And | Operator::Or => { - is_exact_on_identity(&b.left, cols) && is_exact_on_identity(&b.right, cols) - } - Operator::Eq - | Operator::NotEq - | Operator::Lt - | Operator::LtEq - | Operator::Gt - | Operator::GtEq => is_simple_compare_on_identity(&b.left, &b.right, cols), - _ => false, - }, - Expr::Not(inner) => is_exact_on_identity(inner, cols), - Expr::IsNull(inner) | Expr::IsNotNull(inner) => is_identity_col(inner, cols), - Expr::InList(l) => { - is_identity_col(&l.expr, cols) && l.list.iter().all(|e| matches!(e, Expr::Literal(..))) - } - _ => false, - } -} - -fn is_simple_compare_on_identity(l: &Expr, r: &Expr, cols: &HashSet) -> bool { - let l_col = is_identity_col(l, cols); - let r_col = is_identity_col(r, cols); - let l_lit = matches!(l, Expr::Literal(..)); - let r_lit = matches!(r, Expr::Literal(..)); - (l_col && r_lit) || (r_col && l_lit) -} - -fn is_identity_col(e: &Expr, cols: &HashSet) -> bool { - matches!(e, Expr::Column(c) if cols.contains(&c.name)) -} - /// Materialize a single-element Arrow array of `dt` holding the value of /// `lit`. The Arrow type must match what DataFusion will see for this column /// at scan time, otherwise `create_hashes` would dispatch on a different type @@ -473,12 +370,10 @@ mod tests { use std::collections::HashMap; use std::sync::Arc; - use datafusion::logical_expr::{col, lit}; use datafusion::prelude::{SessionConfig, SessionContext}; use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; use iceberg::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, - Transform, Type, UnboundPartitionSpec, + DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, Type, }; use iceberg::transaction::{ApplyTransactionAction, Transaction}; use iceberg::{Catalog, CatalogBuilder, NamespaceIdent, TableCreation, TableIdent}; @@ -486,58 +381,6 @@ mod tests { use super::*; - async fn make_catalog_and_partitioned_table( - partition_spec: Option, - ) -> (Arc, NamespaceIdent, String, TempDir) { - let temp_dir = TempDir::new().unwrap(); - let warehouse = temp_dir.path().to_str().unwrap().to_string(); - - let catalog = Arc::new( - MemoryCatalogBuilder::default() - .load( - "memory", - HashMap::from([(MEMORY_CATALOG_WAREHOUSE.to_string(), warehouse.clone())]), - ) - .await - .unwrap(), - ); - - let namespace = NamespaceIdent::new("ns".to_string()); - catalog - .create_namespace(&namespace, HashMap::new()) - .await - .unwrap(); - - let schema = Schema::builder() - .with_schema_id(0) - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), - ]) - .build() - .unwrap(); - - let creation = match partition_spec { - Some(spec) => TableCreation::builder() - .name("t".to_string()) - .location(format!("{warehouse}/t")) - .schema(schema) - .partition_spec(spec) - .properties(HashMap::new()) - .build(), - None => TableCreation::builder() - .name("t".to_string()) - .location(format!("{warehouse}/t")) - .schema(schema) - .properties(HashMap::new()) - .build(), - }; - - catalog.create_table(&namespace, creation).await.unwrap(); - - (catalog, namespace, "t".to_string(), temp_dir) - } - async fn make_catalog_and_table() -> (Arc, NamespaceIdent, String, TempDir) { let temp_dir = TempDir::new().unwrap(); let warehouse = temp_dir.path().to_str().unwrap().to_string(); @@ -703,90 +546,6 @@ mod tests { assert_eq!(scan.buckets().len(), 2); } - /// Filters that only touch identity-partition columns with literal RHS - /// can be marked `Exact` because Iceberg's manifest-level pruning already - /// removes every file whose partition value fails the predicate. - #[tokio::test] - async fn test_pushdown_exact_on_identity_column() { - let spec = UnboundPartitionSpec::builder() - .add_partition_field(1, "id_part", Transform::Identity) - .unwrap() - .build(); - let (catalog, namespace, table_name, _temp_dir) = - make_catalog_and_partitioned_table(Some(spec)).await; - let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) - .await - .unwrap(); - - let f_eq = col("id").eq(lit(5_i32)); - let f_neq = col("id").not_eq(lit(5_i32)); - let f_isnull = col("id").is_null(); - let f_and = col("id").eq(lit(5_i32)).and(col("id").lt(lit(10_i32))); - - let supports = provider - .supports_filters_pushdown(&[&f_eq, &f_neq, &f_isnull, &f_and]) - .unwrap(); - for (i, s) in supports.iter().enumerate() { - assert!( - matches!(s, TableProviderFilterPushDown::Exact), - "filter index {i} should be Exact, got {s:?}" - ); - } - } - - /// Filters touching non-partition columns or columns with non-identity - /// transforms must remain `Inexact`: the partition value is either - /// missing or lossy (bucket/truncate/etc.), so DataFusion still needs to - /// re-apply the filter against actual row values. - #[tokio::test] - async fn test_pushdown_inexact_on_non_identity_column() { - let spec = UnboundPartitionSpec::builder() - .add_partition_field(1, "id_part", Transform::Identity) - .unwrap() - .build(); - let (catalog, namespace, table_name, _temp_dir) = - make_catalog_and_partitioned_table(Some(spec)).await; - let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) - .await - .unwrap(); - - // `name` is not partitioned — manifest pruning cannot eliminate files - // by it, so the filter must re-execute post-scan. - let f_name = col("name").eq(lit("alice")); - // Mixed AND: even though `id` is identity-partitioned, the `name` arm - // is not exact, so the whole expression is Inexact. - let f_mixed = col("id").eq(lit(5_i32)).and(col("name").eq(lit("alice"))); - - let supports = provider - .supports_filters_pushdown(&[&f_name, &f_mixed]) - .unwrap(); - for (i, s) in supports.iter().enumerate() { - assert!( - matches!(s, TableProviderFilterPushDown::Inexact), - "filter index {i} should be Inexact, got {s:?}" - ); - } - } - - /// Unpartitioned tables must mark every filter `Inexact` regardless of - /// shape: there is no partition pruning that could make the scan - /// authoritative. - #[tokio::test] - async fn test_pushdown_unpartitioned_table_all_inexact() { - let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; - let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) - .await - .unwrap(); - - let f_id = col("id").eq(lit(5_i32)); - let f_name = col("name").eq(lit("alice")); - let supports = provider - .supports_filters_pushdown(&[&f_id, &f_name]) - .unwrap(); - assert!(matches!(supports[0], TableProviderFilterPushDown::Inexact)); - assert!(matches!(supports[1], TableProviderFilterPushDown::Inexact)); - } - /// target_partitions = 1 collapses every task into a single bucket, giving /// the same execution profile as `IcebergTableScan`. #[tokio::test] From 23f3d8f2bf6e44c0d4a0380d6274c90fb1fe00bb Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Tue, 28 Apr 2026 17:29:19 +0200 Subject: [PATCH 12/16] refactor(datafusion): merge IcebergPartitionedTableProvider into IcebergTableProvider IcebergPartitionedTableProvider and IcebergPartitionedScan were introduced to enable parallel file scanning by bucketing FileScanTasks across DataFusion partitions. However, maintaining two TableProvider implementations is redundant: the new provider is strictly more capable, and its degenerate case (target_partitions=1) reproduces the old single-partition behavior exactly. This commit folds the partitioned provider into IcebergTableProvider and the partitioned scan into IcebergTableScan, eliminating the parallel types. Changes: - IcebergTableProvider::scan() now eagerly calls plan_files() and distributes FileScanTasks into buckets using the same identity-hash strategy (REPARTITION_RANDOM_STATE + create_hashes) that was in IcebergPartitionedTableProvider, enabling Partitioning::Hash declarations that align with DataFusion's RepartitionExec. - IcebergTableScan gains a new_with_tasks() constructor that accepts pre-planned buckets and a caller-supplied Partitioning. execute(i) streams the tasks in buckets[i] via TableScan::to_arrow_with_tasks, rebuilding the TableScan per-partition to avoid serializing PlanContext Arc-shared caches across workers. - The original new() constructor and the to_arrow() lazy path are kept unchanged for IcebergStaticTableProvider, which does not pre-plan tasks. - Limit slicing (try_filter_map truncation) from the old IcebergTableScan is preserved in both execution paths. - Bucketing helpers (IdentityCol, compute_identity_cols, bucket_tasks, identity_hash, fallback_hash, literal_to_array, is_supported_dtype) are moved verbatim into a new private table/bucketing.rs module. - Unit tests from partitioned.rs are migrated to table/mod.rs and updated to use IcebergTableProvider and IcebergTableScan. - integration_datafusion_test.rs: fix test_provider_plan_stream_schema to call execute(0) instead of execute(1). The old call worked only because the previous IcebergTableScan silently ignored the partition index. (cherry picked from commit d2e5e0412c1e7e17f85f9fa549af1544d07eaae1) --- crates/integrations/datafusion/src/lib.rs | 2 - .../datafusion/src/physical_plan/mod.rs | 2 - .../src/physical_plan/partitioned_scan.rs | 253 -------- .../datafusion/src/physical_plan/scan.rs | 241 ++++++-- .../datafusion/src/table/bucketing.rs | 189 ++++++ .../integrations/datafusion/src/table/mod.rs | 301 ++++++++- .../datafusion/src/table/partitioned.rs | 571 ------------------ .../tests/integration_datafusion_test.rs | 2 +- 8 files changed, 676 insertions(+), 885 deletions(-) delete mode 100644 crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs create mode 100644 crates/integrations/datafusion/src/table/bucketing.rs delete mode 100644 crates/integrations/datafusion/src/table/partitioned.rs diff --git a/crates/integrations/datafusion/src/lib.rs b/crates/integrations/datafusion/src/lib.rs index 9a84832d88..4b0ea8606d 100644 --- a/crates/integrations/datafusion/src/lib.rs +++ b/crates/integrations/datafusion/src/lib.rs @@ -24,8 +24,6 @@ pub use error::*; pub mod physical_plan; mod schema; pub mod table; -pub use physical_plan::IcebergPartitionedScan; -pub use table::partitioned::IcebergPartitionedTableProvider; pub use table::table_provider_factory::IcebergTableProviderFactory; pub use table::*; diff --git a/crates/integrations/datafusion/src/physical_plan/mod.rs b/crates/integrations/datafusion/src/physical_plan/mod.rs index a257fe9e20..aeac30de32 100644 --- a/crates/integrations/datafusion/src/physical_plan/mod.rs +++ b/crates/integrations/datafusion/src/physical_plan/mod.rs @@ -18,7 +18,6 @@ pub(crate) mod commit; pub(crate) mod expr_to_predicate; pub(crate) mod metadata_scan; -pub(crate) mod partitioned_scan; pub(crate) mod project; pub(crate) mod repartition; pub(crate) mod scan; @@ -28,6 +27,5 @@ pub(crate) mod write; pub(crate) const DATA_FILES_COL_NAME: &str = "data_files"; pub use expr_to_predicate::convert_filters_to_predicate; -pub use partitioned_scan::IcebergPartitionedScan; pub use project::project_with_partition; pub use scan::IcebergTableScan; diff --git a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs b/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs deleted file mode 100644 index 159665dddd..0000000000 --- a/crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs +++ /dev/null @@ -1,253 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::any::Any; -use std::sync::Arc; - -use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; -use datafusion::error::{DataFusionError, Result as DFResult}; -use datafusion::execution::{SendableRecordBatchStream, TaskContext}; -use datafusion::physical_expr::EquivalenceProperties; -use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion::physical_plan::stream::RecordBatchStreamAdapter; -use datafusion::physical_plan::{DisplayAs, ExecutionPlan, Partitioning, PlanProperties}; -use datafusion::prelude::Expr; -use futures::TryStreamExt; -use iceberg::expr::Predicate; -use iceberg::scan::FileScanTask; -use iceberg::table::Table; - -use super::expr_to_predicate::convert_filters_to_predicate; -use super::scan::get_column_names; -use crate::to_datafusion_error; - -/// A DataFusion [`ExecutionPlan`] that reads a bucket of [`FileScanTask`]s per partition. -/// -/// Each DataFusion partition `i` streams every [`FileScanTask`] in `buckets[i]`, -/// concatenated into a single Arrow record-batch stream. The caller decides how -/// tasks are assigned to buckets and supplies the resulting [`Partitioning`] -/// (typically [`Partitioning::Hash`] when files are bucketed by identity-partition -/// values matching DataFusion's repartition hash, otherwise -/// [`Partitioning::UnknownPartitioning`]). -/// -/// Arrow reader configuration (row-group filtering, row selection, concurrency -/// limit, batch size) matches [`IcebergTableScan`][super::scan::IcebergTableScan]: -/// it is sourced from the underlying [`TableScan`][iceberg::scan::TableScan] -/// rebuilt in [`execute`](ExecutionPlan::execute) and applied via -/// [`TableScan::to_arrow_with_tasks`][iceberg::scan::TableScan::to_arrow_with_tasks]. -/// -/// Note: the `TableScan` is rebuilt on every `execute(partition)` call rather -/// than cached as an `Arc` on the struct. Caching would avoid -/// redundant schema resolution and predicate binding per partition, but -/// `TableScan` carries a `PlanContext` with `Arc`-shared evaluator caches -/// which is awkward to serialize if this plan ever needs to be shipped across -/// workers. The per-build cost is bounded (no I/O), so the rebuild is kept -/// for now; revisit once the cross-worker story is clearer. -#[derive(Debug)] -pub struct IcebergPartitionedScan { - /// A table in the catalog. - table: Table, - /// Snapshot of the table to scan. - snapshot_id: Option, - /// Stores certain, often expensive to compute, - /// plan properties used in query optimization. - plan_properties: Arc, - /// Projection column names, None means all columns. - projection: Option>, - /// Filters to apply to the table scan. - predicates: Option, - /// Pre-planned file scan tasks grouped by output DataFusion partition. - /// `buckets[i]` holds every task that `execute(i)` will read. - buckets: Vec>, -} - -impl IcebergPartitionedScan { - pub(crate) fn new( - table: Table, - snapshot_id: Option, - schema: ArrowSchemaRef, - projection: Option<&Vec>, - filters: &[Expr], - buckets: Vec>, - partitioning: Partitioning, - ) -> Self { - let output_schema = match projection { - None => schema.clone(), - Some(projection) => Arc::new(schema.project(projection).unwrap()), - }; - let plan_properties = Arc::new(PlanProperties::new( - EquivalenceProperties::new(output_schema), - partitioning, - EmissionType::Incremental, - Boundedness::Bounded, - )); - let projection = get_column_names(schema, projection); - let predicates = convert_filters_to_predicate(filters); - - Self { - table, - snapshot_id, - plan_properties, - projection, - predicates, - buckets, - } - } - - pub fn table(&self) -> &Table { - &self.table - } - - pub fn snapshot_id(&self) -> Option { - self.snapshot_id - } - - pub fn projection(&self) -> Option<&[String]> { - self.projection.as_deref() - } - - pub fn predicates(&self) -> Option<&Predicate> { - self.predicates.as_ref() - } - - pub fn buckets(&self) -> &[Vec] { - &self.buckets - } - - fn total_file_count(&self) -> usize { - self.buckets.iter().map(|b| b.len()).sum() - } -} - -impl ExecutionPlan for IcebergPartitionedScan { - fn name(&self) -> &str { - "IcebergPartitionedScan" - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn children(&self) -> Vec<&Arc> { - vec![] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> DFResult> { - if !children.is_empty() { - return Err(DataFusionError::Internal(format!( - "{} is a leaf node and expects no children, but {} were provided", - self.name(), - children.len() - ))); - } - Ok(self) - } - - fn properties(&self) -> &Arc { - &self.plan_properties - } - - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> DFResult { - let bucket = self.buckets.get(partition).cloned().ok_or_else(|| { - DataFusionError::Internal(format!( - "{}: partition index {partition} is out of bounds (total buckets: {})", - self.name(), - self.buckets.len() - )) - })?; - - let table = self.table.clone(); - let snapshot_id = self.snapshot_id; - let column_names = self.projection.clone(); - let predicates = self.predicates.clone(); - - let fut = async move { - // Rebuild a TableScan mirroring IcebergTableScan::get_batch_stream so we - // inherit the same defaults (row-group filtering, batch size, concurrency, ...). - let scan_builder = match snapshot_id { - Some(id) => table.scan().snapshot_id(id), - None => table.scan(), - }; - let mut scan_builder = match column_names { - Some(names) => scan_builder.select(names), - None => scan_builder.select_all(), - }; - if let Some(pred) = predicates { - scan_builder = scan_builder.with_filter(pred); - } - let table_scan = scan_builder.build().map_err(to_datafusion_error)?; - - let task_stream = Box::pin(futures::stream::iter( - bucket.into_iter().map(Ok::<_, iceberg::Error>), - )); - let record_batch_stream = table_scan - .to_arrow_with_tasks(task_stream) - .map_err(to_datafusion_error)? - .map_err(to_datafusion_error); - Ok::<_, DataFusionError>(record_batch_stream) - }; - - let stream = futures::stream::once(fut).try_flatten(); - - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - stream, - ))) - } -} - -impl DisplayAs for IcebergPartitionedScan { - fn fmt_as( - &self, - _t: datafusion::physical_plan::DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - let projection = self - .projection - .clone() - .map_or(String::new(), |v| v.join(",")); - let predicate = self - .predicates - .clone() - .map_or(String::new(), |p| format!("{p}")); - let file_count = self.total_file_count(); - let bucket_count = self.buckets.len(); - write!( - f, - "{} projection:[{projection}] predicate:[{predicate}] \ - buckets:[{bucket_count}] file_count:[{file_count}]", - self.name() - )?; - if file_count <= 5 { - let files = self - .buckets - .iter() - .flat_map(|b| b.iter().map(|t| t.data_file_path())) - .collect::>() - .join(", "); - write!(f, " files:[{files}]")?; - } - Ok(()) - } -} diff --git a/crates/integrations/datafusion/src/physical_plan/scan.rs b/crates/integrations/datafusion/src/physical_plan/scan.rs index c5a892abeb..49ff61e999 100644 --- a/crates/integrations/datafusion/src/physical_plan/scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/scan.rs @@ -18,11 +18,10 @@ use std::any::Any; use std::pin::Pin; use std::sync::Arc; -use std::vec; use datafusion::arrow::array::RecordBatch; use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; -use datafusion::error::Result as DFResult; +use datafusion::error::{DataFusionError, Result as DFResult}; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; @@ -31,6 +30,7 @@ use datafusion::physical_plan::{DisplayAs, ExecutionPlan, Partitioning, PlanProp use datafusion::prelude::Expr; use futures::{Stream, TryStreamExt}; use iceberg::expr::Predicate; +use iceberg::scan::FileScanTask; use iceberg::table::Table; use super::expr_to_predicate::convert_filters_to_predicate; @@ -38,6 +38,26 @@ use crate::to_datafusion_error; /// Manages the scanning process of an Iceberg [`Table`], encapsulating the /// necessary details and computed properties required for execution planning. +/// +/// When constructed with pre-planned [`FileScanTask`] buckets via +/// [`IcebergTableScan::new_with_tasks`], each DataFusion partition `i` streams +/// every task in `buckets[i]` using +/// [`TableScan::to_arrow_with_tasks`][iceberg::scan::TableScan::to_arrow_with_tasks]. +/// +/// When constructed via [`IcebergTableScan::new`] (no pre-planned tasks), the +/// full table is scanned lazily in a single partition using +/// [`TableScan::to_arrow`][iceberg::scan::TableScan::to_arrow]. This mode is +/// used by [`IcebergStaticTableProvider`][crate::table::IcebergStaticTableProvider]. +/// +/// In both modes the optional `limit` field truncates the output stream to at +/// most that many rows. +/// +/// Note: when using pre-planned tasks, the `TableScan` is rebuilt on every +/// `execute(partition)` call rather than cached. `TableScan` carries a +/// `PlanContext` with `Arc`-shared evaluator caches which is awkward to +/// serialize if this plan ever needs to be shipped across workers. The +/// per-build cost is bounded (no I/O), so the rebuild is kept for now; +/// revisit once the cross-worker story is clearer. #[derive(Debug)] pub struct IcebergTableScan { /// A table in the catalog. @@ -47,16 +67,25 @@ pub struct IcebergTableScan { /// Stores certain, often expensive to compute, /// plan properties used in query optimization. plan_properties: Arc, - /// Projection column names, None means all columns + /// Projection column names, None means all columns. projection: Option>, - /// Filters to apply to the table scan + /// Filters to apply to the table scan. predicates: Option, - /// Optional limit on the number of rows to return + /// Pre-planned file scan tasks grouped by output DataFusion partition. + /// `None` in lazy mode (single-partition scan via `to_arrow()`). + /// `Some(buckets)` in eager mode: `buckets[i]` holds every task that + /// `execute(i)` will read. + buckets: Option>>, + /// Optional limit on the number of rows to return. limit: Option, } impl IcebergTableScan { - /// Creates a new [`IcebergTableScan`] object. + /// Creates a lazy single-partition scan. + /// + /// All file tasks are discovered and read inside `execute(0)` via + /// [`TableScan::to_arrow`][iceberg::scan::TableScan::to_arrow]. + /// Used by [`IcebergStaticTableProvider`][crate::table::IcebergStaticTableProvider]. pub(crate) fn new( table: Table, snapshot_id: Option, @@ -69,8 +98,13 @@ impl IcebergTableScan { None => schema.clone(), Some(projection) => Arc::new(schema.project(projection).unwrap()), }; - let plan_properties = Self::compute_properties(output_schema.clone()); - let projection = get_column_names(schema.clone(), projection); + let plan_properties = Arc::new(PlanProperties::new( + EquivalenceProperties::new(output_schema), + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Bounded, + )); + let projection = get_column_names(schema, projection); let predicates = convert_filters_to_predicate(filters); Self { @@ -79,6 +113,49 @@ impl IcebergTableScan { plan_properties, projection, predicates, + buckets: None, + limit, + } + } + + /// Creates an eager multi-partition scan from pre-planned file task buckets. + /// + /// Each DataFusion partition `i` streams the tasks in `buckets[i]` via + /// [`TableScan::to_arrow_with_tasks`][iceberg::scan::TableScan::to_arrow_with_tasks]. + /// The `partitioning` argument is used directly for [`PlanProperties`], so the + /// caller is responsible for ensuring it matches the bucketing strategy. + /// Used by [`IcebergTableProvider`][crate::table::IcebergTableProvider]. + #[allow(clippy::too_many_arguments)] + pub(crate) fn new_with_tasks( + table: Table, + snapshot_id: Option, + schema: ArrowSchemaRef, + projection: Option<&Vec>, + filters: &[Expr], + limit: Option, + buckets: Vec>, + partitioning: Partitioning, + ) -> Self { + let output_schema = match projection { + None => schema.clone(), + Some(projection) => Arc::new(schema.project(projection).unwrap()), + }; + let plan_properties = Arc::new(PlanProperties::new( + EquivalenceProperties::new(output_schema), + partitioning, + EmissionType::Incremental, + Boundedness::Bounded, + )); + let projection = get_column_names(schema, projection); + let predicates = convert_filters_to_predicate(filters); + + Self { + table, + snapshot_id, + plan_properties, + projection, + predicates, + buckets: Some(buckets), limit, } } @@ -99,21 +176,17 @@ impl IcebergTableScan { self.predicates.as_ref() } + /// Returns the pre-planned file task buckets, or an empty slice in lazy mode. + pub fn buckets(&self) -> &[Vec] { + self.buckets.as_deref().unwrap_or(&[]) + } + pub fn limit(&self) -> Option { self.limit } - /// Computes [`PlanProperties`] used in query optimization. - fn compute_properties(schema: ArrowSchemaRef) -> Arc { - // TODO: - // This is more or less a placeholder, to be replaced - // once we support output-partitioning - Arc::new(PlanProperties::new( - EquivalenceProperties::new(schema), - Partitioning::UnknownPartitioning(1), - EmissionType::Incremental, - Boundedness::Bounded, - )) + fn total_file_count(&self) -> usize { + self.buckets().iter().map(|b| b.len()).sum() } } @@ -132,8 +205,15 @@ impl ExecutionPlan for IcebergTableScan { fn with_new_children( self: Arc, - _children: Vec>, + children: Vec>, ) -> DFResult> { + if !children.is_empty() { + return Err(DataFusionError::Internal(format!( + "{} is a leaf node and expects no children, but {} were provided", + self.name(), + children.len() + ))); + } Ok(self) } @@ -143,20 +223,66 @@ impl ExecutionPlan for IcebergTableScan { fn execute( &self, - _partition: usize, + partition: usize, _context: Arc, ) -> DFResult { - let fut = get_batch_stream( - self.table.clone(), - self.snapshot_id, - self.projection.clone(), - self.predicates.clone(), - ); - let stream = futures::stream::once(fut).try_flatten(); - - // Apply limit if specified + let table = self.table.clone(); + let snapshot_id = self.snapshot_id; + let column_names = self.projection.clone(); + let predicates = self.predicates.clone(); + let limit = self.limit; + + let stream = match &self.buckets { + Some(buckets) => { + // Eager mode: stream the pre-planned bucket for this partition. + let bucket = buckets.get(partition).cloned().ok_or_else(|| { + DataFusionError::Internal(format!( + "{}: partition index {partition} is out of bounds (total buckets: {})", + self.name(), + buckets.len() + )) + })?; + + let fut = async move { + // Rebuild a TableScan so we inherit the same defaults + // (row-group filtering, batch size, concurrency, ...). + let scan_builder = match snapshot_id { + Some(id) => table.scan().snapshot_id(id), + None => table.scan(), + }; + let mut scan_builder = match column_names { + Some(names) => scan_builder.select(names), + None => scan_builder.select_all(), + }; + if let Some(pred) = predicates { + scan_builder = scan_builder.with_filter(pred); + } + let table_scan = scan_builder.build().map_err(to_datafusion_error)?; + + let task_stream = Box::pin(futures::stream::iter( + bucket.into_iter().map(Ok::<_, iceberg::Error>), + )); + let record_batch_stream = table_scan + .to_arrow_with_tasks(task_stream) + .map_err(to_datafusion_error)? + .map_err(to_datafusion_error); + Ok::<_, DataFusionError>(record_batch_stream) + }; + + let s = futures::stream::once(fut).try_flatten(); + Box::pin(s) as Pin> + Send>> + } + None => { + // Lazy mode: discover and read all tasks inside execute(). + let fut = get_batch_stream(table, snapshot_id, column_names, predicates); + let s = futures::stream::once(fut).try_flatten(); + Box::pin(s) + } + }; + + // Apply limit if specified. let limited_stream: Pin> + Send>> = - if let Some(limit) = self.limit { + if let Some(limit) = limit { let mut remaining = limit; Box::pin(stream.try_filter_map(move |batch| { futures::future::ready(if remaining == 0 { @@ -171,7 +297,7 @@ impl ExecutionPlan for IcebergTableScan { }) })) } else { - Box::pin(stream) + stream }; Ok(Box::pin(RecordBatchStreamAdapter::new( @@ -187,16 +313,40 @@ impl DisplayAs for IcebergTableScan { _t: datafusion::physical_plan::DisplayFormatType, f: &mut std::fmt::Formatter, ) -> std::fmt::Result { - write!( - f, - "IcebergTableScan projection:[{}] predicate:[{}]", - self.projection - .clone() - .map_or(String::new(), |v| v.join(",")), - self.predicates - .clone() - .map_or(String::from(""), |p| format!("{p}")) - )?; + let projection = self + .projection + .clone() + .map_or(String::new(), |v| v.join(",")); + let predicate = self + .predicates + .clone() + .map_or(String::new(), |p| format!("{p}")); + + match &self.buckets { + Some(buckets) => { + let file_count = self.total_file_count(); + let bucket_count = buckets.len(); + write!( + f, + "{} projection:[{projection}] predicate:[{predicate}] \ + buckets:[{bucket_count}] file_count:[{file_count}]", + self.name() + )?; + if file_count <= 5 { + let files = buckets + .iter() + .flat_map(|b| b.iter().map(|t| t.data_file_path())) + .collect::>() + .join(", "); + write!(f, " files:[{files}]")?; + } + } + None => write!( + f, + "{} projection:[{projection}] predicate:[{predicate}]", + self.name() + )?, + } if let Some(limit) = self.limit { write!(f, " limit:[{limit}]")?; } @@ -204,11 +354,8 @@ impl DisplayAs for IcebergTableScan { } } -/// Asynchronously retrieves a stream of [`RecordBatch`] instances -/// from a given table. -/// -/// This function initializes a [`TableScan`], builds it, -/// and then converts it into a stream of Arrow [`RecordBatch`]es. +/// Asynchronously retrieves a stream of [`RecordBatch`] instances from a +/// given table. Used in lazy (single-partition) scan mode. async fn get_batch_stream( table: Table, snapshot_id: Option, diff --git a/crates/integrations/datafusion/src/table/bucketing.rs b/crates/integrations/datafusion/src/table/bucketing.rs new file mode 100644 index 0000000000..4c58730ce5 --- /dev/null +++ b/crates/integrations/datafusion/src/table/bucketing.rs @@ -0,0 +1,189 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use datafusion::arrow::array::{ + ArrayRef, BooleanArray, Date32Array, Float32Array, Float64Array, Int32Array, Int64Array, + StringArray, +}; +use datafusion::arrow::datatypes::{DataType, Schema as ArrowSchema}; +use datafusion::common::hash_utils::create_hashes; +use datafusion::physical_plan::repartition::REPARTITION_RANDOM_STATE; +use iceberg::scan::FileScanTask; +use iceberg::spec::{Literal, PrimitiveLiteral, Transform}; +use iceberg::table::Table; + +/// Identity-partitioned column that is also present in the output projection +/// and whose Arrow type can be reconstructed from a `Literal` for hashing. +pub(super) struct IdentityCol { + pub(super) name: String, + /// Position of this column in the *output* schema (after projection). + pub(super) output_idx: usize, + /// Position of this column inside the partition spec's `fields()` slice, + /// matching the slot order of `FileScanTask::partition`. + pub(super) spec_field_idx: usize, + pub(super) output_dtype: DataType, +} + +/// Inspect the table's default partition spec and return the list of identity +/// columns that can support a [`Partitioning::Hash`] declaration. Returns +/// `None` if any condition is violated: +/// - the source column for an identity field is not in the output projection +/// - the source column's Arrow type is not currently supported by +/// [`literal_to_array`] +/// - the table has spec evolution (>1 historical specs), since older files +/// may carry a partition tuple that does not align with the default spec +/// +/// Returning `None` forces the scan to declare `UnknownPartitioning` even if +/// bucketing succeeds. +pub(super) fn compute_identity_cols( + table: &Table, + output_schema: &ArrowSchema, +) -> Option> { + let metadata = table.metadata(); + if metadata.partition_specs_iter().len() > 1 { + return None; + } + let spec = metadata.default_partition_spec(); + let table_schema = metadata.current_schema(); + + let mut cols = Vec::new(); + for (spec_field_idx, pf) in spec.fields().iter().enumerate() { + if pf.transform != Transform::Identity { + continue; + } + let source_field = table_schema.field_by_id(pf.source_id)?; + let output_idx = output_schema.index_of(source_field.name.as_str()).ok()?; + let output_dtype = output_schema.field(output_idx).data_type().clone(); + if !is_supported_dtype(&output_dtype) { + return None; + } + cols.push(IdentityCol { + name: source_field.name.clone(), + output_idx, + spec_field_idx, + output_dtype, + }); + } + Some(cols) +} + +fn is_supported_dtype(dt: &DataType) -> bool { + matches!( + dt, + DataType::Boolean + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Utf8 + | DataType::Date32 + ) +} + +/// Distribute `tasks` across `n_partitions` buckets. When `identity_cols` +/// describes a non-empty, hashable identity key, each task is hashed on +/// that key using DataFusion's repartition hash so the resulting partitioning +/// matches what `RepartitionExec` would produce on the same data. Tasks +/// missing partition data fall back to hashing `data_file_path`, which still +/// distributes evenly but breaks the `Hash` contract — the second tuple +/// element flags whether every task supplied a full identity key. +pub(super) fn bucket_tasks( + tasks: Vec, + n_partitions: usize, + identity_cols: Option<&[IdentityCol]>, +) -> (Vec>, bool) { + if n_partitions == 0 { + return (Vec::new(), tasks.is_empty()); + } + let mut buckets: Vec> = (0..n_partitions).map(|_| Vec::new()).collect(); + let mut all_full_key = true; + let cols = identity_cols.unwrap_or(&[]); + + for task in tasks { + let bucket_idx = match identity_hash(&task, cols) { + Some(h) => (h % n_partitions as u64) as usize, + None => { + all_full_key = false; + fallback_hash(&task) as usize % n_partitions + } + }; + buckets[bucket_idx].push(task); + } + (buckets, all_full_key) +} + +/// Hash the identity-partition values of `task` using +/// [`REPARTITION_RANDOM_STATE`] so the bucket assignment matches DataFusion's +/// hash-repartition convention. Returns `None` if the task lacks partition +/// data or any required slot is null/unsupported. +fn identity_hash(task: &FileScanTask, cols: &[IdentityCol]) -> Option { + if cols.is_empty() { + return None; + } + let partition = task.partition.as_ref()?; + let mut arrays: Vec = Vec::with_capacity(cols.len()); + for col in cols { + let lit = partition.fields().get(col.spec_field_idx)?.as_ref()?; + arrays.push(literal_to_array(lit, &col.output_dtype)?); + } + let mut hashes = vec![0u64; 1]; + create_hashes( + &arrays, + REPARTITION_RANDOM_STATE.random_state(), + &mut hashes, + ) + .ok()?; + Some(hashes[0]) +} + +/// Deterministic per-file fallback used when `identity_hash` cannot produce a +/// bucket. The hash function does not need to match DataFusion's because any +/// task taking this path causes the scan to drop to `UnknownPartitioning`. +fn fallback_hash(task: &FileScanTask) -> u64 { + use std::collections::hash_map::DefaultHasher; + use std::hash::{Hash, Hasher}; + let mut hasher = DefaultHasher::new(); + task.data_file_path.hash(&mut hasher); + hasher.finish() +} + +/// Materialize a single-element Arrow array of `dt` holding the value of +/// `lit`. The Arrow type must match what DataFusion will see for this column +/// at scan time, otherwise `create_hashes` would dispatch on a different type +/// and produce a hash that disagrees with DataFusion's row-wise hashing. +fn literal_to_array(lit: &Literal, dt: &DataType) -> Option { + let prim = match lit { + Literal::Primitive(p) => p, + _ => return None, + }; + Some(match (prim, dt) { + (PrimitiveLiteral::Boolean(v), DataType::Boolean) => Arc::new(BooleanArray::from(vec![*v])), + (PrimitiveLiteral::Int(v), DataType::Int32) => Arc::new(Int32Array::from(vec![*v])), + (PrimitiveLiteral::Int(v), DataType::Date32) => Arc::new(Date32Array::from(vec![*v])), + (PrimitiveLiteral::Long(v), DataType::Int64) => Arc::new(Int64Array::from(vec![*v])), + (PrimitiveLiteral::Float(v), DataType::Float32) => Arc::new(Float32Array::from(vec![v.0])), + (PrimitiveLiteral::Double(v), DataType::Float64) => { + Arc::new(Float64Array::from(vec![v.0])) + } + (PrimitiveLiteral::String(v), DataType::Utf8) => { + Arc::new(StringArray::from(vec![v.as_str()])) + } + _ => return None, + }) +} diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index e2c9ca6efa..df9631cb3c 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -21,16 +21,14 @@ //! //! - [`IcebergTableProvider`]: Catalog-backed provider with automatic metadata refresh. //! Use for write operations and when you need to see the latest table state. +//! On each scan, file tasks are eagerly planned and distributed across DataFusion +//! partitions for parallel execution. //! //! - [`IcebergStaticTableProvider`]: Static provider for read-only access to a specific //! table snapshot. Use for consistent analytical queries or time-travel scenarios. -//! -//! - [`IcebergPartitionedTableProvider`]: Catalog-backed provider that assigns one -//! DataFusion partition per data file, enabling parallel file-level scanning. -//! Read-only; use [`IcebergTableProvider`] for write operations. +mod bucketing; pub mod metadata_table; -pub mod partitioned; pub mod table_provider_factory; use std::any::Any; @@ -45,10 +43,15 @@ use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::Result as DFResult; use datafusion::logical_expr::dml::InsertOp; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_expr::expressions::Column; use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_plan::Partitioning; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use futures::TryStreamExt; use iceberg::arrow::schema_to_arrow_schema; use iceberg::inspect::MetadataTableType; +use iceberg::scan::FileScanTask; use iceberg::spec::TableProperties; use iceberg::table::Table; use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; @@ -56,6 +59,7 @@ use metadata_table::IcebergMetadataTableProvider; use crate::error::to_datafusion_error; use crate::physical_plan::commit::IcebergCommitExec; +use crate::physical_plan::expr_to_predicate::convert_filters_to_predicate; use crate::physical_plan::project::project_with_partition; use crate::physical_plan::repartition::repartition; use crate::physical_plan::scan::IcebergTableScan; @@ -129,26 +133,93 @@ impl TableProvider for IcebergTableProvider { async fn scan( &self, - _state: &dyn Session, + state: &dyn Session, projection: Option<&Vec>, filters: &[Expr], limit: Option, ) -> DFResult> { - // Load fresh table metadata from catalog + // Second load: fetch the latest snapshot so scans always reflect current table state. let table = self .catalog .load_table(&self.table_ident) .await .map_err(to_datafusion_error)?; - // Create scan with fresh metadata (always use current snapshot) - Ok(Arc::new(IcebergTableScan::new( + // Build a TableScan mirroring the inputs we'll hand to IcebergTableScan, + // so plan_files() uses the same projection/filters the scan will replay in execute(). + let col_names = projection.map(|indices| { + indices + .iter() + .map(|&i| self.schema.field(i).name().clone()) + .collect::>() + }); + + let predicate = convert_filters_to_predicate(filters); + + let mut builder = table.scan(); + builder = match col_names { + Some(names) => builder.select(names), + None => builder.select_all(), + }; + if let Some(pred) = predicate { + builder = builder.with_filter(pred); + } + + let tasks: Vec = builder + .build() + .map_err(to_datafusion_error)? + .plan_files() + .await + .map_err(to_datafusion_error)? + .try_collect::>() + .await + .map_err(to_datafusion_error)?; + + // Output schema after projection: column indices in `Hash` exprs and any + // Arrow array we hash must reference this schema, not the full table schema. + let output_schema = match projection { + None => self.schema.clone(), + Some(p) => Arc::new(self.schema.project(p).map_err(|e| { + to_datafusion_error(Error::new(ErrorKind::DataInvalid, e.to_string())) + })?), + }; + + let target_partitions = state.config().target_partitions(); + // Always produce at least 1 partition so that DataFusion can schedule + // the plan normally and callers can safely call execute(0). An empty + // bucket simply yields an empty record-batch stream. + let n_partitions = target_partitions.min(tasks.len()).max(1); + + // identity_cols is Some(non-empty) iff every condition for declaring + // Partitioning::Hash is met: the table's default spec has identity-transform + // fields, every such source column is present in the output projection, and + // every column type is supported by literal_to_array. Any miss collapses to + // None, which forces UnknownPartitioning regardless of bucketing strategy. + let identity_cols = bucketing::compute_identity_cols(&table, &output_schema); + + let (buckets, all_had_full_key) = + bucketing::bucket_tasks(tasks, n_partitions, identity_cols.as_deref()); + + let partitioning = match identity_cols { + Some(cols) if !cols.is_empty() && all_had_full_key && n_partitions > 0 => { + let exprs: Vec> = cols + .iter() + .map(|c| Arc::new(Column::new(&c.name, c.output_idx)) as Arc) + .collect(); + Partitioning::Hash(exprs, n_partitions) + } + _ => Partitioning::UnknownPartitioning(n_partitions), + }; + + Ok(Arc::new(IcebergTableScan::new_with_tasks( table, None, // Always use current snapshot for catalog-backed provider self.schema.clone(), projection, filters, limit, + buckets, + partitioning, ))) } @@ -870,4 +941,216 @@ mod tests { "Limit should be None when not specified" ); } + + // ── IcebergTableProvider bucketed scan tests ───────────────────────────── + // (Originally from table/partitioned.rs; updated to use IcebergTableProvider + // and IcebergTableScan after the IcebergPartitionedTableProvider merge.) + + async fn make_catalog_and_table_for_bucketing( + ) -> (Arc, NamespaceIdent, String, tempfile::TempDir) { + use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; + use iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; + use iceberg::{CatalogBuilder, TableCreation}; + + let temp_dir = tempfile::TempDir::new().unwrap(); + let warehouse = temp_dir.path().to_str().unwrap().to_string(); + + let catalog = Arc::new( + MemoryCatalogBuilder::default() + .load( + "memory", + std::collections::HashMap::from([( + MEMORY_CATALOG_WAREHOUSE.to_string(), + warehouse.clone(), + )]), + ) + .await + .unwrap(), + ); + + let namespace = NamespaceIdent::new("ns".to_string()); + catalog + .create_namespace(&namespace, std::collections::HashMap::new()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + catalog + .create_table( + &namespace, + TableCreation::builder() + .name("t".to_string()) + .location(format!("{warehouse}/t")) + .schema(schema) + .properties(std::collections::HashMap::new()) + .build(), + ) + .await + .unwrap(); + + (catalog, namespace, "t".to_string(), temp_dir) + } + + /// Registers `n` synthetic data files in the table metadata via the iceberg + /// transaction API. No actual parquet files are written, only the metadata + /// entries that `plan_files()` reads are created. + async fn append_fake_data_files( + catalog: &Arc, + namespace: &NamespaceIdent, + table_name: &str, + n: usize, + ) { + use iceberg::spec::{DataContentType, DataFileBuilder, DataFileFormat}; + use iceberg::transaction::{ApplyTransactionAction, Transaction}; + + let table = catalog + .load_table(&TableIdent::new(namespace.clone(), table_name.to_string())) + .await + .unwrap(); + + let data_files = (0..n) + .map(|i| { + DataFileBuilder::default() + .content(DataContentType::Data) + .file_path(format!( + "{}/data/fake_{i}.parquet", + table.metadata().location() + )) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(128) + .record_count(1) + .partition_spec_id(table.metadata().default_partition_spec_id()) + .build() + .unwrap() + }) + .collect::>(); + + let tx = Transaction::new(&table); + let action = tx.fast_append().add_data_files(data_files); + action + .apply(tx) + .unwrap() + .commit(catalog.as_ref()) + .await + .unwrap(); + } + + fn ctx_with_target_partitions(n: usize) -> SessionContext { + use datafusion::prelude::SessionConfig; + SessionContext::new_with_config(SessionConfig::new().with_target_partitions(n)) + } + + /// An empty table must produce a single empty-bucket scan so that DataFusion + /// can schedule the plan normally. execute(0) on an empty bucket simply + /// returns an empty record-batch stream. + #[tokio::test] + async fn test_empty_table_single_empty_bucket() { + let (catalog, namespace, table_name, _temp_dir) = + make_catalog_and_table_for_bucketing().await; + // no files appended + let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&ctx_with_target_partitions(8).state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(scan.buckets().len(), 1); + assert_eq!(scan.buckets()[0].len(), 0); + assert_eq!(scan.properties().partitioning.partition_count(), 1); + } + + /// When the table has no identity-partition columns, every task takes the + /// fallback (file_path) bucket path, so the declaration must drop to + /// `UnknownPartitioning`. The bucket count should still equal + /// min(target_partitions, num_files). + #[tokio::test] + async fn test_unpartitioned_falls_back_to_unknown() { + use datafusion::physical_plan::Partitioning; + + let (catalog, namespace, table_name, _temp_dir) = + make_catalog_and_table_for_bucketing().await; + append_fake_data_files(&catalog, &namespace, &table_name, 5).await; + + let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&ctx_with_target_partitions(3).state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + let total_files: usize = scan.buckets().iter().map(|b| b.len()).sum(); + assert_eq!(total_files, 5); + assert_eq!(scan.buckets().len(), 3); + assert!(matches!( + scan.properties().partitioning, + Partitioning::UnknownPartitioning(3) + )); + } + + /// Bucket count must be capped at the number of files: spinning up more + /// DataFusion partitions than there are tasks would just leave empty + /// streams, wasting scheduler slots. + #[tokio::test] + async fn test_bucket_count_capped_at_file_count() { + let (catalog, namespace, table_name, _temp_dir) = + make_catalog_and_table_for_bucketing().await; + append_fake_data_files(&catalog, &namespace, &table_name, 2).await; + + let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&ctx_with_target_partitions(16).state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(scan.buckets().len(), 2); + } + + /// target_partitions = 1 collapses every task into a single bucket, giving + /// the same execution profile as a single-partition scan. + #[tokio::test] + async fn test_single_target_partition_single_bucket() { + let (catalog, namespace, table_name, _temp_dir) = + make_catalog_and_table_for_bucketing().await; + append_fake_data_files(&catalog, &namespace, &table_name, 4).await; + + let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&ctx_with_target_partitions(1).state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(scan.buckets().len(), 1); + assert_eq!(scan.buckets()[0].len(), 4); + } } diff --git a/crates/integrations/datafusion/src/table/partitioned.rs b/crates/integrations/datafusion/src/table/partitioned.rs deleted file mode 100644 index 580cf68a47..0000000000 --- a/crates/integrations/datafusion/src/table/partitioned.rs +++ /dev/null @@ -1,571 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::any::Any; -use std::sync::Arc; - -use async_trait::async_trait; -use datafusion::arrow::array::{ - ArrayRef, BooleanArray, Date32Array, Float32Array, Float64Array, Int32Array, Int64Array, - StringArray, -}; -use datafusion::arrow::datatypes::{DataType, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; -use datafusion::catalog::Session; -use datafusion::common::hash_utils::create_hashes; -use datafusion::datasource::{TableProvider, TableType}; -use datafusion::error::Result as DFResult; -use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; -use datafusion::physical_expr::PhysicalExpr; -use datafusion::physical_expr::expressions::Column; -use datafusion::physical_plan::repartition::REPARTITION_RANDOM_STATE; -use datafusion::physical_plan::{ExecutionPlan, Partitioning}; -use futures::TryStreamExt; -use iceberg::arrow::schema_to_arrow_schema; -use iceberg::scan::FileScanTask; -use iceberg::spec::{Literal, PrimitiveLiteral, Transform}; -use iceberg::table::Table; -use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent}; - -use crate::error::to_datafusion_error; -use crate::physical_plan::expr_to_predicate::convert_filters_to_predicate; -use crate::physical_plan::partitioned_scan::IcebergPartitionedScan; - -/// Catalog-backed table provider that scans each data file in a separate DataFusion partition. -/// -/// This provider reloads table metadata from the catalog on every [`scan`][Self::scan] call -/// to guarantee freshness, then issues one DataFusion partition per data file so that -/// DataFusion's scheduler can execute file reads in parallel. -/// -/// Write operations are not supported. Use [`IcebergTableProvider`] for write access. -/// -/// For consistent read-only access to a fixed snapshot without per-scan catalog overhead, -/// use [`IcebergStaticTableProvider`] instead. -#[derive(Debug, Clone)] -pub struct IcebergPartitionedTableProvider { - catalog: Arc, - table_ident: TableIdent, - schema: ArrowSchemaRef, -} - -impl IcebergPartitionedTableProvider { - pub async fn try_new( - catalog: Arc, - namespace: NamespaceIdent, - name: impl Into, - ) -> Result { - let table_ident = TableIdent::new(namespace, name.into()); - // First load: used only to snapshot the Arrow schema for DataFusion planning. - // A second load_table is issued at scan time to guarantee the freshest snapshot. - let table = catalog.load_table(&table_ident).await?; - let schema = Arc::new(schema_to_arrow_schema(table.metadata().current_schema())?); - Ok(Self { - catalog, - table_ident, - schema, - }) - } -} - -#[async_trait] -impl TableProvider for IcebergPartitionedTableProvider { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> ArrowSchemaRef { - self.schema.clone() - } - - fn table_type(&self) -> TableType { - TableType::Base - } - - async fn scan( - &self, - state: &dyn Session, - projection: Option<&Vec>, - filters: &[Expr], - _limit: Option, - ) -> DFResult> { - // Per-partition row limits are not yet implemented for IcebergPartitionedScan. - // DataFusion will apply a GlobalLimitExec on top of this node when needed. - - // Second load: fetch the latest snapshot so scans always reflect current table state. - let table = self - .catalog - .load_table(&self.table_ident) - .await - .map_err(to_datafusion_error)?; - - // Build a TableScan mirroring the inputs we'll hand to IcebergPartitionedScan, - // so plan_files() uses the same projection/filters the scan will replay in execute(). - let col_names = projection.map(|indices| { - indices - .iter() - .map(|&i| self.schema.field(i).name().clone()) - .collect::>() - }); - - let predicate = convert_filters_to_predicate(filters); - - let mut builder = table.scan(); - builder = match col_names { - Some(names) => builder.select(names), - None => builder.select_all(), - }; - if let Some(pred) = predicate { - builder = builder.with_filter(pred); - } - - let tasks: Vec = builder - .build() - .map_err(to_datafusion_error)? - .plan_files() - .await - .map_err(to_datafusion_error)? - .try_collect::>() - .await - .map_err(to_datafusion_error)?; - - // Output schema after projection: column indices in `Hash` exprs and any - // Arrow array we hash must reference this schema, not the full table schema. - let output_schema = match projection { - None => self.schema.clone(), - Some(p) => Arc::new(self.schema.project(p).map_err(|e| { - to_datafusion_error(Error::new(ErrorKind::DataInvalid, e.to_string())) - })?), - }; - - let target_partitions = state.config().target_partitions(); - let n_partitions = if tasks.is_empty() { - 0 - } else { - target_partitions.min(tasks.len()).max(1) - }; - - // identity_cols is Some(non-empty) iff every condition for declaring - // Partitioning::Hash is met: the table's default spec has identity-transform - // fields, every such source column is present in the output projection, and - // every column type is supported by literal_to_array. Any miss collapses to - // None, which forces UnknownPartitioning regardless of bucketing strategy. - let identity_cols = compute_identity_cols(&table, &output_schema); - - let (buckets, all_had_full_key) = - bucket_tasks(tasks, n_partitions, identity_cols.as_deref()); - - let partitioning = match identity_cols { - Some(cols) if !cols.is_empty() && all_had_full_key && n_partitions > 0 => { - let exprs: Vec> = cols - .iter() - .map(|c| Arc::new(Column::new(&c.name, c.output_idx)) as Arc) - .collect(); - Partitioning::Hash(exprs, n_partitions) - } - _ => Partitioning::UnknownPartitioning(n_partitions), - }; - - Ok(Arc::new(IcebergPartitionedScan::new( - table, - None, // Always use current snapshot for catalog-backed provider - self.schema.clone(), - projection, - filters, - buckets, - partitioning, - ))) - } - - fn supports_filters_pushdown( - &self, - filters: &[&Expr], - ) -> DFResult> { - Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) - } - - async fn insert_into( - &self, - _state: &dyn Session, - _input: Arc, - _insert_op: datafusion::logical_expr::dml::InsertOp, - ) -> DFResult> { - Err(to_datafusion_error(Error::new( - ErrorKind::FeatureUnsupported, - "IcebergPartitionedTableProvider does not support writes; \ - use IcebergTableProvider instead", - ))) - } -} - -/// Identity-partitioned column that is also present in the output projection -/// and whose Arrow type can be reconstructed from a `Literal` for hashing. -struct IdentityCol { - name: String, - /// Position of this column in the *output* schema (after projection). - output_idx: usize, - /// Position of this column inside the partition spec's `fields()` slice, - /// matching the slot order of `FileScanTask::partition`. - spec_field_idx: usize, - output_dtype: DataType, -} - -/// Inspect the table's default partition spec and return the list of identity -/// columns that can support a [`Partitioning::Hash`] declaration. Returns -/// `None` if any condition is violated: -/// - the source column for an identity field is not in the output projection -/// - the source column's Arrow type is not currently supported by -/// [`literal_to_array`] -/// - the table has spec evolution (>1 historical specs), since older files -/// may carry a partition tuple that does not align with the default spec -/// -/// Returning `None` forces the scan to declare `UnknownPartitioning` even if -/// bucketing succeeds. -fn compute_identity_cols(table: &Table, output_schema: &ArrowSchema) -> Option> { - let metadata = table.metadata(); - if metadata.partition_specs_iter().len() > 1 { - return None; - } - let spec = metadata.default_partition_spec(); - let table_schema = metadata.current_schema(); - - let mut cols = Vec::new(); - for (spec_field_idx, pf) in spec.fields().iter().enumerate() { - if pf.transform != Transform::Identity { - continue; - } - let source_field = table_schema.field_by_id(pf.source_id)?; - let output_idx = output_schema.index_of(source_field.name.as_str()).ok()?; - let output_dtype = output_schema.field(output_idx).data_type().clone(); - if !is_supported_dtype(&output_dtype) { - return None; - } - cols.push(IdentityCol { - name: source_field.name.clone(), - output_idx, - spec_field_idx, - output_dtype, - }); - } - Some(cols) -} - -fn is_supported_dtype(dt: &DataType) -> bool { - matches!( - dt, - DataType::Boolean - | DataType::Int32 - | DataType::Int64 - | DataType::Float32 - | DataType::Float64 - | DataType::Utf8 - | DataType::Date32 - ) -} - -/// Distribute `tasks` across `n_partitions` buckets. When `identity_cols` -/// describes a non-empty, hashable identity key, each task is hashed on -/// that key using DataFusion's repartition hash so the resulting partitioning -/// matches what `RepartitionExec` would produce on the same data. Tasks -/// missing partition data fall back to hashing `data_file_path`, which still -/// distributes evenly but breaks the `Hash` contract — the second tuple -/// element flags whether every task supplied a full identity key. -fn bucket_tasks( - tasks: Vec, - n_partitions: usize, - identity_cols: Option<&[IdentityCol]>, -) -> (Vec>, bool) { - if n_partitions == 0 { - return (Vec::new(), tasks.is_empty()); - } - let mut buckets: Vec> = (0..n_partitions).map(|_| Vec::new()).collect(); - let mut all_full_key = true; - let cols = identity_cols.unwrap_or(&[]); - - for task in tasks { - let bucket_idx = match identity_hash(&task, cols) { - Some(h) => (h % n_partitions as u64) as usize, - None => { - all_full_key = false; - fallback_hash(&task) as usize % n_partitions - } - }; - buckets[bucket_idx].push(task); - } - (buckets, all_full_key) -} - -/// Hash the identity-partition values of `task` using -/// [`REPARTITION_RANDOM_STATE`] so the bucket assignment matches DataFusion's -/// hash-repartition convention. Returns `None` if the task lacks partition -/// data or any required slot is null/unsupported. -fn identity_hash(task: &FileScanTask, cols: &[IdentityCol]) -> Option { - if cols.is_empty() { - return None; - } - let partition = task.partition.as_ref()?; - let mut arrays: Vec = Vec::with_capacity(cols.len()); - for col in cols { - let lit = partition.fields().get(col.spec_field_idx)?.as_ref()?; - arrays.push(literal_to_array(lit, &col.output_dtype)?); - } - let mut hashes = vec![0u64; 1]; - create_hashes( - &arrays, - REPARTITION_RANDOM_STATE.random_state(), - &mut hashes, - ) - .ok()?; - Some(hashes[0]) -} - -/// Deterministic per-file fallback used when `identity_hash` cannot produce a -/// bucket. The hash function does not need to match DataFusion's because any -/// task taking this path causes the scan to drop to `UnknownPartitioning`. -fn fallback_hash(task: &FileScanTask) -> u64 { - use std::collections::hash_map::DefaultHasher; - use std::hash::{Hash, Hasher}; - let mut hasher = DefaultHasher::new(); - task.data_file_path.hash(&mut hasher); - hasher.finish() -} - -/// Materialize a single-element Arrow array of `dt` holding the value of -/// `lit`. The Arrow type must match what DataFusion will see for this column -/// at scan time, otherwise `create_hashes` would dispatch on a different type -/// and produce a hash that disagrees with DataFusion's row-wise hashing. -fn literal_to_array(lit: &Literal, dt: &DataType) -> Option { - let prim = match lit { - Literal::Primitive(p) => p, - _ => return None, - }; - Some(match (prim, dt) { - (PrimitiveLiteral::Boolean(v), DataType::Boolean) => Arc::new(BooleanArray::from(vec![*v])), - (PrimitiveLiteral::Int(v), DataType::Int32) => Arc::new(Int32Array::from(vec![*v])), - (PrimitiveLiteral::Int(v), DataType::Date32) => Arc::new(Date32Array::from(vec![*v])), - (PrimitiveLiteral::Long(v), DataType::Int64) => Arc::new(Int64Array::from(vec![*v])), - (PrimitiveLiteral::Float(v), DataType::Float32) => Arc::new(Float32Array::from(vec![v.0])), - (PrimitiveLiteral::Double(v), DataType::Float64) => Arc::new(Float64Array::from(vec![v.0])), - (PrimitiveLiteral::String(v), DataType::Utf8) => { - Arc::new(StringArray::from(vec![v.as_str()])) - } - _ => return None, - }) -} - -#[cfg(test)] -mod tests { - use std::collections::HashMap; - use std::sync::Arc; - - use datafusion::prelude::{SessionConfig, SessionContext}; - use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; - use iceberg::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, NestedField, PrimitiveType, Schema, Type, - }; - use iceberg::transaction::{ApplyTransactionAction, Transaction}; - use iceberg::{Catalog, CatalogBuilder, NamespaceIdent, TableCreation, TableIdent}; - use tempfile::TempDir; - - use super::*; - - async fn make_catalog_and_table() -> (Arc, NamespaceIdent, String, TempDir) { - let temp_dir = TempDir::new().unwrap(); - let warehouse = temp_dir.path().to_str().unwrap().to_string(); - - let catalog = Arc::new( - MemoryCatalogBuilder::default() - .load( - "memory", - HashMap::from([(MEMORY_CATALOG_WAREHOUSE.to_string(), warehouse.clone())]), - ) - .await - .unwrap(), - ); - - let namespace = NamespaceIdent::new("ns".to_string()); - catalog - .create_namespace(&namespace, HashMap::new()) - .await - .unwrap(); - - let schema = Schema::builder() - .with_schema_id(0) - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), - ]) - .build() - .unwrap(); - - catalog - .create_table( - &namespace, - TableCreation::builder() - .name("t".to_string()) - .location(format!("{warehouse}/t")) - .schema(schema) - .properties(HashMap::new()) - .build(), - ) - .await - .unwrap(); - - (catalog, namespace, "t".to_string(), temp_dir) - } - - /// Registers `n` synthetic data files in the table metadata via the iceberg - /// transaction API. No actual parquet files are written, only the metadata - /// entries that `plan_files()` reads are created. - async fn append_fake_data_files( - catalog: &Arc, - namespace: &NamespaceIdent, - table_name: &str, - n: usize, - ) { - let table = catalog - .load_table(&TableIdent::new(namespace.clone(), table_name.to_string())) - .await - .unwrap(); - - let data_files = (0..n) - .map(|i| { - DataFileBuilder::default() - .content(DataContentType::Data) - .file_path(format!( - "{}/data/fake_{i}.parquet", - table.metadata().location() - )) - .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(128) - .record_count(1) - .partition_spec_id(table.metadata().default_partition_spec_id()) - .build() - .unwrap() - }) - .collect::>(); - - let tx = Transaction::new(&table); - let action = tx.fast_append().add_data_files(data_files); - action - .apply(tx) - .unwrap() - .commit(catalog.as_ref()) - .await - .unwrap(); - } - - fn ctx_with_target_partitions(n: usize) -> SessionContext { - SessionContext::new_with_config(SessionConfig::new().with_target_partitions(n)) - } - - /// An empty table must produce a zero-partition scan so DataFusion never calls - /// execute(0), which would otherwise return an out-of-bounds error. - #[tokio::test] - async fn test_empty_table_zero_partitions() { - let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; - // no files appended - let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) - .await - .unwrap(); - let plan = provider - .scan(&ctx_with_target_partitions(8).state(), None, &[], None) - .await - .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); - - assert_eq!(scan.buckets().len(), 0); - assert_eq!(scan.properties().partitioning.partition_count(), 0); - } - - /// When the table has no identity-partition columns, every task takes the - /// fallback (file_path) bucket path, so the declaration must drop to - /// `UnknownPartitioning`. The bucket count should still equal - /// min(target_partitions, num_files). - #[tokio::test] - async fn test_unpartitioned_falls_back_to_unknown() { - let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; - append_fake_data_files(&catalog, &namespace, &table_name, 5).await; - - let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) - .await - .unwrap(); - let plan = provider - .scan(&ctx_with_target_partitions(3).state(), None, &[], None) - .await - .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); - - let total_files: usize = scan.buckets().iter().map(|b| b.len()).sum(); - assert_eq!(total_files, 5); - assert_eq!(scan.buckets().len(), 3); - assert!(matches!( - scan.properties().partitioning, - Partitioning::UnknownPartitioning(3) - )); - } - - /// Bucket count must be capped at the number of files: spinning up more - /// DataFusion partitions than there are tasks would just leave empty - /// streams, wasting scheduler slots. - #[tokio::test] - async fn test_bucket_count_capped_at_file_count() { - let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; - append_fake_data_files(&catalog, &namespace, &table_name, 2).await; - - let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) - .await - .unwrap(); - let plan = provider - .scan(&ctx_with_target_partitions(16).state(), None, &[], None) - .await - .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); - - assert_eq!(scan.buckets().len(), 2); - } - - /// target_partitions = 1 collapses every task into a single bucket, giving - /// the same execution profile as `IcebergTableScan`. - #[tokio::test] - async fn test_single_target_partition_single_bucket() { - let (catalog, namespace, table_name, _temp_dir) = make_catalog_and_table().await; - append_fake_data_files(&catalog, &namespace, &table_name, 4).await; - - let provider = IcebergPartitionedTableProvider::try_new(catalog, namespace, table_name) - .await - .unwrap(); - let plan = provider - .scan(&ctx_with_target_partitions(1).state(), None, &[], None) - .await - .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); - - assert_eq!(scan.buckets().len(), 1); - assert_eq!(scan.buckets()[0].len(), 4); - } -} diff --git a/crates/integrations/datafusion/tests/integration_datafusion_test.rs b/crates/integrations/datafusion/tests/integration_datafusion_test.rs index cebac75dd9..8a58e94577 100644 --- a/crates/integrations/datafusion/tests/integration_datafusion_test.rs +++ b/crates/integrations/datafusion/tests/integration_datafusion_test.rs @@ -131,7 +131,7 @@ async fn test_provider_plan_stream_schema() -> Result<()> { let task_ctx = Arc::new(df.task_ctx()); let plan = df.create_physical_plan().await.unwrap(); - let stream = plan.execute(1, task_ctx).unwrap(); + let stream = plan.execute(0, task_ctx).unwrap(); // Ensure both the plan and the stream conform to the same schema assert_eq!(plan.schema(), stream.schema()); From 616dcdc669f9fe67601c437b1071f4431a9c264b Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Wed, 29 Apr 2026 14:40:51 +0200 Subject: [PATCH 13/16] refactor(datafusion): polish scan API and add bucketing tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Review pass over the partitioned-scan branch ahead of upstream contribution. - Rename `TableScan::to_arrow_with_tasks` to `to_arrow_from_tasks` — `from` better signals that the tasks are the input source rather than a builder-style modifier. - Restructure the doc with a `# Correctness` section that calls out the projection/filter contract while clarifying that reader-side configuration (concurrency, batch size, row-group filtering, row selection) is taken from `self`. - Make `IcebergTableScan::new` and `new_with_tasks` `pub` (were `pub(crate)`) so external users can construct the node directly, matching the public visibility of the struct itself. - Drop the `convert_filters_to_predicate` re-export from `physical_plan/mod.rs`: it was unused outside the module. - Extract a private `new_inner` constructor on `IcebergTableScan` so `new` and `new_with_tasks` share a single source of truth for the `PlanProperties` / projection / predicate setup. - Split `IcebergTableScan::execute` into a linear pipeline backed by three helpers: `build_table_scan` (synchronous scan-builder plumbing), `build_record_batch_stream` (async stream construction for the lazy/eager modes), and `apply_limit`. - Trim the `IcebergTableScan` struct doc and field comments to match the rest of the file's style; drop the verbose `to_arrow_with_tasks` rationale (the `# Correctness` doc carries the load-bearing info). - Tighten `DisplayAs::fmt_as`: remove the file-path enumeration (file count alone is enough for `EXPLAIN`) and factor the common prefix. - Trim several narrating comments in `table/mod.rs` and the module doc that duplicated information already evident from the code. - Add `test_identity_partitioned_declares_hash`: verifies the happy path where an identity-partitioned table with the partition column in the projection produces `Partitioning::Hash` referencing that column. This was the main missing coverage for the bucketing logic. - Add `test_projection_without_partition_col_falls_back_to_unknown`: verifies the `compute_identity_cols → None` branch when the projection omits the partition source column. - Add helpers (`make_partitioned_catalog_and_table_for_bucketing`, `append_partitioned_fake_data_files`) to build identity-partitioned fixtures without writing real Parquet files. (cherry picked from commit b1f2d6632509d6af474fe5a4be16341bd54ff13e) --- crates/iceberg/src/scan/mod.rs | 28 +- .../datafusion/src/physical_plan/mod.rs | 1 - .../datafusion/src/physical_plan/scan.rs | 315 ++++++++---------- .../integrations/datafusion/src/table/mod.rs | 211 +++++++++++- .../tests/integration_datafusion_test.rs | 18 +- 5 files changed, 362 insertions(+), 211 deletions(-) diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index eff1bce3ff..85e83f6e18 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -433,29 +433,21 @@ impl TableScan { /// Returns an [`ArrowRecordBatchStream`]. pub async fn to_arrow(&self) -> Result { - self.to_arrow_with_tasks(self.plan_files().await?) + self.to_arrow_from_tasks(self.plan_files().await?) } - /// Consumes an externally-planned [`FileScanTask`] stream and returns an - /// [`ArrowRecordBatchStream`] using this scan's [`ArrowReaderBuilder`] - /// configuration (row-group filtering, row selection, data-file - /// concurrency limit, batch size). - /// - /// Equivalent to [`TableScan::to_arrow`] — which delegates to this method - /// after awaiting [`TableScan::plan_files`] — but lets the caller supply - /// a pre-computed task stream. This decouples planning from reading, so - /// external executors (e.g. a DataFusion partitioned scan) can plan once, - /// distribute tasks across workers, and replay them here without - /// re-running `plan_files()`. + /// Like [`TableScan::to_arrow`], but accepts a caller-supplied + /// [`FileScanTask`] stream instead of running [`TableScan::plan_files`] + /// internally. /// /// # Correctness /// - /// The tasks passed in must have been produced by a [`TableScan`] whose - /// projection and filter match `self`: filters are already baked into - /// each [`FileScanTask::predicate`] at planning time and are not - /// re-applied here. Using tasks from a scan with a different projection - /// or filter yields undefined behavior. - pub fn to_arrow_with_tasks(&self, tasks: FileScanTaskStream) -> Result { + /// Tasks must come from a [`TableScan`] with the same projection and + /// filter as `self`: predicates are baked into each task at planning + /// time and are not re-applied here. Reader-side configuration + /// (concurrency, batch size, row-group filtering, row selection) is + /// taken from `self` and may differ from the planning scan. + pub fn to_arrow_from_tasks(&self, tasks: FileScanTaskStream) -> Result { let mut arrow_reader_builder = ArrowReaderBuilder::new(self.file_io.clone()) .with_data_file_concurrency_limit(self.concurrency_limit_data_files) .with_row_group_filtering_enabled(self.row_group_filtering_enabled) diff --git a/crates/integrations/datafusion/src/physical_plan/mod.rs b/crates/integrations/datafusion/src/physical_plan/mod.rs index aeac30de32..5a9845cde0 100644 --- a/crates/integrations/datafusion/src/physical_plan/mod.rs +++ b/crates/integrations/datafusion/src/physical_plan/mod.rs @@ -26,6 +26,5 @@ pub(crate) mod write; pub(crate) const DATA_FILES_COL_NAME: &str = "data_files"; -pub use expr_to_predicate::convert_filters_to_predicate; pub use project::project_with_partition; pub use scan::IcebergTableScan; diff --git a/crates/integrations/datafusion/src/physical_plan/scan.rs b/crates/integrations/datafusion/src/physical_plan/scan.rs index 49ff61e999..0a386bded7 100644 --- a/crates/integrations/datafusion/src/physical_plan/scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/scan.rs @@ -30,63 +30,45 @@ use datafusion::physical_plan::{DisplayAs, ExecutionPlan, Partitioning, PlanProp use datafusion::prelude::Expr; use futures::{Stream, TryStreamExt}; use iceberg::expr::Predicate; -use iceberg::scan::FileScanTask; +use iceberg::scan::{FileScanTask, TableScan}; use iceberg::table::Table; use super::expr_to_predicate::convert_filters_to_predicate; use crate::to_datafusion_error; -/// Manages the scanning process of an Iceberg [`Table`], encapsulating the -/// necessary details and computed properties required for execution planning. +/// Iceberg [`Table`] scan as a DataFusion [`ExecutionPlan`]. /// -/// When constructed with pre-planned [`FileScanTask`] buckets via -/// [`IcebergTableScan::new_with_tasks`], each DataFusion partition `i` streams -/// every task in `buckets[i]` using -/// [`TableScan::to_arrow_with_tasks`][iceberg::scan::TableScan::to_arrow_with_tasks]. +/// Has two construction modes: [`IcebergTableScan::new`] for a lazy +/// single-partition scan, and [`IcebergTableScan::new_with_tasks`] for an +/// eager multi-partition scan over pre-planned [`FileScanTask`] buckets. /// -/// When constructed via [`IcebergTableScan::new`] (no pre-planned tasks), the -/// full table is scanned lazily in a single partition using -/// [`TableScan::to_arrow`][iceberg::scan::TableScan::to_arrow]. This mode is -/// used by [`IcebergStaticTableProvider`][crate::table::IcebergStaticTableProvider]. -/// -/// In both modes the optional `limit` field truncates the output stream to at -/// most that many rows. -/// -/// Note: when using pre-planned tasks, the `TableScan` is rebuilt on every -/// `execute(partition)` call rather than cached. `TableScan` carries a -/// `PlanContext` with `Arc`-shared evaluator caches which is awkward to -/// serialize if this plan ever needs to be shipped across workers. The -/// per-build cost is bounded (no I/O), so the rebuild is kept for now; -/// revisit once the cross-worker story is clearer. +/// Note: in eager mode the underlying `TableScan` is rebuilt on every +/// `execute(partition)` call. The per-build cost is bounded (no I/O) and +/// keeps the plan free of `Arc`-shared evaluator caches that are awkward to +/// serialize across workers. #[derive(Debug)] pub struct IcebergTableScan { /// A table in the catalog. table: Table, /// Snapshot of the table to scan. snapshot_id: Option, - /// Stores certain, often expensive to compute, - /// plan properties used in query optimization. + /// Cached plan properties used by query optimization. plan_properties: Arc, /// Projection column names, None means all columns. projection: Option>, /// Filters to apply to the table scan. predicates: Option, - /// Pre-planned file scan tasks grouped by output DataFusion partition. - /// `None` in lazy mode (single-partition scan via `to_arrow()`). - /// `Some(buckets)` in eager mode: `buckets[i]` holds every task that - /// `execute(i)` will read. + /// Pre-planned file scan tasks per partition (eager mode), or `None` (lazy mode). buckets: Option>>, /// Optional limit on the number of rows to return. limit: Option, } impl IcebergTableScan { - /// Creates a lazy single-partition scan. - /// - /// All file tasks are discovered and read inside `execute(0)` via - /// [`TableScan::to_arrow`][iceberg::scan::TableScan::to_arrow]. - /// Used by [`IcebergStaticTableProvider`][crate::table::IcebergStaticTableProvider]. - pub(crate) fn new( + /// Creates a lazy single-partition scan that plans and reads all tasks + /// inside `execute(0)`. Used by + /// [`IcebergStaticTableProvider`][crate::table::IcebergStaticTableProvider]. + pub fn new( table: Table, snapshot_id: Option, schema: ArrowSchemaRef, @@ -94,47 +76,55 @@ impl IcebergTableScan { filters: &[Expr], limit: Option, ) -> Self { - let output_schema = match projection { - None => schema.clone(), - Some(projection) => Arc::new(schema.project(projection).unwrap()), - }; - let plan_properties = Arc::new(PlanProperties::new( - EquivalenceProperties::new(output_schema), + Self::new_inner( + table, + snapshot_id, + schema, + projection, + filters, + limit, Partitioning::UnknownPartitioning(1), - EmissionType::Incremental, - Boundedness::Bounded, - )); - let projection = get_column_names(schema, projection); - let predicates = convert_filters_to_predicate(filters); + None, + ) + } - Self { + /// Creates an eager multi-partition scan over pre-planned task buckets. + /// Partition `i` streams `buckets[i]`. The caller is responsible for + /// ensuring `partitioning` matches the bucketing. Used by + /// [`IcebergTableProvider`][crate::table::IcebergTableProvider]. + #[allow(clippy::too_many_arguments)] + pub fn new_with_tasks( + table: Table, + snapshot_id: Option, + schema: ArrowSchemaRef, + projection: Option<&Vec>, + filters: &[Expr], + limit: Option, + buckets: Vec>, + partitioning: Partitioning, + ) -> Self { + Self::new_inner( table, snapshot_id, - plan_properties, + schema, projection, - predicates, - buckets: None, + filters, limit, - } + partitioning, + Some(buckets), + ) } - /// Creates an eager multi-partition scan from pre-planned file task buckets. - /// - /// Each DataFusion partition `i` streams the tasks in `buckets[i]` via - /// [`TableScan::to_arrow_with_tasks`][iceberg::scan::TableScan::to_arrow_with_tasks]. - /// The `partitioning` argument is used directly for [`PlanProperties`], so the - /// caller is responsible for ensuring it matches the bucketing strategy. - /// Used by [`IcebergTableProvider`][crate::table::IcebergTableProvider]. #[allow(clippy::too_many_arguments)] - pub(crate) fn new_with_tasks( + fn new_inner( table: Table, snapshot_id: Option, schema: ArrowSchemaRef, projection: Option<&Vec>, filters: &[Expr], limit: Option, - buckets: Vec>, partitioning: Partitioning, + buckets: Option>>, ) -> Self { let output_schema = match projection { None => schema.clone(), @@ -155,7 +145,7 @@ impl IcebergTableScan { plan_properties, projection, predicates, - buckets: Some(buckets), + buckets, limit, } } @@ -226,79 +216,31 @@ impl ExecutionPlan for IcebergTableScan { partition: usize, _context: Arc, ) -> DFResult { - let table = self.table.clone(); - let snapshot_id = self.snapshot_id; - let column_names = self.projection.clone(); - let predicates = self.predicates.clone(); - let limit = self.limit; - - let stream = match &self.buckets { - Some(buckets) => { - // Eager mode: stream the pre-planned bucket for this partition. - let bucket = buckets.get(partition).cloned().ok_or_else(|| { - DataFusionError::Internal(format!( - "{}: partition index {partition} is out of bounds (total buckets: {})", - self.name(), - buckets.len() - )) - })?; - - let fut = async move { - // Rebuild a TableScan so we inherit the same defaults - // (row-group filtering, batch size, concurrency, ...). - let scan_builder = match snapshot_id { - Some(id) => table.scan().snapshot_id(id), - None => table.scan(), - }; - let mut scan_builder = match column_names { - Some(names) => scan_builder.select(names), - None => scan_builder.select_all(), - }; - if let Some(pred) = predicates { - scan_builder = scan_builder.with_filter(pred); - } - let table_scan = scan_builder.build().map_err(to_datafusion_error)?; - - let task_stream = Box::pin(futures::stream::iter( - bucket.into_iter().map(Ok::<_, iceberg::Error>), - )); - let record_batch_stream = table_scan - .to_arrow_with_tasks(task_stream) - .map_err(to_datafusion_error)? - .map_err(to_datafusion_error); - Ok::<_, DataFusionError>(record_batch_stream) - }; - - let s = futures::stream::once(fut).try_flatten(); - Box::pin(s) as Pin> + Send>> - } - None => { - // Lazy mode: discover and read all tasks inside execute(). - let fut = get_batch_stream(table, snapshot_id, column_names, predicates); - let s = futures::stream::once(fut).try_flatten(); - Box::pin(s) - } + let bucket = match &self.buckets { + Some(buckets) => Some(buckets.get(partition).cloned().ok_or_else(|| { + DataFusionError::Internal(format!( + "{}: partition index {partition} is out of bounds (total buckets: {})", + self.name(), + buckets.len() + )) + })?), + None => None, }; - // Apply limit if specified. - let limited_stream: Pin> + Send>> = - if let Some(limit) = limit { - let mut remaining = limit; - Box::pin(stream.try_filter_map(move |batch| { - futures::future::ready(if remaining == 0 { - Ok(None) - } else if batch.num_rows() <= remaining { - remaining -= batch.num_rows(); - Ok(Some(batch)) - } else { - let limited_batch = batch.slice(0, remaining); - remaining = 0; - Ok(Some(limited_batch)) - }) - })) - } else { - stream - }; + let fut = build_record_batch_stream( + self.table.clone(), + self.snapshot_id, + self.projection.clone(), + self.predicates.clone(), + bucket, + ); + let stream = Box::pin(futures::stream::once(fut).try_flatten()) + as Pin> + Send>>; + + let limited_stream = match self.limit { + Some(limit) => apply_limit(stream, limit), + None => stream, + }; Ok(Box::pin(RecordBatchStreamAdapter::new( self.schema(), @@ -315,37 +257,22 @@ impl DisplayAs for IcebergTableScan { ) -> std::fmt::Result { let projection = self .projection - .clone() + .as_deref() .map_or(String::new(), |v| v.join(",")); let predicate = self .predicates - .clone() - .map_or(String::new(), |p| format!("{p}")); - - match &self.buckets { - Some(buckets) => { - let file_count = self.total_file_count(); - let bucket_count = buckets.len(); - write!( - f, - "{} projection:[{projection}] predicate:[{predicate}] \ - buckets:[{bucket_count}] file_count:[{file_count}]", - self.name() - )?; - if file_count <= 5 { - let files = buckets - .iter() - .flat_map(|b| b.iter().map(|t| t.data_file_path())) - .collect::>() - .join(", "); - write!(f, " files:[{files}]")?; - } - } - None => write!( - f, - "{} projection:[{projection}] predicate:[{predicate}]", - self.name() - )?, + .as_ref() + .map_or(String::new(), |p| p.to_string()); + + write!( + f, + "{} projection:[{projection}] predicate:[{predicate}]", + self.name() + )?; + if let Some(buckets) = &self.buckets { + let file_count = self.total_file_count(); + let bucket_count = buckets.len(); + write!(f, " buckets:[{bucket_count}] file_count:[{file_count}]")?; } if let Some(limit) = self.limit { write!(f, " limit:[{limit}]")?; @@ -354,34 +281,78 @@ impl DisplayAs for IcebergTableScan { } } -/// Asynchronously retrieves a stream of [`RecordBatch`] instances from a -/// given table. Used in lazy (single-partition) scan mode. -async fn get_batch_stream( +fn build_table_scan( table: Table, snapshot_id: Option, column_names: Option>, predicates: Option, -) -> DFResult> + Send>>> { +) -> DFResult { let scan_builder = match snapshot_id { - Some(snapshot_id) => table.scan().snapshot_id(snapshot_id), + Some(id) => table.scan().snapshot_id(id), None => table.scan(), }; - let mut scan_builder = match column_names { - Some(column_names) => scan_builder.select(column_names), + Some(names) => scan_builder.select(names), None => scan_builder.select_all(), }; if let Some(pred) = predicates { scan_builder = scan_builder.with_filter(pred); } - let table_scan = scan_builder.build().map_err(to_datafusion_error)?; + scan_builder.build().map_err(to_datafusion_error) +} + +/// Builds the `RecordBatch` stream for a single partition. When `bucket` is +/// `Some`, streams the pre-planned tasks via `to_arrow_from_tasks`; when +/// `None`, plans and reads the full scan via `to_arrow`. +async fn build_record_batch_stream( + table: Table, + snapshot_id: Option, + column_names: Option>, + predicates: Option, + bucket: Option>, +) -> DFResult> + Send>>> { + let table_scan = build_table_scan(table, snapshot_id, column_names, predicates)?; + let stream: Pin> + Send>> = match bucket { + Some(bucket) => { + let task_stream = Box::pin(futures::stream::iter( + bucket.into_iter().map(Ok::<_, iceberg::Error>), + )); + Box::pin( + table_scan + .to_arrow_from_tasks(task_stream) + .map_err(to_datafusion_error)? + .map_err(to_datafusion_error), + ) + } + None => Box::pin( + table_scan + .to_arrow() + .await + .map_err(to_datafusion_error)? + .map_err(to_datafusion_error), + ), + }; + Ok(stream) +} - let stream = table_scan - .to_arrow() - .await - .map_err(to_datafusion_error)? - .map_err(to_datafusion_error); - Ok(Box::pin(stream)) +/// Truncates a stream of `RecordBatch` to at most `limit` rows. +fn apply_limit( + stream: Pin> + Send>>, + limit: usize, +) -> Pin> + Send>> { + let mut remaining = limit; + Box::pin(stream.try_filter_map(move |batch| { + futures::future::ready(if remaining == 0 { + Ok(None) + } else if batch.num_rows() <= remaining { + remaining -= batch.num_rows(); + Ok(Some(batch)) + } else { + let limited_batch = batch.slice(0, remaining); + remaining = 0; + Ok(Some(limited_batch)) + }) + })) } pub(super) fn get_column_names( diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index df9631cb3c..3c27587429 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -17,12 +17,10 @@ //! Iceberg table providers for DataFusion. //! -//! This module provides various table provider implementations: +//! This module provides two table provider implementations: //! //! - [`IcebergTableProvider`]: Catalog-backed provider with automatic metadata refresh. //! Use for write operations and when you need to see the latest table state. -//! On each scan, file tasks are eagerly planned and distributed across DataFusion -//! partitions for parallel execution. //! //! - [`IcebergStaticTableProvider`]: Static provider for read-only access to a specific //! table snapshot. Use for consistent analytical queries or time-travel scenarios. @@ -96,7 +94,6 @@ impl IcebergTableProvider { ) -> Result { let table_ident = TableIdent::new(namespace, name.into()); - // Load table once to get initial schema let table = catalog.load_table(&table_ident).await?; let schema = Arc::new(schema_to_arrow_schema(table.metadata().current_schema())?); @@ -111,7 +108,6 @@ impl IcebergTableProvider { &self, r#type: MetadataTableType, ) -> Result { - // Load fresh table metadata for metadata table access let table = self.catalog.load_table(&self.table_ident).await?; Ok(IcebergMetadataTableProvider { table, r#type }) } @@ -237,7 +233,6 @@ impl TableProvider for IcebergTableProvider { input: Arc, _insert_op: InsertOp, ) -> DFResult> { - // Load fresh table metadata from catalog let table = self .catalog .load_table(&self.table_ident) @@ -264,7 +259,6 @@ impl TableProvider for IcebergTableProvider { let repartitioned_plan = repartition(plan_with_partition, table.metadata_ref(), target_partitions)?; - // Apply sort node when it's not fanout mode let fanout_enabled = table .metadata() .properties() @@ -390,7 +384,6 @@ impl TableProvider for IcebergStaticTableProvider { filters: &[Expr], limit: Option, ) -> DFResult> { - // Use cached table (no refresh) Ok(Arc::new(IcebergTableScan::new( self.table.clone(), self.snapshot_id, @@ -942,9 +935,7 @@ mod tests { ); } - // ── IcebergTableProvider bucketed scan tests ───────────────────────────── - // (Originally from table/partitioned.rs; updated to use IcebergTableProvider - // and IcebergTableScan after the IcebergPartitionedTableProvider merge.) + // ── Bucketed scan tests ────────────────────────────────────────────────── async fn make_catalog_and_table_for_bucketing( ) -> (Arc, NamespaceIdent, String, tempfile::TempDir) { @@ -1153,4 +1144,202 @@ mod tests { assert_eq!(scan.buckets().len(), 1); assert_eq!(scan.buckets()[0].len(), 4); } + + async fn make_partitioned_catalog_and_table_for_bucketing( + ) -> (Arc, NamespaceIdent, String, tempfile::TempDir) { + use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; + use iceberg::spec::{ + NestedField, PrimitiveType, Schema, Transform, Type, UnboundPartitionSpec, + }; + use iceberg::{CatalogBuilder, TableCreation}; + + let temp_dir = tempfile::TempDir::new().unwrap(); + let warehouse = temp_dir.path().to_str().unwrap().to_string(); + + let catalog = Arc::new( + MemoryCatalogBuilder::default() + .load( + "memory", + std::collections::HashMap::from([( + MEMORY_CATALOG_WAREHOUSE.to_string(), + warehouse.clone(), + )]), + ) + .await + .unwrap(), + ); + + let namespace = NamespaceIdent::new("ns".to_string()); + catalog + .create_namespace(&namespace, std::collections::HashMap::new()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + let partition_spec = UnboundPartitionSpec::builder() + .with_spec_id(0) + .add_partition_field(2, "name_part", Transform::Identity) + .unwrap() + .build(); + + catalog + .create_table( + &namespace, + TableCreation::builder() + .name("t".to_string()) + .location(format!("{warehouse}/t")) + .schema(schema) + .partition_spec(partition_spec) + .properties(std::collections::HashMap::new()) + .build(), + ) + .await + .unwrap(); + + (catalog, namespace, "t".to_string(), temp_dir) + } + + /// Like [`append_fake_data_files`] but each file carries a partition tuple + /// matching the table's identity-partition spec on `name`. + async fn append_partitioned_fake_data_files( + catalog: &Arc, + namespace: &NamespaceIdent, + table_name: &str, + partition_values: Vec<&str>, + ) { + use iceberg::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, Literal, Struct, + }; + use iceberg::transaction::{ApplyTransactionAction, Transaction}; + + let table = catalog + .load_table(&TableIdent::new(namespace.clone(), table_name.to_string())) + .await + .unwrap(); + + let data_files = partition_values + .iter() + .enumerate() + .map(|(i, value)| { + DataFileBuilder::default() + .content(DataContentType::Data) + .file_path(format!( + "{}/data/fake_{i}.parquet", + table.metadata().location() + )) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(128) + .record_count(1) + .partition_spec_id(table.metadata().default_partition_spec_id()) + .partition(Struct::from_iter(vec![Some(Literal::string(*value))])) + .build() + .unwrap() + }) + .collect::>(); + + let tx = Transaction::new(&table); + let action = tx.fast_append().add_data_files(data_files); + action + .apply(tx) + .unwrap() + .commit(catalog.as_ref()) + .await + .unwrap(); + } + + /// Identity-partitioned table whose source column is in the projection + /// must produce `Partitioning::Hash` referencing that column. + #[tokio::test] + async fn test_identity_partitioned_declares_hash() { + use datafusion::physical_expr::expressions::Column; + use datafusion::physical_plan::Partitioning; + + let (catalog, namespace, table_name, _temp_dir) = + make_partitioned_catalog_and_table_for_bucketing().await; + append_partitioned_fake_data_files( + &catalog, + &namespace, + &table_name, + vec!["a", "b", "c", "a", "b", "c"], + ) + .await; + + let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + let plan = provider + .scan(&ctx_with_target_partitions(3).state(), None, &[], None) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + let total_files: usize = scan.buckets().iter().map(|b| b.len()).sum(); + assert_eq!(total_files, 6); + + match &scan.properties().partitioning { + Partitioning::Hash(exprs, n) => { + assert_eq!(*n, 3); + assert_eq!(exprs.len(), 1); + let col = exprs[0] + .as_any() + .downcast_ref::() + .expect("expected Column expr"); + assert_eq!(col.name(), "name"); + } + other => panic!("expected Partitioning::Hash, got {other:?}"), + } + } + + /// A projection that omits the partition source column drops + /// `compute_identity_cols` to `None`, collapsing to `UnknownPartitioning`. + #[tokio::test] + async fn test_projection_without_partition_col_falls_back_to_unknown() { + use datafusion::physical_plan::Partitioning; + + let (catalog, namespace, table_name, _temp_dir) = + make_partitioned_catalog_and_table_for_bucketing().await; + append_partitioned_fake_data_files( + &catalog, + &namespace, + &table_name, + vec!["a", "b"], + ) + .await; + + let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) + .await + .unwrap(); + // Project only "id" (idx 0), excluding the partition column "name" (idx 1). + let projection = vec![0_usize]; + let plan = provider + .scan( + &ctx_with_target_partitions(3).state(), + Some(&projection), + &[], + None, + ) + .await + .unwrap(); + let scan = plan + .as_any() + .downcast_ref::() + .unwrap(); + + assert!(matches!( + scan.properties().partitioning, + Partitioning::UnknownPartitioning(_) + )); + } + } diff --git a/crates/integrations/datafusion/tests/integration_datafusion_test.rs b/crates/integrations/datafusion/tests/integration_datafusion_test.rs index 8a58e94577..7603c8b7ab 100644 --- a/crates/integrations/datafusion/tests/integration_datafusion_test.rs +++ b/crates/integrations/datafusion/tests/integration_datafusion_test.rs @@ -600,8 +600,8 @@ async fn test_insert_into_nested() -> Result<()> { // Insert data with nested structs let insert_sql = r#" INSERT INTO catalog.test_insert_nested.nested_table - SELECT - 1 as id, + SELECT + 1 as id, 'Alice' as name, named_struct( 'address', named_struct( @@ -615,8 +615,8 @@ async fn test_insert_into_nested() -> Result<()> { ) ) as profile UNION ALL - SELECT - 2 as id, + SELECT + 2 as id, 'Bob' as name, named_struct( 'address', named_struct( @@ -738,15 +738,15 @@ async fn test_insert_into_nested() -> Result<()> { let df = ctx .sql( r#" - SELECT - id, + SELECT + id, name, profile.address.street, profile.address.city, profile.address.zip, profile.contact.email, profile.contact.phone - FROM catalog.test_insert_nested.nested_table + FROM catalog.test_insert_nested.nested_table ORDER BY id "#, ) @@ -852,8 +852,8 @@ async fn test_insert_into_partitioned() -> Result<()> { let df = ctx .sql( r#" - INSERT INTO catalog.test_partitioned_write.partitioned_table - VALUES + INSERT INTO catalog.test_partitioned_write.partitioned_table + VALUES (1, 'electronics', 'laptop'), (2, 'electronics', 'phone'), (3, 'books', 'novel'), From 581dde750b3b283224e9beb73462583bfefca52d Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Wed, 29 Apr 2026 15:36:04 +0200 Subject: [PATCH 14/16] test(sqllogictest): update EXPLAIN snapshots for eager bucketing output IcebergTableProvider::scan now plans files eagerly and buckets them across DataFusion partitions before returning the ExecutionPlan. As a result, IcebergTableScan's DisplayAs output always includes `buckets:[N] file_count:[M]` - even for unpartitioned tables where N = 1. Update the four .slt files whose EXPLAIN snapshots were missing this suffix, and fix the like_predicate_pushdown snapshots that also had a stale input_partitions count on RepartitionExec (the table now has multiple files across multiple buckets). (cherry picked from commit 6ae4a71247a158380257a4b6339348c8ff232f1c) --- .../sqllogictest/testdata/slts/df_test/basic_queries.slt | 2 +- .../testdata/slts/df_test/binary_predicate_pushdown.slt | 2 +- .../testdata/slts/df_test/boolean_predicate_pushdown.slt | 6 +++--- .../testdata/slts/df_test/like_predicate_pushdown.slt | 8 ++++---- .../slts/df_test/timestamp_predicate_pushdown.slt | 8 ++++---- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/crates/sqllogictest/testdata/slts/df_test/basic_queries.slt b/crates/sqllogictest/testdata/slts/df_test/basic_queries.slt index a5ca4de46a..d9933e0f87 100644 --- a/crates/sqllogictest/testdata/slts/df_test/basic_queries.slt +++ b/crates/sqllogictest/testdata/slts/df_test/basic_queries.slt @@ -53,7 +53,7 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=3 02)--CooperativeExec -03)----IcebergTableScan projection:[id,name,score,category] predicate:[] limit:[3] +03)----IcebergTableScan projection:[id,name,score,category] predicate:[] buckets:[1] file_count:[1] limit:[3] # Test SELECT * with ORDER BY and LIMIT query ITRT diff --git a/crates/sqllogictest/testdata/slts/df_test/binary_predicate_pushdown.slt b/crates/sqllogictest/testdata/slts/df_test/binary_predicate_pushdown.slt index aa68ab2762..249d52edd0 100644 --- a/crates/sqllogictest/testdata/slts/df_test/binary_predicate_pushdown.slt +++ b/crates/sqllogictest/testdata/slts/df_test/binary_predicate_pushdown.slt @@ -28,7 +28,7 @@ logical_plan physical_plan 01)FilterExec: data@1 = 0102 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,data] predicate:[data = 0102] +03)----IcebergTableScan projection:[id,data] predicate:[data = 0102] buckets:[1] file_count:[0] # Verify empty result from empty table query I? diff --git a/crates/sqllogictest/testdata/slts/df_test/boolean_predicate_pushdown.slt b/crates/sqllogictest/testdata/slts/df_test/boolean_predicate_pushdown.slt index 496f719261..b4596ba6ba 100644 --- a/crates/sqllogictest/testdata/slts/df_test/boolean_predicate_pushdown.slt +++ b/crates/sqllogictest/testdata/slts/df_test/boolean_predicate_pushdown.slt @@ -39,7 +39,7 @@ logical_plan physical_plan 01)FilterExec: is_active@1 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,is_active,description] predicate:[is_active = true] +03)----IcebergTableScan projection:[id,is_active,description] predicate:[is_active = true] buckets:[1] file_count:[1] # Query with is_active = true query ITT rowsort @@ -59,7 +59,7 @@ logical_plan physical_plan 01)FilterExec: NOT is_active@1 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,is_active,description] predicate:[is_active = false] +03)----IcebergTableScan projection:[id,is_active,description] predicate:[is_active = false] buckets:[1] file_count:[1] # Query with is_active = false query ITT rowsort @@ -78,7 +78,7 @@ logical_plan physical_plan 01)FilterExec: NOT is_active@1 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,is_active,description] predicate:[is_active = false] +03)----IcebergTableScan projection:[id,is_active,description] predicate:[is_active = false] buckets:[1] file_count:[1] # Query with is_active != true (includes false and NULL) query ITT rowsort diff --git a/crates/sqllogictest/testdata/slts/df_test/like_predicate_pushdown.slt b/crates/sqllogictest/testdata/slts/df_test/like_predicate_pushdown.slt index 3d8b151aa9..698046046a 100644 --- a/crates/sqllogictest/testdata/slts/df_test/like_predicate_pushdown.slt +++ b/crates/sqllogictest/testdata/slts/df_test/like_predicate_pushdown.slt @@ -36,8 +36,8 @@ logical_plan 02)--TableScan: default.default.test_unpartitioned_table projection=[id, name], partial_filters=[default.default.test_unpartitioned_table.name LIKE Utf8("Al%")] physical_plan 01)FilterExec: name@1 LIKE Al% -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,name] predicate:[name STARTS WITH "Al"] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +03)----IcebergTableScan projection:[id,name] predicate:[name STARTS WITH "Al"] buckets:[2] file_count:[2] # Test LIKE filtering with case-sensitive match query IT rowsort @@ -55,8 +55,8 @@ logical_plan 02)--TableScan: default.default.test_unpartitioned_table projection=[id, name], partial_filters=[default.default.test_unpartitioned_table.name NOT LIKE Utf8("Al%")] physical_plan 01)FilterExec: name@1 NOT LIKE Al% -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,name] predicate:[name NOT STARTS WITH "Al"] +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 +03)----IcebergTableScan projection:[id,name] predicate:[name NOT STARTS WITH "Al"] buckets:[3] file_count:[3] # Test NOT LIKE filtering query IT rowsort diff --git a/crates/sqllogictest/testdata/slts/df_test/timestamp_predicate_pushdown.slt b/crates/sqllogictest/testdata/slts/df_test/timestamp_predicate_pushdown.slt index ffa74173dc..47100cc36d 100644 --- a/crates/sqllogictest/testdata/slts/df_test/timestamp_predicate_pushdown.slt +++ b/crates/sqllogictest/testdata/slts/df_test/timestamp_predicate_pushdown.slt @@ -50,7 +50,7 @@ logical_plan physical_plan 01)FilterExec: ts@1 = 1672921800000000000 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,ts] predicate:[ts = 2023-01-05 12:30:00] +03)----IcebergTableScan projection:[id,ts] predicate:[ts = 2023-01-05 12:30:00] buckets:[1] file_count:[1] # Verify timestamp equality filtering works query I? @@ -68,7 +68,7 @@ logical_plan physical_plan 01)FilterExec: ts@1 > 1673308800000000000 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,ts] predicate:[ts > 2023-01-10 00:00:00] +03)----IcebergTableScan projection:[id,ts] predicate:[ts > 2023-01-10 00:00:00] buckets:[1] file_count:[1] # Verify timestamp greater than filtering query I? rowsort @@ -97,7 +97,7 @@ logical_plan physical_plan 01)FilterExec: ts@1 >= 1672876800000000000 AND ts@1 <= 1673827199000000000 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,ts] predicate:[(ts >= 2023-01-05 00:00:00) AND (ts <= 2023-01-15 23:59:59)] +03)----IcebergTableScan projection:[id,ts] predicate:[(ts >= 2023-01-05 00:00:00) AND (ts <= 2023-01-15 23:59:59)] buckets:[1] file_count:[1] # Test timestamp range predicate filtering query I? rowsort @@ -162,7 +162,7 @@ logical_plan physical_plan 01)FilterExec: ts@1 > 1672531200000000 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----IcebergTableScan projection:[id,ts] predicate:[ts > 2023-01-01 00:00:00] +03)----IcebergTableScan projection:[id,ts] predicate:[ts > 2023-01-01 00:00:00] buckets:[1] file_count:[1] query I? SELECT * FROM default.default.test_timestamp_micros WHERE ts > CAST('2023-01-01 00:00:00' AS TIMESTAMP) From 7ad9dcc42de196aa72d58ef017ab2d6e9d43c982 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Wed, 29 Apr 2026 16:47:46 +0200 Subject: [PATCH 15/16] fix(datafusion): resolve conflicts --- crates/iceberg/src/scan/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 85e83f6e18..d205f56a92 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -457,7 +457,7 @@ impl TableScan { arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size); } - arrow_reader_builder.build().read(tasks) + arrow_reader_builder.build().read(tasks).map(|r| r.stream()) } /// Returns a reference to the column names of the table scan. From 7ff1f6dafcb2515dd19a16a857521abef1522168 Mon Sep 17 00:00:00 2001 From: Charles-Antoine Leger Date: Wed, 29 Apr 2026 17:17:45 +0200 Subject: [PATCH 16/16] fix(datafusion): format --- .../datafusion/src/table/bucketing.rs | 4 +- .../integrations/datafusion/src/table/mod.rs | 63 +++++-------------- 2 files changed, 17 insertions(+), 50 deletions(-) diff --git a/crates/integrations/datafusion/src/table/bucketing.rs b/crates/integrations/datafusion/src/table/bucketing.rs index 4c58730ce5..55ab23b6fd 100644 --- a/crates/integrations/datafusion/src/table/bucketing.rs +++ b/crates/integrations/datafusion/src/table/bucketing.rs @@ -178,9 +178,7 @@ fn literal_to_array(lit: &Literal, dt: &DataType) -> Option { (PrimitiveLiteral::Int(v), DataType::Date32) => Arc::new(Date32Array::from(vec![*v])), (PrimitiveLiteral::Long(v), DataType::Int64) => Arc::new(Int64Array::from(vec![*v])), (PrimitiveLiteral::Float(v), DataType::Float32) => Arc::new(Float32Array::from(vec![v.0])), - (PrimitiveLiteral::Double(v), DataType::Float64) => { - Arc::new(Float64Array::from(vec![v.0])) - } + (PrimitiveLiteral::Double(v), DataType::Float64) => Arc::new(Float64Array::from(vec![v.0])), (PrimitiveLiteral::String(v), DataType::Utf8) => { Arc::new(StringArray::from(vec![v.as_str()])) } diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index 3c27587429..14aff7af3a 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -43,9 +43,8 @@ use datafusion::logical_expr::dml::InsertOp; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_expr::expressions::Column; -use datafusion::physical_plan::ExecutionPlan; -use datafusion::physical_plan::Partitioning; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; use futures::TryStreamExt; use iceberg::arrow::schema_to_arrow_schema; use iceberg::inspect::MetadataTableType; @@ -937,8 +936,8 @@ mod tests { // ── Bucketed scan tests ────────────────────────────────────────────────── - async fn make_catalog_and_table_for_bucketing( - ) -> (Arc, NamespaceIdent, String, tempfile::TempDir) { + async fn make_catalog_and_table_for_bucketing() + -> (Arc, NamespaceIdent, String, tempfile::TempDir) { use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; use iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; use iceberg::{CatalogBuilder, TableCreation}; @@ -1054,10 +1053,7 @@ mod tests { .scan(&ctx_with_target_partitions(8).state(), None, &[], None) .await .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); + let scan = plan.as_any().downcast_ref::().unwrap(); assert_eq!(scan.buckets().len(), 1); assert_eq!(scan.buckets()[0].len(), 0); @@ -1083,10 +1079,7 @@ mod tests { .scan(&ctx_with_target_partitions(3).state(), None, &[], None) .await .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); + let scan = plan.as_any().downcast_ref::().unwrap(); let total_files: usize = scan.buckets().iter().map(|b| b.len()).sum(); assert_eq!(total_files, 5); @@ -1113,10 +1106,7 @@ mod tests { .scan(&ctx_with_target_partitions(16).state(), None, &[], None) .await .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); + let scan = plan.as_any().downcast_ref::().unwrap(); assert_eq!(scan.buckets().len(), 2); } @@ -1136,17 +1126,14 @@ mod tests { .scan(&ctx_with_target_partitions(1).state(), None, &[], None) .await .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); + let scan = plan.as_any().downcast_ref::().unwrap(); assert_eq!(scan.buckets().len(), 1); assert_eq!(scan.buckets()[0].len(), 4); } - async fn make_partitioned_catalog_and_table_for_bucketing( - ) -> (Arc, NamespaceIdent, String, tempfile::TempDir) { + async fn make_partitioned_catalog_and_table_for_bucketing() + -> (Arc, NamespaceIdent, String, tempfile::TempDir) { use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; use iceberg::spec::{ NestedField, PrimitiveType, Schema, Transform, Type, UnboundPartitionSpec, @@ -1215,9 +1202,7 @@ mod tests { table_name: &str, partition_values: Vec<&str>, ) { - use iceberg::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, Literal, Struct, - }; + use iceberg::spec::{DataContentType, DataFileBuilder, DataFileFormat, Literal, Struct}; use iceberg::transaction::{ApplyTransactionAction, Transaction}; let table = catalog @@ -1264,12 +1249,9 @@ mod tests { let (catalog, namespace, table_name, _temp_dir) = make_partitioned_catalog_and_table_for_bucketing().await; - append_partitioned_fake_data_files( - &catalog, - &namespace, - &table_name, - vec!["a", "b", "c", "a", "b", "c"], - ) + append_partitioned_fake_data_files(&catalog, &namespace, &table_name, vec![ + "a", "b", "c", "a", "b", "c", + ]) .await; let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) @@ -1279,10 +1261,7 @@ mod tests { .scan(&ctx_with_target_partitions(3).state(), None, &[], None) .await .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); + let scan = plan.as_any().downcast_ref::().unwrap(); let total_files: usize = scan.buckets().iter().map(|b| b.len()).sum(); assert_eq!(total_files, 6); @@ -1309,13 +1288,7 @@ mod tests { let (catalog, namespace, table_name, _temp_dir) = make_partitioned_catalog_and_table_for_bucketing().await; - append_partitioned_fake_data_files( - &catalog, - &namespace, - &table_name, - vec!["a", "b"], - ) - .await; + append_partitioned_fake_data_files(&catalog, &namespace, &table_name, vec!["a", "b"]).await; let provider = IcebergTableProvider::try_new(catalog, namespace, table_name) .await @@ -1331,15 +1304,11 @@ mod tests { ) .await .unwrap(); - let scan = plan - .as_any() - .downcast_ref::() - .unwrap(); + let scan = plan.as_any().downcast_ref::().unwrap(); assert!(matches!( scan.properties().partitioning, Partitioning::UnknownPartitioning(_) )); } - }