|
| 1 | +// Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +// or more contributor license agreements. See the NOTICE file |
| 3 | +// distributed with this work for additional information |
| 4 | +// regarding copyright ownership. The ASF licenses this file |
| 5 | +// to you under the Apache License, Version 2.0 (the |
| 6 | +// "License"); you may not use this file except in compliance |
| 7 | +// with the License. You may obtain a copy of the License at |
| 8 | +// |
| 9 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +// |
| 11 | +// Unless required by applicable law or agreed to in writing, |
| 12 | +// software distributed under the License is distributed on an |
| 13 | +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +// KIND, either express or implied. See the License for the |
| 15 | +// specific language governing permissions and limitations |
| 16 | +// under the License. |
| 17 | + |
| 18 | +use std::{any::Any, collections::HashMap, sync::Arc}; |
| 19 | + |
| 20 | +use datafusion::catalog::{schema::SchemaProvider, CatalogProvider}; |
| 21 | +use futures::future::try_join_all; |
| 22 | +use iceberg::{Catalog, NamespaceIdent, Result}; |
| 23 | + |
| 24 | +use crate::schema::IcebergSchemaProvider; |
| 25 | + |
| 26 | +/// Provides an interface to manage and access multiple schemas |
| 27 | +/// within an Iceberg [`Catalog`]. |
| 28 | +/// |
| 29 | +/// Acts as a centralized catalog provider that aggregates |
| 30 | +/// multiple [`SchemaProvider`], each associated with distinct namespaces. |
| 31 | +pub struct IcebergCatalogProvider { |
| 32 | + /// A `HashMap` where keys are namespace names |
| 33 | + /// and values are dynamic references to objects implementing the |
| 34 | + /// [`SchemaProvider`] trait. |
| 35 | + schemas: HashMap<String, Arc<dyn SchemaProvider>>, |
| 36 | +} |
| 37 | + |
| 38 | +impl IcebergCatalogProvider { |
| 39 | + /// Asynchronously tries to construct a new [`IcebergCatalogProvider`] |
| 40 | + /// using the given client to fetch and initialize schema providers for |
| 41 | + /// each namespace in the Iceberg [`Catalog`]. |
| 42 | + /// |
| 43 | + /// This method retrieves the list of namespace names |
| 44 | + /// attempts to create a schema provider for each namespace, and |
| 45 | + /// collects these providers into a `HashMap`. |
| 46 | + pub async fn try_new(client: Arc<dyn Catalog>) -> Result<Self> { |
| 47 | + // TODO: |
| 48 | + // Schemas and providers should be cached and evicted based on time |
| 49 | + // As of right now; schemas might become stale. |
| 50 | + let schema_names: Vec<_> = client |
| 51 | + .list_namespaces(None) |
| 52 | + .await? |
| 53 | + .iter() |
| 54 | + .flat_map(|ns| ns.as_ref().clone()) |
| 55 | + .collect(); |
| 56 | + |
| 57 | + let providers = try_join_all( |
| 58 | + schema_names |
| 59 | + .iter() |
| 60 | + .map(|name| { |
| 61 | + IcebergSchemaProvider::try_new( |
| 62 | + client.clone(), |
| 63 | + NamespaceIdent::new(name.clone()), |
| 64 | + ) |
| 65 | + }) |
| 66 | + .collect::<Vec<_>>(), |
| 67 | + ) |
| 68 | + .await?; |
| 69 | + |
| 70 | + let schemas: HashMap<String, Arc<dyn SchemaProvider>> = schema_names |
| 71 | + .into_iter() |
| 72 | + .zip(providers.into_iter()) |
| 73 | + .map(|(name, provider)| { |
| 74 | + let provider = Arc::new(provider) as Arc<dyn SchemaProvider>; |
| 75 | + (name, provider) |
| 76 | + }) |
| 77 | + .collect(); |
| 78 | + |
| 79 | + Ok(IcebergCatalogProvider { schemas }) |
| 80 | + } |
| 81 | +} |
| 82 | + |
| 83 | +impl CatalogProvider for IcebergCatalogProvider { |
| 84 | + fn as_any(&self) -> &dyn Any { |
| 85 | + self |
| 86 | + } |
| 87 | + |
| 88 | + fn schema_names(&self) -> Vec<String> { |
| 89 | + self.schemas.keys().cloned().collect() |
| 90 | + } |
| 91 | + |
| 92 | + fn schema(&self, name: &str) -> Option<Arc<dyn SchemaProvider>> { |
| 93 | + self.schemas.get(name).cloned() |
| 94 | + } |
| 95 | +} |
0 commit comments