alamb commented on code in PR #13722: URL: https://github.com/apache/datafusion/pull/13722#discussion_r1885905925
########## datafusion-examples/examples/remote_catalog.rs: ########## @@ -0,0 +1,404 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// 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. + +/// This example shows how to implement the DataFusion [`CatalogProvider`] API +/// for catalogs that are remote (require network access) and/or offer only +/// asynchronous APIs such as [Polaris], [Unity], and [Hive]. +/// +/// Integrating with this catalogs is a bit more complex than with local +/// catalogs because calls like `ctx.sql("SELECT * FROM db.schm.tbl")` may need +/// to perform remote network requests, but many Catalog APIs are synchronous. +/// See the documentation on [`CatalogProvider`] for more details. +/// +/// [`CatalogProvider`]: datafusion_catalog::CatalogProvider +/// +/// [Polaris]: https://github.com/apache/polaris +/// [Unity]: https://github.com/unitycatalog/unitycatalog +/// [Hive]: https://hive.apache.org/ +use arrow::array::record_batch; +use arrow_schema::{Field, Fields, Schema, SchemaRef}; +use async_trait::async_trait; +use datafusion::catalog::{SchemaProvider, TableProvider}; +use datafusion::common::DataFusionError; +use datafusion::common::Result; +use datafusion::execution::SendableRecordBatchStream; +use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::{DataFrame, SessionContext}; +use datafusion_catalog::Session; +use datafusion_common::{ + assert_batches_eq, internal_datafusion_err, plan_err, TableReference, +}; +use datafusion_expr::{Expr, TableType}; +use futures::TryStreamExt; +use std::any::Any; +use std::sync::{Arc, Mutex}; + +#[tokio::main] +async fn main() -> Result<()> { + // As always, we create a session context to interact with DataFusion + let ctx = SessionContext::new(); + + // Make a connection to the remote catalog, asynchronously, and configure it + let remote_catalog_interface = RemoteCatalogInterface::connect().await?; + + // Register a SchemaProvider for tables in a schema named "remote_schema". + // + // This will let DataFusion query tables such as + // `datafusion.remote_schema.remote_table` + let remote_schema: Arc<dyn SchemaProvider> = + Arc::new(RemoteSchema::new(Arc::new(remote_catalog_interface))); + ctx.catalog("datafusion") + .ok_or_else(|| internal_datafusion_err!("default catalog was not installed"))? + .register_schema("remote_schema", Arc::clone(&remote_schema))?; + + // Here is a query that selects data from a table in the remote catalog. + let sql = "SELECT * from remote_schema.remote_table"; + + // While the `SessionContext::sql` interface is async, but it does not + // support asynchronous access to catalogs, so the following query errors. + let results = ctx.sql(sql).await; + assert_eq!( + results.unwrap_err().to_string(), + "Error during planning: table 'datafusion.remote_schema.remote_table' not found" + ); + + // Instead, to use a remote catalog, we must use lower level APIs on + // SessionState (what `SessionContext::sql` does internally). + let state = ctx.state(); + + // First, parse the SQL (but don't plan it / resolve any table references) + let dialect = state.config().options().sql_parser.dialect.as_str(); + let statement = state.sql_to_statement(sql, dialect)?; + + // Find all `TableReferences` in the parsed queries. These correspond to the + // tables referred to by the query (in this case + // `remote_schema.remote_table`) + let references = state.resolve_table_references(&statement)?; + + // Call `load_tables` to load information from the remote catalog for each + // of the referenced tables. Best practice is to fetch the the information + // for all tables required by the query once (rather than one per table) to + // minimize network overhead + let table_names = references.iter().filter_map(|r| { + if refers_to_schema("datafusion", "remote_schema", r) { + Some(r.table()) + } else { + None + } + }); + remote_schema + .as_any() + .downcast_ref::<RemoteSchema>() + .expect("correct types") + .load_tables(table_names) + .await?; + + // Now continue planing the query after having fetched the remote table and + // it can run as normal + let plan = state.statement_to_plan(statement).await?; + let results = DataFrame::new(state, plan).collect().await?; + assert_batches_eq!( + [ + "+----+-------+", + "| id | name |", + "+----+-------+", + "| 1 | alpha |", + "| 2 | beta |", + "| 3 | gamma |", + "+----+-------+", + ], + &results + ); + + Ok(()) +} + +/// This is an example of an API that interacts with a remote catalog. +/// +/// Specifically, its APIs are all `async` and thus can not be used by +/// [`SchemaProvider`] or [`TableProvider`] directly. +#[derive(Debug)] +struct RemoteCatalogInterface {} + +impl RemoteCatalogInterface { + /// Establish a connection to the remote catalog + pub async fn connect() -> Result<Self> { + // In a real implementation this method might connect to a remote + // catalog, validate credentials, cache basic information, etc + Ok(Self {}) + } + + /// Fetches information for a specific table + pub async fn table_info(&self, name: &str) -> Result<SchemaRef> { + if name != "remote_table" { + return plan_err!("Remote table not found: {}", name); + } + + // In this example, we'll model a remote table with columns "id" and + // "name" + // + // A real remote catalog would make a network call to fetch this + // information from a remote source. + let schema = Schema::new(Fields::from(vec![ + Field::new("id", arrow::datatypes::DataType::Int32, false), + Field::new("name", arrow::datatypes::DataType::Utf8, false), + ])); + Ok(Arc::new(schema)) + } + + /// Fetches data for a table from a remote data source + pub async fn read_data(&self, name: &str) -> Result<SendableRecordBatchStream> { + if name != "remote_table" { + return plan_err!("Remote table not found: {}", name); + } + + // In a real remote catalog this call would likely perform network IO to + // open and begin reading from a remote datasource, prefetching + // information, etc. + // + // In this example we are just demonstrating how the API works so simply + // return back some static data as a stream. + let batch = record_batch!( + ("id", Int32, [1, 2, 3]), + ("name", Utf8, ["alpha", "beta", "gamma"]) + ) + .unwrap(); + let schema = batch.schema(); + + let stream = futures::stream::iter([Ok(batch)]); + Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream))) + } +} + +/// Implements the DataFusion Catalog API interface for tables +/// stored in a remote catalog. +#[derive(Debug)] +struct RemoteSchema { + /// Connection with the remote catalog + remote_catalog_interface: Arc<RemoteCatalogInterface>, + /// Local cache of tables that have been preloaded from the remote + /// catalog + tables: Mutex<Vec<Arc<dyn TableProvider>>>, Review Comment: I was likely being pedantic here and trying to avoid storing the table name twice (once in the Catalog and once on the Table). I think perhaps that is just more confusing -- I'll change this to use HashMap -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
