Skip to content
56 changes: 54 additions & 2 deletions crates/integrations/datafusion/src/table/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,17 +24,22 @@ 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::dml::InsertOp;
use datafusion::logical_expr::{Expr, TableProviderFilterPushDown};
use datafusion::physical_plan::ExecutionPlan;
use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
use iceberg::arrow::schema_to_arrow_schema;
use iceberg::inspect::MetadataTableType;
use iceberg::table::Table;
use iceberg::{Catalog, Error, ErrorKind, NamespaceIdent, Result, TableIdent};
use metadata_table::IcebergMetadataTableProvider;

use crate::physical_plan::commit::IcebergCommitExec;
use crate::physical_plan::scan::IcebergTableScan;
use crate::physical_plan::write::IcebergWriteExec;

/// Represents a [`TableProvider`] for the Iceberg [`Catalog`],
/// managing access to a [`Table`].
Expand All @@ -46,6 +51,8 @@ pub struct IcebergTableProvider {
snapshot_id: Option<i64>,
/// A reference-counted arrow `Schema`.
schema: ArrowSchemaRef,
/// The catalog that the table belongs to.
catalog: Option<Arc<dyn Catalog>>,
}

impl IcebergTableProvider {
Expand All @@ -54,6 +61,7 @@ impl IcebergTableProvider {
table,
snapshot_id: None,
schema,
catalog: None,
}
}
/// Asynchronously tries to construct a new [`IcebergTableProvider`]
Expand All @@ -73,6 +81,7 @@ impl IcebergTableProvider {
table,
snapshot_id: None,
schema,
catalog: Some(client),
})
}

Expand All @@ -84,6 +93,7 @@ impl IcebergTableProvider {
table,
snapshot_id: None,
schema,
catalog: None,
})
}

Expand All @@ -108,6 +118,7 @@ impl IcebergTableProvider {
table,
snapshot_id: Some(snapshot_id),
schema,
catalog: None,
})
}

Expand Down Expand Up @@ -152,11 +163,52 @@ impl TableProvider for IcebergTableProvider {
fn supports_filters_pushdown(
&self,
filters: &[&Expr],
) -> std::result::Result<Vec<TableProviderFilterPushDown>, datafusion::error::DataFusionError>
{
) -> DFResult<Vec<TableProviderFilterPushDown>> {
// Push down all filters, as a single source of truth, the scanner will drop the filters which couldn't be push down
Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()])
}

async fn insert_into(
&self,
_state: &dyn Session,
input: Arc<dyn ExecutionPlan>,
_insert_op: InsertOp,
) -> DFResult<Arc<dyn ExecutionPlan>> {
if !self
.table
Copy link
Contributor Author

@CTTY CTTY Aug 14, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ideally we should refresh the table here and every otherself.table usages in IcebergTableProvider, but I think we should fix that in a separate PR if needed

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds reasonable to me.

.metadata()
.default_partition_spec()
.is_unpartitioned()
{
// TODO add insert into support for partitioned tables
return Err(DataFusionError::NotImplemented(
"IcebergTableProvider::insert_into does not support partitioned tables yet"
.to_string(),
));
}

let Some(catalog) = self.catalog.clone() else {
return Err(DataFusionError::Execution(
"Catalog cannot be none for insert_into".to_string(),
));
};

let write_plan = Arc::new(IcebergWriteExec::new(
self.table.clone(),
input,
self.schema.clone(),
));

// Merge the outputs of write_plan into one so we can commit all files together
let coalesce_partitions = Arc::new(CoalescePartitionsExec::new(write_plan));

Ok(Arc::new(IcebergCommitExec::new(
self.table.clone(),
catalog,
coalesce_partitions,
self.schema.clone(),
)))
}
}

#[cfg(test)]
Expand Down
Loading
Loading