tshauck commented on code in PR #324:
URL: https://github.com/apache/iceberg-rust/pull/324#discussion_r1561209884


##########
Cargo.toml:
##########
@@ -21,6 +21,7 @@ members = [
     "crates/catalog/*",
     "crates/examples",
     "crates/iceberg",
+    "crates/integrations",

Review Comment:
   Perhaps as a V2 of this integration it'd make sense to use features and/or 
move the datafusion dep down into the integrations crate (or maybe have a crate 
per integration).
   
   To your point about polars or other extensions, if this concept grew the 
dependency load would get challenging for folks who don't need it.



##########
crates/integrations/src/datafusion/schema.rs:
##########
@@ -0,0 +1,97 @@
+// 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, sync::Arc};
+
+use dashmap::DashMap;
+use datafusion::{
+    catalog::schema::SchemaProvider, datasource::TableProvider, 
error::DataFusionError,
+};
+use futures::FutureExt;
+use iceberg::{Catalog, NamespaceIdent, Result};
+
+use crate::datafusion::table::IcebergTableProvider;
+
+pub(crate) struct IcebergSchemaProvider {
+    tables: DashMap<String, Arc<dyn TableProvider>>,
+}
+
+impl IcebergSchemaProvider {
+    pub(crate) async fn try_new(
+        client: Arc<dyn Catalog>,
+        namespace: &NamespaceIdent,
+    ) -> Result<Self> {
+        let table_names: Vec<String> = client
+            .list_tables(namespace)
+            .await?
+            .iter()
+            .map(|t| t.name().to_owned())
+            .collect();
+
+        let mut tables = Vec::new();
+        for name in table_names {
+            let provider = IcebergTableProvider::try_new(client.clone(), 
namespace, &name).await?;
+            let provider = Arc::new(provider) as Arc<dyn TableProvider>;
+
+            tables.push((name, provider));
+        }
+
+        Ok(IcebergSchemaProvider {
+            tables: tables.into_iter().collect(),
+        })
+    }
+}
+
+impl SchemaProvider for IcebergSchemaProvider {

Review Comment:
   I realize these are just stubs, I'd be happy to help fill in the guts here 
and for the `TableProvider` traits.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to