datafusion_table_providers/clickhouse/
federation.rs1use crate::sql::db_connection_pool::dbconnection::{get_schema, Error as DbError};
2use crate::sql::db_connection_pool::{DbConnectionPool, JoinPushDown};
3use crate::sql::sql_provider_datafusion::{get_stream, to_execution_error};
4use arrow::datatypes::SchemaRef;
5use async_trait::async_trait;
6use datafusion::sql::unparser::dialect::Dialect;
7use datafusion_federation::sql::{
8 AstAnalyzer, RemoteTableRef, SQLExecutor, SQLFederationProvider, SQLTableSource,
9};
10use datafusion_federation::{FederatedTableProviderAdaptor, FederatedTableSource};
11use futures::TryStreamExt;
12use snafu::ResultExt;
13use std::sync::Arc;
14
15use datafusion::{
16 error::{DataFusionError, Result as DataFusionResult},
17 execution::SendableRecordBatchStream,
18 physical_plan::stream::RecordBatchStreamAdapter,
19 sql::TableReference,
20};
21
22use super::ClickHouseTable;
23
24impl ClickHouseTable {
25 fn create_federated_table_source(
26 self: Arc<Self>,
27 ) -> DataFusionResult<Arc<dyn FederatedTableSource>> {
28 let table_reference = self.table_reference.clone();
29 let schema = self.schema.clone();
30 let fed_provider = Arc::new(SQLFederationProvider::new(self));
31 Ok(Arc::new(SQLTableSource::new_with_schema(
32 fed_provider,
33 RemoteTableRef::from(table_reference),
34 schema,
35 )))
36 }
37
38 pub fn create_federated_table_provider(
39 self: Arc<Self>,
40 ) -> DataFusionResult<FederatedTableProviderAdaptor> {
41 let table_source = Self::create_federated_table_source(Arc::clone(&self))?;
42 Ok(FederatedTableProviderAdaptor::new_with_provider(
43 table_source,
44 self,
45 ))
46 }
47}
48
49#[async_trait]
50impl SQLExecutor for ClickHouseTable {
51 fn name(&self) -> &str {
52 "clickhouse"
53 }
54
55 fn compute_context(&self) -> Option<String> {
56 match self.pool.join_push_down() {
57 JoinPushDown::Disallow => Some(format!("{}", std::ptr::from_ref(self) as usize)),
58 JoinPushDown::AllowedFor(s) => Some(s),
59 }
60 }
61
62 fn dialect(&self) -> Arc<dyn Dialect> {
63 self.dialect.clone()
64 }
65
66 fn ast_analyzer(&self) -> Option<AstAnalyzer> {
67 None
68 }
69
70 fn execute(
71 &self,
72 query: &str,
73 schema: SchemaRef,
74 ) -> DataFusionResult<SendableRecordBatchStream> {
75 let fut = get_stream(self.pool.clone(), query.to_string(), Arc::clone(&schema));
76 let stream = futures::stream::once(fut).try_flatten();
77 Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
78 }
79
80 async fn table_names(&self) -> DataFusionResult<Vec<String>> {
81 Err(DataFusionError::NotImplemented(
82 "table inference not implemented".to_string(),
83 ))
84 }
85
86 async fn get_table_schema(&self, table_name: &str) -> DataFusionResult<SchemaRef> {
87 get_schema(
88 self.pool.connect().await?,
89 &TableReference::from(table_name),
90 )
91 .await
92 .boxed()
93 .map_err(|e| DbError::UnableToGetSchema { source: e })
94 .map_err(to_execution_error)
95 }
96}