JanKaul commented on issue #3777:
URL: 
https://github.com/apache/arrow-datafusion/issues/3777#issuecomment-1310752816

   **Is your feature request related to a problem or challenge? Please describe
   what you are trying to do.**
   
   The traits `CatalogList`/`CatalogProvider`/`SchemaProvider` currently 
provide a
   synchronous interfaces for catalog and schema information. But due to their
   synchronous nature it is not possible to implement the traits for use cases
   where the information cannot simply be stored in memory. These use cases 
include
   catalog implementations based on:
   
   - ZooKeeper/Etcd
   - Databases
   - Catalogs for Delta_lake/Iceberg
   - API endpoints
   
   **Describe the solution you'd like**
   
   The same traits with asynchronous functions. The proposed traits would look 
like
   the following:
   
   ```rust
   [async_trait]
   pub trait CatalogList: Sync + Send {
       fn as_any(&self) -> &dyn Any;
       async fn register_catalog(
           &self,
           name: String,
           catalog: Arc<dyn CatalogProvider>
       ) -> Option<Arc<dyn CatalogProvider>>;
       async fn catalog_names(&self) -> Vec<String>ⓘ;
       async fn catalog(&self, name: &str) -> Option<Arc<dyn CatalogProvider>>;
   }
   ```
   
   ```rust
   [async_trait]
   pub trait CatalogProvider: Sync + Send {
       fn as_any(&self) -> &dyn Any;
       async fn schema_names(&self) -> Vec<String>ⓘ;
       async fn schema(&self, name: &str) -> Option<Arc<dyn SchemaProvider>>;
   
       async fn register_schema(
           &self,
           name: &str,
           schema: Arc<dyn SchemaProvider>
       ) -> Result<Option<Arc<dyn SchemaProvider>>> { ... }
   }
   ```
   
   ```rust
   [async_trait]
   pub trait SchemaProvider: Sync + Send {
       fn as_any(&self) -> &dyn Any;
       async fn table_names(&self) -> Vec<String>ⓘ;
       async fn table(&self, name: &str) -> Option<Arc<dyn TableProvider>>;
       async fn table_exist(&self, name: &str) -> bool;
   
       async fn register_table(
           &self,
           name: String,
           table: Arc<dyn TableProvider>
       ) -> Result<Option<Arc<dyn TableProvider>>> { ... }
       async fn deregister_table(
           &self,
           name: &str
       ) -> Result<Option<Arc<dyn TableProvider>>> { ... }
   }
   ```
   
   **Consequences**
   
   These traits are extensively used across the datafusion crate and would 
require
   many internal changes. Additionally, the proposal leads to changes in the 
public
   interface of datafusion. 
[SessionContext](https://docs.rs/datafusion/14.0.0/datafusion/execution/context/struct.SessionContext.html)
 would probably be affected the 
   most by the change. Methods like `catalog`, `register_catalog`, `table`,
   `register_table` will also be async.
   
   Due to the changes in the public interface of datafusion the proposal will 
lead
   to breaking changes for dependent crates such as ballista.
   
   **Describe alternatives you've considered**
   
   1. Async call beforehand
      - fetch async information beforehand and provide synchronous interface
      - potentially large overhead
      - doesn't account for catalog/schema changes
   2. use `futures::executor::block_on`
      - tricky to get it working with tokio
      - potentially exposes complexity to the user
   


-- 
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]

Reply via email to