|
| 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 async_trait::async_trait; |
| 19 | +use datafusion::catalog::catalog::{CatalogList, CatalogProvider}; |
| 20 | +use datafusion::catalog::schema::SchemaProvider; |
| 21 | +use datafusion::datasource::listing::{ |
| 22 | + ListingTable, ListingTableConfig, ListingTableUrl, |
| 23 | +}; |
| 24 | +use datafusion::datasource::TableProvider; |
| 25 | +use datafusion::error::Result; |
| 26 | +use datafusion::execution::context::SessionState; |
| 27 | +use parking_lot::RwLock; |
| 28 | +use std::any::Any; |
| 29 | +use std::sync::{Arc, Weak}; |
| 30 | + |
| 31 | +/// Wraps another catalog, automatically creating table providers |
| 32 | +/// for local files if needed |
| 33 | +pub struct DynamicFileCatalog { |
| 34 | + inner: Arc<dyn CatalogList>, |
| 35 | + state: Weak<RwLock<SessionState>>, |
| 36 | +} |
| 37 | + |
| 38 | +impl DynamicFileCatalog { |
| 39 | + pub fn new(inner: Arc<dyn CatalogList>, state: Weak<RwLock<SessionState>>) -> Self { |
| 40 | + Self { inner, state } |
| 41 | + } |
| 42 | +} |
| 43 | + |
| 44 | +impl CatalogList for DynamicFileCatalog { |
| 45 | + fn as_any(&self) -> &dyn Any { |
| 46 | + self |
| 47 | + } |
| 48 | + |
| 49 | + fn register_catalog( |
| 50 | + &self, |
| 51 | + name: String, |
| 52 | + catalog: Arc<dyn CatalogProvider>, |
| 53 | + ) -> Option<Arc<dyn CatalogProvider>> { |
| 54 | + self.inner.register_catalog(name, catalog) |
| 55 | + } |
| 56 | + |
| 57 | + fn catalog_names(&self) -> Vec<String> { |
| 58 | + self.inner.catalog_names() |
| 59 | + } |
| 60 | + |
| 61 | + fn catalog(&self, name: &str) -> Option<Arc<dyn CatalogProvider>> { |
| 62 | + let state = self.state.clone(); |
| 63 | + self.inner |
| 64 | + .catalog(name) |
| 65 | + .map(|catalog| Arc::new(DynamicFileCatalogProvider::new(catalog, state)) as _) |
| 66 | + } |
| 67 | +} |
| 68 | + |
| 69 | +/// Wraps another catalog provider |
| 70 | +struct DynamicFileCatalogProvider { |
| 71 | + inner: Arc<dyn CatalogProvider>, |
| 72 | + state: Weak<RwLock<SessionState>>, |
| 73 | +} |
| 74 | + |
| 75 | +impl DynamicFileCatalogProvider { |
| 76 | + pub fn new( |
| 77 | + inner: Arc<dyn CatalogProvider>, |
| 78 | + state: Weak<RwLock<SessionState>>, |
| 79 | + ) -> Self { |
| 80 | + Self { inner, state } |
| 81 | + } |
| 82 | +} |
| 83 | + |
| 84 | +impl CatalogProvider for DynamicFileCatalogProvider { |
| 85 | + fn as_any(&self) -> &dyn Any { |
| 86 | + self |
| 87 | + } |
| 88 | + |
| 89 | + fn schema_names(&self) -> Vec<String> { |
| 90 | + self.inner.schema_names() |
| 91 | + } |
| 92 | + |
| 93 | + fn schema(&self, name: &str) -> Option<Arc<dyn SchemaProvider>> { |
| 94 | + let state = self.state.clone(); |
| 95 | + self.inner |
| 96 | + .schema(name) |
| 97 | + .map(|schema| Arc::new(DynamicFileSchemaProvider::new(schema, state)) as _) |
| 98 | + } |
| 99 | + |
| 100 | + fn register_schema( |
| 101 | + &self, |
| 102 | + name: &str, |
| 103 | + schema: Arc<dyn SchemaProvider>, |
| 104 | + ) -> Result<Option<Arc<dyn SchemaProvider>>> { |
| 105 | + self.inner.register_schema(name, schema) |
| 106 | + } |
| 107 | +} |
| 108 | + |
| 109 | +/// Wraps another schema provider |
| 110 | +struct DynamicFileSchemaProvider { |
| 111 | + inner: Arc<dyn SchemaProvider>, |
| 112 | + state: Weak<RwLock<SessionState>>, |
| 113 | +} |
| 114 | + |
| 115 | +impl DynamicFileSchemaProvider { |
| 116 | + pub fn new( |
| 117 | + inner: Arc<dyn SchemaProvider>, |
| 118 | + state: Weak<RwLock<SessionState>>, |
| 119 | + ) -> Self { |
| 120 | + Self { inner, state } |
| 121 | + } |
| 122 | +} |
| 123 | + |
| 124 | +#[async_trait] |
| 125 | +impl SchemaProvider for DynamicFileSchemaProvider { |
| 126 | + fn as_any(&self) -> &dyn Any { |
| 127 | + self |
| 128 | + } |
| 129 | + |
| 130 | + fn table_names(&self) -> Vec<String> { |
| 131 | + self.inner.table_names() |
| 132 | + } |
| 133 | + |
| 134 | + fn register_table( |
| 135 | + &self, |
| 136 | + name: String, |
| 137 | + table: Arc<dyn TableProvider>, |
| 138 | + ) -> Result<Option<Arc<dyn TableProvider>>> { |
| 139 | + self.inner.register_table(name, table) |
| 140 | + } |
| 141 | + |
| 142 | + async fn table(&self, name: &str) -> Option<Arc<dyn TableProvider>> { |
| 143 | + let inner_table = self.inner.table(name).await; |
| 144 | + if inner_table.is_some() { |
| 145 | + return inner_table; |
| 146 | + } |
| 147 | + |
| 148 | + // if the inner schema provider didn't have a table by |
| 149 | + // that name, try to treat it as a listing table |
| 150 | + let state = self.state.upgrade()?.read().clone(); |
| 151 | + let config = ListingTableConfig::new(ListingTableUrl::parse(name).ok()?) |
| 152 | + .infer(&state) |
| 153 | + .await |
| 154 | + .ok()?; |
| 155 | + Some(Arc::new(ListingTable::try_new(config).ok()?)) |
| 156 | + } |
| 157 | + |
| 158 | + fn deregister_table(&self, name: &str) -> Result<Option<Arc<dyn TableProvider>>> { |
| 159 | + self.inner.deregister_table(name) |
| 160 | + } |
| 161 | + |
| 162 | + fn table_exist(&self, name: &str) -> bool { |
| 163 | + self.inner.table_exist(name) |
| 164 | + } |
| 165 | +} |
0 commit comments