Skip to content
Open
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
1 change: 1 addition & 0 deletions bindings/python/Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

7 changes: 5 additions & 2 deletions bindings/python/src/datafusion_table_provider.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ use iceberg::TableIdent;
use iceberg::io::FileIO;
use iceberg::table::StaticTable;
use iceberg_datafusion::table::IcebergTableProvider;
use iceberg_datafusion::table::static_catalog::StaticCatalog;
use pyo3::exceptions::PyRuntimeError;
use pyo3::prelude::*;
use pyo3::types::PyCapsule;
Expand Down Expand Up @@ -61,15 +62,17 @@ impl PyIcebergDataFusionTable {
.map_err(|e| PyRuntimeError::new_err(format!("Failed to build FileIO: {e}")))?;

let static_table =
StaticTable::from_metadata_file(&metadata_location, table_ident, file_io)
StaticTable::from_metadata_file(&metadata_location, table_ident.clone(), file_io)
.await
.map_err(|e| {
PyRuntimeError::new_err(format!("Failed to load static table: {e}"))
})?;

let table = static_table.into_table();

IcebergTableProvider::try_new_from_table(table)
let static_catalog = Arc::new(StaticCatalog::new(table));

IcebergTableProvider::try_new(static_catalog, table_ident)
.await
.map_err(|e| {
PyRuntimeError::new_err(format!("Failed to create table provider: {e}"))
Expand Down
13 changes: 13 additions & 0 deletions crates/iceberg/src/catalog/memory/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,19 @@ impl MemoryCatalog {
warehouse_location,
}
}

/// Register an existing table in the memory catalog.
pub async fn register_existing_table(
&self,
table_ident: &TableIdent,
metadata_location: String,
) -> Result<()> {
let mut root_namespace_state = self.root_namespace_state.lock().await;

root_namespace_state.insert_new_table(table_ident, metadata_location.clone())?;

Ok(())
}
}

#[async_trait]
Expand Down
16 changes: 7 additions & 9 deletions crates/integration_tests/tests/shared_tests/datafusion.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ use datafusion::assert_batches_eq;
use datafusion::catalog::TableProvider;
use datafusion::error::DataFusionError;
use datafusion::prelude::SessionContext;
use iceberg::{Catalog, TableIdent};
use iceberg::TableIdent;
use iceberg_catalog_rest::RestCatalog;
use iceberg_datafusion::IcebergTableProvider;
use parquet::arrow::PARQUET_FIELD_ID_META_KEY;
Expand All @@ -36,17 +36,15 @@ async fn test_basic_queries() -> Result<(), DataFusionError> {
let fixture = get_shared_containers();
let rest_catalog = RestCatalog::new(fixture.catalog_config.clone());

let table = rest_catalog
.load_table(&TableIdent::from_strs(["default", "types_test"]).unwrap())
.await
.unwrap();

let ctx = SessionContext::new();

let table_provider = Arc::new(
IcebergTableProvider::try_new_from_table(table)
.await
.unwrap(),
IcebergTableProvider::try_new(
Arc::new(rest_catalog),
TableIdent::from_strs(["default", "types_test"]).unwrap(),
)
.await
.unwrap(),
);

let schema = table_provider.schema();
Expand Down
9 changes: 6 additions & 3 deletions crates/integrations/datafusion/src/schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ use datafusion::datasource::TableProvider;
use datafusion::error::{DataFusionError, Result as DFResult};
use futures::future::try_join_all;
use iceberg::inspect::MetadataTableType;
use iceberg::{Catalog, NamespaceIdent, Result};
use iceberg::{Catalog, NamespaceIdent, Result, TableIdent};

use crate::table::IcebergTableProvider;

Expand Down Expand Up @@ -65,7 +65,10 @@ impl IcebergSchemaProvider {
let providers = try_join_all(
table_names
.iter()
.map(|name| IcebergTableProvider::try_new(client.clone(), namespace.clone(), name))
.map(|name| {
let table_ident = TableIdent::new(namespace.clone(), name.clone());
IcebergTableProvider::try_new(client.clone(), table_ident)
})
.collect::<Vec<_>>(),
)
.await?;
Expand Down Expand Up @@ -113,7 +116,7 @@ impl SchemaProvider for IcebergSchemaProvider {
let metadata_table_type =
MetadataTableType::try_from(metadata_table_name).map_err(DataFusionError::Plan)?;
if let Some(table) = self.tables.get(table_name) {
let metadata_table = table.metadata_table(metadata_table_type);
let metadata_table = table.metadata_table(metadata_table_type).await;
return Ok(Some(Arc::new(metadata_table)));
} else {
return Ok(None);
Expand Down
Loading
Loading