// 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, collections::HashMap, sync::Arc}; use async_trait::async_trait; use datafusion::error::Result as DFResult; use datafusion::{catalog::schema::SchemaProvider, datasource::TableProvider}; use futures::future::try_join_all; use iceberg::{Catalog, NamespaceIdent, Result}; use crate::table::IcebergTableProvider; /// Represents a [`SchemaProvider`] for the Iceberg [`Catalog`], managing /// access to table providers within a specific namespace. pub(crate) struct IcebergSchemaProvider { /// A `HashMap` where keys are table names /// and values are dynamic references to objects implementing the /// [`TableProvider`] trait. tables: HashMap>, } impl IcebergSchemaProvider { /// Asynchronously tries to construct a new [`IcebergSchemaProvider`] /// using the given client to fetch and initialize table providers for /// the provided namespace in the Iceberg [`Catalog`]. /// /// This method retrieves a list of table names /// attempts to create a table provider for each table name, and /// collects these providers into a `HashMap`. pub(crate) async fn try_new( client: Arc, namespace: NamespaceIdent, ) -> Result { // TODO: // Tables and providers should be cached based on table_name // if we have a cache miss; we update our internal cache & check again // As of right now; tables might become stale. let table_names: Vec<_> = client .list_tables(&namespace) .await? .iter() .map(|tbl| tbl.name().to_string()) .collect(); let providers = try_join_all( table_names .iter() .map(|name| IcebergTableProvider::try_new(client.clone(), namespace.clone(), name)) .collect::>(), ) .await?; let tables: HashMap> = table_names .into_iter() .zip(providers.into_iter()) .map(|(name, provider)| { let provider = Arc::new(provider) as Arc; (name, provider) }) .collect(); Ok(IcebergSchemaProvider { tables }) } } #[async_trait] impl SchemaProvider for IcebergSchemaProvider { fn as_any(&self) -> &dyn Any { self } fn table_names(&self) -> Vec { self.tables.keys().cloned().collect() } fn table_exist(&self, name: &str) -> bool { self.tables.get(name).is_some() } async fn table(&self, name: &str) -> DFResult>> { Ok(self.tables.get(name).cloned()) } }