|
| 1 | +use std::sync::{Arc, LazyLock}; |
| 2 | + |
| 3 | +use arrow::array::{RecordBatch, StringArray}; |
| 4 | +use arrow_schema::{DataType, Field, Schema, SchemaRef}; |
| 5 | +use datafusion::{ |
| 6 | + catalog::{MemTable, TableFunctionArgs, TableFunctionImpl, TableProvider}, |
| 7 | + common::Result, |
| 8 | + execution::SessionState, |
| 9 | + prelude::SessionContext, |
| 10 | +}; |
| 11 | +use datafusion_common::{DataFusionError, plan_err}; |
| 12 | +use futures::executor::block_on; |
| 13 | + |
| 14 | +const FUNCTION_NAME: &str = "table_list"; |
| 15 | + |
| 16 | +// The example shows, how to create UDTF that depends on the session state. |
| 17 | +// There is `table_list` UDTF is defined which returns list of tables within session. |
| 18 | + |
| 19 | +pub async fn table_list_udtf() -> Result<()> { |
| 20 | + let ctx = SessionContext::new(); |
| 21 | + ctx.register_udtf(FUNCTION_NAME, Arc::new(TableListUdtf)); |
| 22 | + |
| 23 | + // Register different kinds of tables. |
| 24 | + ctx.sql("create view v as select 1") |
| 25 | + .await? |
| 26 | + .collect() |
| 27 | + .await?; |
| 28 | + ctx.sql("create table t(a int)").await?.collect().await?; |
| 29 | + |
| 30 | + // Print results. |
| 31 | + ctx.sql("select * from table_list()").await?.show().await?; |
| 32 | + |
| 33 | + Ok(()) |
| 34 | +} |
| 35 | + |
| 36 | +#[derive(Debug, Default)] |
| 37 | +struct TableListUdtf; |
| 38 | + |
| 39 | +static SCHEMA: LazyLock<SchemaRef> = LazyLock::new(|| { |
| 40 | + SchemaRef::new(Schema::new(vec![ |
| 41 | + Field::new("catalog", DataType::Utf8, false), |
| 42 | + Field::new("schema", DataType::Utf8, false), |
| 43 | + Field::new("table", DataType::Utf8, false), |
| 44 | + Field::new("type", DataType::Utf8, false), |
| 45 | + ])) |
| 46 | +}); |
| 47 | + |
| 48 | +impl TableFunctionImpl for TableListUdtf { |
| 49 | + fn call_with_args(&self, args: TableFunctionArgs) -> Result<Arc<dyn TableProvider>> { |
| 50 | + if !args.args.is_empty() { |
| 51 | + return plan_err!( |
| 52 | + "{}: unexpected number of arguments: {}, expected: 0", |
| 53 | + FUNCTION_NAME, |
| 54 | + args.args.len() |
| 55 | + ); |
| 56 | + } |
| 57 | + let state = args |
| 58 | + .session |
| 59 | + .as_any() |
| 60 | + .downcast_ref::<SessionState>() |
| 61 | + .ok_or_else(|| { |
| 62 | + DataFusionError::Internal("failed to downcast state".into()) |
| 63 | + })?; |
| 64 | + |
| 65 | + let mut catalogs = vec![]; |
| 66 | + let mut schemas = vec![]; |
| 67 | + let mut tables = vec![]; |
| 68 | + let mut types = vec![]; |
| 69 | + |
| 70 | + let catalog_list = state.catalog_list(); |
| 71 | + for catalog_name in catalog_list.catalog_names() { |
| 72 | + let Some(catalog) = catalog_list.catalog(&catalog_name) else { |
| 73 | + continue; |
| 74 | + }; |
| 75 | + for schema_name in catalog.schema_names() { |
| 76 | + let Some(schema) = catalog.schema(&schema_name) else { |
| 77 | + continue; |
| 78 | + }; |
| 79 | + for table_name in schema.table_names() { |
| 80 | + let Some(provider) = block_on(schema.table(&table_name))? else { |
| 81 | + continue; |
| 82 | + }; |
| 83 | + catalogs.push(catalog_name.clone()); |
| 84 | + schemas.push(schema_name.clone()); |
| 85 | + tables.push(table_name.clone()); |
| 86 | + types.push(provider.table_type().to_string()) |
| 87 | + } |
| 88 | + } |
| 89 | + } |
| 90 | + |
| 91 | + let batch = RecordBatch::try_new( |
| 92 | + Arc::clone(&SCHEMA), |
| 93 | + vec![ |
| 94 | + Arc::new(StringArray::from(catalogs)), |
| 95 | + Arc::new(StringArray::from(schemas)), |
| 96 | + Arc::new(StringArray::from(tables)), |
| 97 | + Arc::new(StringArray::from(types)), |
| 98 | + ], |
| 99 | + )?; |
| 100 | + |
| 101 | + Ok(Arc::new(MemTable::try_new( |
| 102 | + batch.schema(), |
| 103 | + vec![vec![batch]], |
| 104 | + )?)) |
| 105 | + } |
| 106 | +} |
0 commit comments