-
Notifications
You must be signed in to change notification settings - Fork 1.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Support User Defined Table Function #8306
Merged
Merged
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
32ecce4
Support User Defined Table Function
Veeupup 2f4e089
fix comments
Veeupup 7a85a43
add udtf test
Veeupup 1fe0c8b
add file header
Veeupup 169b13e
Simply table function example, add some comments
alamb 2df01db
Simplfy exprs
alamb 180e02e
Merge pull request #1 from alamb/alamb/simplified_table_function
Veeupup c2e7cb0
make clippy happy
Veeupup 0e98c0b
Update datafusion/core/tests/user_defined/user_defined_table_function…
alamb ccc2959
Merge remote-tracking branch 'apache/main' into table_function
alamb File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,177 @@ | ||
// Licensed to the Apache Software Foundation (ASF) under one | ||
// or more contributor license agreements. See the NOTICE file | ||
// distributed with this work for additional information | ||
// regarding copyright ownership. The ASF licenses this file | ||
// to you under the Apache License, Version 2.0 (the | ||
// "License"); you may not use this file except in compliance | ||
// with the License. You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, | ||
// software distributed under the License is distributed on an | ||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
// KIND, either express or implied. See the License for the | ||
// specific language governing permissions and limitations | ||
// under the License. | ||
|
||
use arrow::csv::reader::Format; | ||
use arrow::csv::ReaderBuilder; | ||
use async_trait::async_trait; | ||
use datafusion::arrow::datatypes::SchemaRef; | ||
use datafusion::arrow::record_batch::RecordBatch; | ||
use datafusion::datasource::function::TableFunctionImpl; | ||
use datafusion::datasource::TableProvider; | ||
use datafusion::error::Result; | ||
use datafusion::execution::context::{ExecutionProps, SessionState}; | ||
use datafusion::physical_plan::memory::MemoryExec; | ||
use datafusion::physical_plan::ExecutionPlan; | ||
use datafusion::prelude::SessionContext; | ||
use datafusion_common::{plan_err, DataFusionError, ScalarValue}; | ||
use datafusion_expr::{Expr, TableType}; | ||
use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyContext}; | ||
use std::fs::File; | ||
use std::io::Seek; | ||
use std::path::Path; | ||
use std::sync::Arc; | ||
|
||
// To define your own table function, you only need to do the following 3 things: | ||
// 1. Implement your own [`TableProvider`] | ||
// 2. Implement your own [`TableFunctionImpl`] and return your [`TableProvider`] | ||
// 3. Register the function using [`SessionContext::register_udtf`] | ||
|
||
/// This example demonstrates how to register a TableFunction | ||
#[tokio::main] | ||
async fn main() -> Result<()> { | ||
// create local execution context | ||
let ctx = SessionContext::new(); | ||
|
||
// register the table function that will be called in SQL statements by `read_csv` | ||
ctx.register_udtf("read_csv", Arc::new(LocalCsvTableFunc {})); | ||
|
||
let testdata = datafusion::test_util::arrow_test_data(); | ||
let csv_file = format!("{testdata}/csv/aggregate_test_100.csv"); | ||
|
||
// Pass 2 arguments, read csv with at most 2 rows (simplify logic makes 1+1 --> 2) | ||
let df = ctx | ||
.sql(format!("SELECT * FROM read_csv('{csv_file}', 1 + 1);").as_str()) | ||
.await?; | ||
df.show().await?; | ||
|
||
// just run, return all rows | ||
let df = ctx | ||
.sql(format!("SELECT * FROM read_csv('{csv_file}');").as_str()) | ||
.await?; | ||
df.show().await?; | ||
|
||
Ok(()) | ||
} | ||
|
||
/// Table Function that mimics the [`read_csv`] function in DuckDB. | ||
/// | ||
/// Usage: `read_csv(filename, [limit])` | ||
/// | ||
/// [`read_csv`]: https://duckdb.org/docs/data/csv/overview.html | ||
struct LocalCsvTable { | ||
schema: SchemaRef, | ||
limit: Option<usize>, | ||
batches: Vec<RecordBatch>, | ||
} | ||
|
||
#[async_trait] | ||
impl TableProvider for LocalCsvTable { | ||
fn as_any(&self) -> &dyn std::any::Any { | ||
self | ||
} | ||
|
||
fn schema(&self) -> SchemaRef { | ||
self.schema.clone() | ||
} | ||
|
||
fn table_type(&self) -> TableType { | ||
TableType::Base | ||
} | ||
|
||
async fn scan( | ||
&self, | ||
_state: &SessionState, | ||
projection: Option<&Vec<usize>>, | ||
_filters: &[Expr], | ||
_limit: Option<usize>, | ||
) -> Result<Arc<dyn ExecutionPlan>> { | ||
let batches = if let Some(max_return_lines) = self.limit { | ||
// get max return rows from self.batches | ||
let mut batches = vec![]; | ||
let mut lines = 0; | ||
for batch in &self.batches { | ||
let batch_lines = batch.num_rows(); | ||
if lines + batch_lines > max_return_lines { | ||
let batch_lines = max_return_lines - lines; | ||
batches.push(batch.slice(0, batch_lines)); | ||
break; | ||
} else { | ||
batches.push(batch.clone()); | ||
lines += batch_lines; | ||
} | ||
} | ||
batches | ||
} else { | ||
self.batches.clone() | ||
}; | ||
Ok(Arc::new(MemoryExec::try_new( | ||
&[batches], | ||
TableProvider::schema(self), | ||
projection.cloned(), | ||
)?)) | ||
} | ||
} | ||
struct LocalCsvTableFunc {} | ||
|
||
impl TableFunctionImpl for LocalCsvTableFunc { | ||
fn call(&self, exprs: &[Expr]) -> Result<Arc<dyn TableProvider>> { | ||
let Some(Expr::Literal(ScalarValue::Utf8(Some(ref path)))) = exprs.get(0) else { | ||
return plan_err!("read_csv requires at least one string argument"); | ||
}; | ||
|
||
let limit = exprs | ||
.get(1) | ||
.map(|expr| { | ||
// try to simpify the expression, so 1+2 becomes 3, for example | ||
let execution_props = ExecutionProps::new(); | ||
let info = SimplifyContext::new(&execution_props); | ||
let expr = ExprSimplifier::new(info).simplify(expr.clone())?; | ||
|
||
if let Expr::Literal(ScalarValue::Int64(Some(limit))) = expr { | ||
Ok(limit as usize) | ||
} else { | ||
plan_err!("Limit must be an integer") | ||
} | ||
}) | ||
.transpose()?; | ||
|
||
let (schema, batches) = read_csv_batches(path)?; | ||
|
||
let table = LocalCsvTable { | ||
schema, | ||
limit, | ||
batches, | ||
}; | ||
Ok(Arc::new(table)) | ||
} | ||
} | ||
|
||
fn read_csv_batches(csv_path: impl AsRef<Path>) -> Result<(SchemaRef, Vec<RecordBatch>)> { | ||
let mut file = File::open(csv_path)?; | ||
let (schema, _) = Format::default().infer_schema(&mut file, None)?; | ||
file.rewind()?; | ||
|
||
let reader = ReaderBuilder::new(Arc::new(schema.clone())) | ||
.with_header(true) | ||
.build(file)?; | ||
let mut batches = vec![]; | ||
for bacth in reader { | ||
batches.push(bacth?); | ||
} | ||
let schema = Arc::new(schema); | ||
Ok((schema, batches)) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,56 @@ | ||
// Licensed to the Apache Software Foundation (ASF) under one | ||
// or more contributor license agreements. See the NOTICE file | ||
// distributed with this work for additional information | ||
// regarding copyright ownership. The ASF licenses this file | ||
// to you under the Apache License, Version 2.0 (the | ||
// "License"); you may not use this file except in compliance | ||
// with the License. You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, | ||
// software distributed under the License is distributed on an | ||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
// KIND, either express or implied. See the License for the | ||
// specific language governing permissions and limitations | ||
// under the License. | ||
|
||
//! A table that uses a function to generate data | ||
|
||
use super::TableProvider; | ||
|
||
use datafusion_common::Result; | ||
use datafusion_expr::Expr; | ||
|
||
use std::sync::Arc; | ||
|
||
/// A trait for table function implementations | ||
pub trait TableFunctionImpl: Sync + Send { | ||
/// Create a table provider | ||
fn call(&self, args: &[Expr]) -> Result<Arc<dyn TableProvider>>; | ||
} | ||
|
||
/// A table that uses a function to generate data | ||
pub struct TableFunction { | ||
/// Name of the table function | ||
name: String, | ||
/// Function implementation | ||
fun: Arc<dyn TableFunctionImpl>, | ||
} | ||
|
||
impl TableFunction { | ||
/// Create a new table function | ||
pub fn new(name: String, fun: Arc<dyn TableFunctionImpl>) -> Self { | ||
Self { name, fun } | ||
} | ||
|
||
/// Get the name of the table function | ||
pub fn name(&self) -> &str { | ||
&self.name | ||
} | ||
|
||
/// Get the function implementation and generate a table | ||
pub fn create_table_provider(&self, args: &[Expr]) -> Result<Arc<dyn TableProvider>> { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 |
||
self.fun.call(args) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This API is nice and is as specified in #7926. I think it will work for using a table function as a relation in the query (aka like a table with parameters)
The one thing I don't think this API supports is TableFunctions that take other arguments (aka that are fed the result of a table / can use the value of correlated subqueries as mentioned by @yukkit and @Jesse-Bakker #7926 (comment).
I can think of two options:
I personally prefer 1 as I think it offers several additional use cases, even though it doesn't cover "take a table input".
Any other thoughts?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One specific use case for table-valued arguments to table-valued functions is, for example windowing tvf's like in apache flink.
Example which cannot be expressed by taking
Expr
arguments (maybe ifExpr::Row()
is added?):That can also be emulated, however, using something like:
which doesn't need table-valued arguments (but does need to resolve
Expr::Column(name=bidtime)
. I'm not sure if the current API can do that?).Anyway, the current API is nice, and definitely very useful 👍