forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 1
Expand file tree
/
Copy pathsimple_udtf.rs
More file actions
183 lines (163 loc) · 6 KB
/
simple_udtf.rs
File metadata and controls
183 lines (163 loc) · 6 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
// 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::catalog::Session;
use datafusion::catalog::TableFunctionImpl;
use datafusion::common::{plan_err, ScalarValue};
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::TableProvider;
use datafusion::error::Result;
use datafusion::execution::context::ExecutionProps;
use datafusion::logical_expr::simplify::SimplifyContext;
use datafusion::logical_expr::{Expr, TableType};
use datafusion::optimizer::simplify_expressions::ExprSimplifier;
use datafusion::physical_plan::ExecutionPlan;
use datafusion::prelude::*;
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
#[derive(Debug)]
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: &dyn Session,
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(MemorySourceConfig::try_new_exec(
&[batches],
TableProvider::schema(self),
projection.cloned(),
)?)
}
}
#[derive(Debug)]
struct LocalCsvTableFunc {}
impl TableFunctionImpl for LocalCsvTableFunc {
fn call(&self, exprs: &[(Expr, Option<String>)]) -> Result<Arc<dyn TableProvider>> {
let Some((Expr::Literal(ScalarValue::Utf8(Some(ref path)), _), _)) =
exprs.first()
else {
return plan_err!("read_csv requires at least one string argument");
};
let limit = exprs
.get(1)
.map(|(expr, _)| {
// try to simplify 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 batch in reader {
batches.push(batch?);
}
let schema = Arc::new(schema);
Ok((schema, batches))
}