Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
25 changes: 24 additions & 1 deletion crates/iceberg/src/scan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -432,6 +432,29 @@ impl TableScan {

/// Returns an [`ArrowRecordBatchStream`].
pub async fn to_arrow(&self) -> Result<ArrowRecordBatchStream> {
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<ArrowRecordBatchStream> {
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)
Expand All @@ -441,7 +464,7 @@ impl TableScan {
arrow_reader_builder = arrow_reader_builder.with_batch_size(batch_size);
}

arrow_reader_builder.build().read(self.plan_files().await?)
arrow_reader_builder.build().read(tasks)
}

/// Returns a reference to the column names of the table scan.
Expand Down
2 changes: 2 additions & 0 deletions crates/integrations/datafusion/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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::*;

Expand Down
3 changes: 3 additions & 0 deletions crates/integrations/datafusion/src/physical_plan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -26,5 +27,7 @@ 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;
253 changes: 253 additions & 0 deletions crates/integrations/datafusion/src/physical_plan/partitioned_scan.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,253 @@
// 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<TableScan>` 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<i64>,
/// Stores certain, often expensive to compute,
/// plan properties used in query optimization.
plan_properties: Arc<PlanProperties>,
/// Projection column names, None means all columns.
projection: Option<Vec<String>>,
/// Filters to apply to the table scan.
predicates: Option<Predicate>,
/// Pre-planned file scan tasks grouped by output DataFusion partition.
/// `buckets[i]` holds every task that `execute(i)` will read.
buckets: Vec<Vec<FileScanTask>>,
}

impl IcebergPartitionedScan {
pub(crate) fn new(
table: Table,
snapshot_id: Option<i64>,
schema: ArrowSchemaRef,
projection: Option<&Vec<usize>>,
filters: &[Expr],
buckets: Vec<Vec<FileScanTask>>,
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<i64> {
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<FileScanTask>] {
&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<dyn ExecutionPlan + 'static>> {
vec![]
}

fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> DFResult<Arc<dyn ExecutionPlan>> {
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<PlanProperties> {
&self.plan_properties
}

fn execute(
&self,
partition: usize,
_context: Arc<TaskContext>,
) -> DFResult<SendableRecordBatchStream> {
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::<Vec<_>>()
.join(", ");
write!(f, " files:[{files}]")?;
}
Ok(())
}
}
2 changes: 1 addition & 1 deletion crates/integrations/datafusion/src/physical_plan/scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,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<usize>>,
) -> Option<Vec<String>> {
Expand Down
7 changes: 6 additions & 1 deletion crates/integrations/datafusion/src/table/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,20 @@

//! Iceberg table providers for DataFusion.
//!
//! This module provides two 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.
//!
//! - [`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;
Expand Down
Loading