clickhouse_datafusion/
builders.rs

1//! The various builders to make registering, updating, and synchronizing remote `ClickHouse`
2//! databases and `DataFusion` easier.
3//!
4//! # `ClickHouseBuilder`
5//!
6//! You begin by creating a `ClickHouseBuilder` instance, which allows you to configure various
7//! aspects of the connection, such as the endpoint, authentication, the underlying `ClickHouse`
8//! connection pool, `clickhouse-arrow` `Client` options, schema mapping, and more.
9//!
10//! NOTE: The schema (database) set as the default on the underlying `ClickHouse` client will be set
11//! as the initial schema after calling `ClickHouseBuilder::build_catalog`. This can be changed on
12//! the fly by calling `with_schema` on the catalog builder returned.
13//!
14//! ## Schema Coercion
15//!
16//! You can also specify whether "schema coercion" should occur during streaming of the data
17//! returned by query execution. If using the full [`super::context::ClickHouseSessionContext`] and
18//! [`super::context::ClickHouseQueryPlanner`], required if you plan on using `ClickHouse`
19//! functions, then the return type provided to the second argument of the `clickhouse(...)` scalar
20//! UDFs will specify the expected return type.
21//!
22//! This means you must be exact, otherwise an arrow error will be returned. By configuring schema
23//! coercion, anything that can be coerced via arrow will be coerced, based on the expected schema
24//! of the `clickhouse` function mentioned above.
25//!
26//! Generally schema conversion should be avoided, as there is a non-zero cost. The cost is incurred
27//! per `RecordBatch` as the results are streamed through `DataFusion`'s execution context, which
28//! means it will have a per `RecordBatch` cost that can be avoided by simply providing the correct
29//! return type to the clickhouse function. But, for non latency-sensitive use cases, this may
30//! provide convenience, especially when dealing with `ClickHouse` higher order functions.
31//!
32//! # `ClickHouseCatalogBuilder`
33//!
34//! After configuration of the `ClickHouseBuilder`, a `ClickHouseCatalogBuilder` will be obtained by
35//! calling `ClickHouseBuilder::build_catalog`. The `ClickHouseCatalogBuilder` returned can then be
36//! used to set the schema being targets, create tables on the remote schema, register existing
37//! tables under aliases, and refresh `DataFusion`'s internal catalog to keep the two in sync.
38//!
39//! Refer to e2e tests in the repository for detailed examples on its usage, especially the tests's
40//! associated "helper" functions. The `basic` example shows a basic usage of the
41//! `ClickHouseBuilder` that suffices for most use cases.
42use std::collections::HashMap;
43use std::sync::Arc;
44
45use clickhouse_arrow::prelude::ClickHouseEngine;
46use clickhouse_arrow::{
47    ArrowConnectionPoolBuilder, ArrowOptions, ArrowPoolBuilder, ClientBuilder, CreateOptions,
48    Destination,
49};
50use datafusion::arrow::datatypes::SchemaRef;
51use datafusion::catalog::{CatalogProvider, TableProviderFactory};
52use datafusion::common::{Constraints, DFSchema};
53use datafusion::error::{DataFusionError, Result};
54use datafusion::logical_expr::CreateExternalTable;
55use datafusion::prelude::*;
56use datafusion::sql::TableReference;
57use tracing::{debug, error};
58
59use crate::connection::ClickHouseConnectionPool;
60use crate::providers::catalog::ClickHouseCatalogProvider;
61use crate::providers::table_factory::ClickHouseTableFactory;
62use crate::table_factory::ClickHouseTableProviderFactory;
63use crate::utils::{self, ENDPOINT_PARAM, default_str_to_expr, register_builtins};
64
65/// The default `DataFusion` catalog name if no other name is provided.
66pub const DEFAULT_CLICKHOUSE_CATALOG: &str = "clickhouse";
67
68/// Simple function to provide default [`ArrowOptions`] fit for common use
69pub fn default_arrow_options() -> ArrowOptions {
70    ArrowOptions::default()
71        .with_strings_as_strings(true)
72        .with_strict_schema(false)
73        .with_disable_strict_schema_ddl(true)
74        .with_nullable_array_default_empty(true)
75}
76
77/// Entrypoint builder for `ClickHouse` and `DataFusion` integration.
78///
79/// Ensures `DataFusion` default UDFs as well as `ClickHouse` udfs are registered in
80/// [`SessionContext`], the pool is attached to the [`ClickHouseTableProviderFactory`], and
81/// `ClickHouse` is reachable by the provided endpoint.
82///
83/// NOTE: While `clickhouse_arrow` defaults to binary encoding for strings (via
84/// [`clickhouse_arrow::ArrowOptions::strings_as_strings`] == false), for `DataFusion` the
85/// default is `true`. This can be disabled by modifying the setting via
86/// [`ClickHouseBuilder::configure_arrow_options`]
87pub struct ClickHouseBuilder {
88    endpoint:     Destination,
89    pool_builder: ArrowConnectionPoolBuilder,
90    factory:      ClickHouseTableProviderFactory,
91}
92
93impl ClickHouseBuilder {
94    /// Create new `ClickHouseBuilder` that can be used to configure both the
95    /// [`clickhouse_arrow::ClientBuilder`], the [`clickhouse_arrow::ArrowPoolBuilder`], and be
96    /// "built" into a [`ClickHouseCatalogBuilder`] for interacting with `ClickHouse` databases and
97    /// tables.
98    pub fn new(endpoint: impl Into<Destination>) -> Self {
99        let endpoint = endpoint.into();
100        let pool_builder = ArrowConnectionPoolBuilder::new(endpoint.clone())
101            .configure_client(|c| c.with_arrow_options(default_arrow_options()));
102        Self { endpoint, pool_builder, factory: ClickHouseTableProviderFactory::new() }
103    }
104
105    pub fn new_with_pool_builder(
106        endpoint: impl Into<Destination>,
107        pool_builder: ArrowConnectionPoolBuilder,
108    ) -> Self {
109        let endpoint = endpoint.into();
110        Self { endpoint, pool_builder, factory: ClickHouseTableProviderFactory::new() }
111    }
112
113    /// Configure whether to coerce schemas to match expected schemas during query execution.
114    /// Remember, there is a non-zero cost incurred per `RecordBatch`, and this is mainly useful for
115    /// allowing looser return types when using clickhouse functions.
116    #[must_use]
117    pub fn with_coercion(mut self, coerce: bool) -> Self {
118        self.factory = self.factory.with_coercion(coerce);
119        self
120    }
121
122    /// Configure the underlying [`clickhouse_arrow::ClientBuilder`].
123    #[must_use]
124    pub fn configure_client(mut self, f: impl FnOnce(ClientBuilder) -> ClientBuilder) -> Self {
125        self.pool_builder = self.pool_builder.configure_client(f);
126        self
127    }
128
129    /// Configure the underlying [`clickhouse_arrow::ArrowPoolBuilder`].
130    #[must_use]
131    pub fn configure_pool(mut self, f: impl FnOnce(ArrowPoolBuilder) -> ArrowPoolBuilder) -> Self {
132        self.pool_builder = self.pool_builder.configure_pool(f);
133        self
134    }
135
136    /// `clickhouse_arrow` defaults to binary encoding for
137    /// [`datafusion::arrow::datatypes::DataType::Utf8`] columns, the default is inverted for
138    /// `DataFusion`. This method allows disabling that change to use binary encoding for strings,
139    /// among other configuration options.
140    ///
141    /// See: [`clickhouse_arrow::ArrowOptions`]
142    #[must_use]
143    pub fn configure_arrow_options(mut self, f: impl FnOnce(ArrowOptions) -> ArrowOptions) -> Self {
144        self.pool_builder = self.pool_builder.configure_client(|c| {
145            let options = c.options().ext.arrow.unwrap_or(default_arrow_options());
146            c.with_arrow_options(f(options))
147        });
148        self
149    }
150
151    /// Build ensures `ClickHouse` builtins are registered (such as nested functions), the pool is
152    /// attached to the factory, the `ClickHouse` endpoint is reachable, and the catalog is created
153    /// and registered to the [`SessionContext`].
154    ///
155    /// # Errors
156    /// - Returns an error if the `ClickHouse` endpoint is unreachable
157    /// - Returns an error if the `ClickHouse` catalog fails to be created
158    pub async fn build_catalog(
159        self,
160        ctx: &SessionContext,
161        catalog: Option<&str>,
162    ) -> Result<ClickHouseCatalogBuilder> {
163        // Register built in functions and clickhouse udfs
164        register_builtins(ctx);
165
166        let catalog = catalog.unwrap_or(DEFAULT_CLICKHOUSE_CATALOG).to_string();
167        let database = self.pool_builder.client_options().default_database.clone();
168        debug!(catalog, database, "Attaching pool to ClickHouse table factory");
169
170        let endpoint = self.endpoint.to_string();
171        let factory = self.factory;
172        let pool = Arc::new(factory.attach_pool_builder(self.endpoint, self.pool_builder).await?);
173
174        ClickHouseCatalogBuilder::try_new(ctx, catalog, database, endpoint, pool, factory).await
175    }
176}
177
178/// [`ClickHouseCatalogBuilder`] can be used to create tables, register existing tables, and finally
179/// refresh the `ClickHouse` catalog in `DataFusion`.
180///
181/// IMPORTANT! After creating tables, one of the build variations, ie `Self::build` or
182/// `Self::build_schema`, must be called to ensure the catalog provider is up to date with the
183/// remote `ClickHouse` database. If you forget to do this, `DataFusion` queries targeting one of
184/// these tables will fail.
185#[derive(Clone)]
186pub struct ClickHouseCatalogBuilder {
187    catalog:  String,
188    /// The current schema the builder is targeting.
189    schema:   String,
190    /// The configured remote endpoint the underlying `ClickHouse` pool is connected.
191    endpoint: String,
192    /// The `ClickHouse` connection used to communicate with the remote `ClickHouse` database.
193    pool:     Arc<ClickHouseConnectionPool>,
194    /// This factory is used to create new tables in the remote `ClickHouse` database. This builder
195    /// must be built with one of the builder variations, ie `Self::build` or `Self::build_schema`,
196    /// so that the provider reflects the most current remote schema.
197    factory:  ClickHouseTableProviderFactory,
198    /// The catalog provider is a passive catalog provider, meaning it must be "refreshed" after
199    /// table creation or schema change of any type to the remove `ClickHouse` database.
200    provider: Arc<ClickHouseCatalogProvider>,
201}
202
203impl ClickHouseCatalogBuilder {
204    // TODO: Docs
205    /// Internal constructor
206    async fn try_new(
207        ctx: &SessionContext,
208        catalog: impl Into<String>,
209        default_schema: impl Into<String>,
210        endpoint: impl Into<String>,
211        pool: Arc<ClickHouseConnectionPool>,
212        factory: ClickHouseTableProviderFactory,
213    ) -> Result<Self> {
214        let schema = default_schema.into();
215        let catalog = catalog.into();
216        let endpoint = endpoint.into();
217
218        // Must have a schema
219        let schema = if schema.is_empty() { "default".to_string() } else { schema };
220
221        // Ensure the database exists if not default
222        if schema != "default" {
223            debug!(schema, "Database not default, attempting create");
224            utils::create_database(&schema, &pool).await?;
225        }
226
227        // Register catalog or create a new one
228        let provider = if factory.coerce_schemas() {
229            ClickHouseCatalogProvider::try_new_with_coercion(Arc::clone(&pool)).await
230        } else {
231            ClickHouseCatalogProvider::try_new(Arc::clone(&pool)).await
232        }
233        .inspect_err(|error| error!(?error, "Failed to register catalog {catalog}"))?;
234
235        let provider = Arc::new(provider);
236
237        drop(
238            ctx.register_catalog(
239                catalog.as_str(),
240                Arc::clone(&provider) as Arc<dyn CatalogProvider>,
241            ),
242        );
243
244        Ok(ClickHouseCatalogBuilder { catalog, schema, endpoint, pool, factory, provider })
245    }
246
247    /// Return the name of the catalog in `DataFusion`'s context that this builder is configuring.
248    pub fn name(&self) -> &str { &self.catalog }
249
250    /// Return the currently set schema (database) being targeted. Can be changed on the fly by
251    /// calling `Self::with_schema`.
252    pub fn schema(&self) -> &str { &self.schema }
253
254    /// Update the current "schema" (database) that this builder is targeting, and continue
255    /// building.
256    ///
257    /// # Errors
258    /// - Returns an error if the schema needs to be created and fails
259    pub async fn with_schema(mut self, name: impl Into<String>) -> Result<Self> {
260        let name = name.into();
261        // Don't re-create if schema is already set
262        if name == self.schema {
263            return Ok(self);
264        }
265        self.schema = name;
266
267        // Ensure the database exists if not default
268        if self.schema != "default" {
269            debug!(schema = self.schema, "Database not default, attempting create");
270            utils::create_database(&self.schema, &self.pool).await?;
271        }
272
273        // Be sure to refresh the catalog before setting the schema
274        self.provider.refresh_catalog(&self.pool).await?;
275
276        Ok(self)
277    }
278
279    /// Create a new table in the remote `ClickHouse` instance.
280    ///
281    /// # Arguments
282    /// - `name`: The name of the table to create.
283    /// - `engine`: The engine to use for the table.
284    /// - `schema`: The schema of the table.
285    ///
286    /// # Returns
287    /// - A [`ClickHouseTableCreator`] that can be used to create tables in the remote `ClickHouse`
288    ///   instance.
289    pub fn with_new_table(
290        &self,
291        name: impl Into<String>,
292        engine: impl Into<ClickHouseEngine>,
293        schema: SchemaRef,
294    ) -> ClickHouseTableCreator {
295        let table = name.into();
296        let options = CreateOptions::new(engine.into().to_string());
297        debug!(schema = self.schema, table, ?options, "Initializing table creator");
298        ClickHouseTableCreator { name: table, builder: self.clone(), options, schema }
299    }
300
301    /// Create a new table in the remote `ClickHouse` instance.
302    ///
303    /// # Arguments
304    /// - `name`: The name of the table to create.
305    /// - `schema`: The schema of the table.
306    /// - `options`: More detailed `CreateOptions` for creating the provided table.
307    ///
308    /// # Returns
309    /// - A [`ClickHouseTableCreator`] that can be used to create tables in the remote `ClickHouse`
310    ///   instance.
311    pub fn with_new_table_and_options(
312        &self,
313        name: impl Into<String>,
314        schema: SchemaRef,
315        options: CreateOptions,
316    ) -> ClickHouseTableCreator {
317        let table = name.into();
318        debug!(schema = self.schema, table, ?options, "Initializing table creator");
319        ClickHouseTableCreator { name: table, builder: self.clone(), options, schema }
320    }
321
322    /// Register an existing `ClickHouse` table, optionally renaming it in the provided session
323    /// state.
324    ///
325    /// # Errors
326    /// - Returns an error if the table does not exist in the remote database
327    /// - Returns an error if the table cannot be registered to the context
328    pub async fn register_existing_table(
329        &self,
330        name: impl Into<TableReference>,
331        name_as: Option<impl Into<TableReference>>,
332        ctx: &SessionContext,
333    ) -> Result<()> {
334        let name = name.into();
335        let database = name.schema().unwrap_or(&self.schema);
336        let exists =
337            self.pool.connect().await?.tables(database).await?.contains(&name.table().to_string());
338
339        if !exists {
340            return Err(DataFusionError::Plan(format!(
341                "Table '{name}' does not exist in ClickHouse database '{database}', use \
342                 `table_creator` instead"
343            )));
344        }
345
346        let table = TableReference::full(self.catalog.as_str(), database, name.table());
347        let table_as = name_as.map(Into::into).unwrap_or(table.clone());
348
349        let factory = ClickHouseTableFactory::new(Arc::clone(&self.pool));
350        let provider = factory.table_provider(table).await?;
351        debug!(?table_as, "Registering ClickHouse table provider");
352        drop(ctx.register_table(table_as, provider)?);
353
354        Ok(())
355    }
356
357    /// Build the current `schema` (database) being managed by this catalog, optionally registering
358    /// a new schema to continue building
359    ///
360    /// Note: For the `SessionContext` to recognize the added tables and updated schema, either this
361    /// function or `Self::build` must be called.
362    ///
363    /// # Errors
364    /// - Returnes an error if an error occurs while refreshing the catalog
365    pub async fn build_schema(
366        mut self,
367        new_schema: Option<String>,
368        ctx: &SessionContext,
369    ) -> Result<Self> {
370        let _catalog = self.build_internal(ctx).await?;
371        self.schema = new_schema.unwrap_or(self.schema);
372        Ok(self)
373    }
374
375    /// Re-register the catalog, updating the [`SessionContext`], and return the updated context.
376    ///
377    /// Note: Important! For the [`SessionContext`] to recognize the added tables and updated
378    /// schema, either this function or `Self::build` must be called. For that reason, it is
379    /// important to  use the [`SessionContext`] provided back from this function.
380    ///
381    /// # Errors
382    /// - Returns an error if the `SessionContext` has not been federated
383    /// - Returnes an error if an error occurs while refreshing the catalog
384    /// - Returns an error if the "federation" feature is enabled but the context is not federated
385    pub async fn build(&self, ctx: &SessionContext) -> Result<Arc<ClickHouseCatalogProvider>> {
386        #[cfg(feature = "federation")]
387        {
388            use datafusion::common::exec_err;
389
390            use crate::federation::FederatedContext as _;
391
392            if !ctx.is_federated() {
393                return exec_err!(
394                    "Building this schema with federation enabled but no federated SessionContext \
395                     will fail. Call `ctx.federate()` before providing a context to build with."
396                );
397            }
398        }
399
400        self.build_internal(ctx).await
401    }
402
403    /// Re-registers the catalog provider, re-configures the [`SessionContext`], and return a
404    /// clone of the catalog.
405    ///
406    /// # Errors
407    /// - Returnes an error if an error occurs while refreshing the catalog
408    async fn build_internal(&self, ctx: &SessionContext) -> Result<Arc<ClickHouseCatalogProvider>> {
409        let catalog = Arc::clone(&self.provider);
410        debug!(catalog = self.catalog, "ClickHouse catalog created");
411
412        // Re-register UDFs
413        register_builtins(ctx);
414
415        // Re-register catalog
416        catalog.refresh_catalog(&self.pool).await?;
417        drop(ctx.register_catalog(&self.catalog, Arc::clone(&catalog) as Arc<dyn CatalogProvider>));
418
419        Ok(catalog)
420    }
421}
422
423/// Builder phase for creating `ClickHouse` tables.
424#[derive(Clone)]
425pub struct ClickHouseTableCreator {
426    name:    String,
427    builder: ClickHouseCatalogBuilder,
428    schema:  SchemaRef,
429    options: CreateOptions,
430}
431
432impl ClickHouseTableCreator {
433    /// Update the underlying table create options that will be passed to clickhouse.
434    ///
435    /// See [`CreateOptions`] for more information.
436    #[must_use]
437    pub fn update_create_options(
438        mut self,
439        update: impl Fn(CreateOptions) -> CreateOptions,
440    ) -> Self {
441        self.options = update(self.options);
442        self
443    }
444
445    /// Create the table, returning back a [`ClickHouseCatalogBuilder`] to create more tables.
446    ///
447    /// # Errors
448    /// - Returns an error if the `TableProviderFactory` fails to create the table
449    /// - Returnes an error if an error occurs while refreshing the catalog
450    pub async fn create(self, ctx: &SessionContext) -> Result<ClickHouseCatalogBuilder> {
451        let schema = self.builder.schema.clone();
452        let table = self.name;
453
454        let column_defaults = self
455            .options
456            .clone()
457            .defaults
458            .unwrap_or_default()
459            .into_iter()
460            .map(|(col, value)| (col, default_str_to_expr(&value)))
461            .collect::<HashMap<_, _>>();
462
463        let mut options = utils::create_options_to_params(self.options).into_params();
464        drop(options.insert(ENDPOINT_PARAM.into(), self.builder.endpoint.clone()));
465
466        let table_ref = TableReference::partial(schema.as_str(), table.as_str());
467        let cmd = CreateExternalTable {
468            name: table_ref.clone(),
469            schema: Arc::new(DFSchema::try_from(Arc::clone(&self.schema))?),
470            options,
471            column_defaults,
472            constraints: Constraints::default(),
473            table_partition_cols: vec![],
474            if_not_exists: false,
475            location: String::new(),
476            file_type: String::new(),
477            temporary: false,
478            definition: None,
479            order_exprs: vec![],
480            unbounded: false,
481        };
482        let _provider = self
483            .builder
484            .factory
485            .create(&ctx.state(), &cmd)
486            .await
487            .inspect_err(|error| error!(?error, table, "Factory error creating table"))?;
488        debug!(table, "Table created, catalog will be refreshed in `build`");
489
490        // Refresh the catalog after creating the table
491        drop(self.builder.build_internal(ctx).await?);
492
493        Ok(self.builder)
494    }
495}