|
| 1 | +// Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +// or more contributor license agreements. See the NOTICE file |
| 3 | +// distributed with this work for additional information |
| 4 | +// regarding copyright ownership. The ASF licenses this file |
| 5 | +// to you under the Apache License, Version 2.0 (the |
| 6 | +// "License"); you may not use this file except in compliance |
| 7 | +// with the License. You may obtain a copy of the License at |
| 8 | +// |
| 9 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +// |
| 11 | +// Unless required by applicable law or agreed to in writing, |
| 12 | +// software distributed under the License is distributed on an |
| 13 | +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +// KIND, either express or implied. See the License for the |
| 15 | +// specific language governing permissions and limitations |
| 16 | +// under the License. |
| 17 | + |
| 18 | +use arrow::csv::reader::Format; |
| 19 | +use arrow::csv::ReaderBuilder; |
| 20 | +use async_trait::async_trait; |
| 21 | +use datafusion::arrow::datatypes::SchemaRef; |
| 22 | +use datafusion::arrow::record_batch::RecordBatch; |
| 23 | +use datafusion::datasource::streaming::StreamingTable; |
| 24 | +use datafusion::datasource::TableProvider; |
| 25 | +use datafusion::error::Result; |
| 26 | +use datafusion::execution::context::SessionState; |
| 27 | +use datafusion::physical_plan::memory::MemoryExec; |
| 28 | +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; |
| 29 | +use datafusion::physical_plan::streaming::PartitionStream; |
| 30 | +use datafusion::physical_plan::ExecutionPlan; |
| 31 | +use datafusion::prelude::SessionContext; |
| 32 | +use datafusion_expr::{Expr, TableType}; |
| 33 | +use std::fs::File; |
| 34 | +use std::io::Seek; |
| 35 | +use std::path::Path; |
| 36 | +use std::sync::Arc; |
| 37 | + |
| 38 | +// To define your own table function, you only need to do the following 3 things: |
| 39 | +// 1. Define a function that returns a Result<Arc<dyn TableProvider>> |
| 40 | +// maybe you can just implement your own TableProvider |
| 41 | +// and you can even implement your own PartitionStream and return it with inner StreamTable |
| 42 | +// 2. Register the function using ctx.register_udtf |
| 43 | + |
| 44 | +/// This example demonstrates how to register a TableFunction |
| 45 | +#[tokio::main] |
| 46 | +async fn main() -> Result<()> { |
| 47 | + // create local execution context |
| 48 | + let ctx = SessionContext::new(); |
| 49 | + |
| 50 | + ctx.register_udtf("read_csv", Arc::new(read_csv)); |
| 51 | + ctx.register_udtf("read_csv_stream", Arc::new(read_csv_stream)); |
| 52 | + |
| 53 | + let testdata = datafusion::test_util::arrow_test_data(); |
| 54 | + let csv_file = format!("{testdata}/csv/aggregate_test_100.csv"); |
| 55 | + |
| 56 | + let df = ctx |
| 57 | + .sql(format!("SELECT * FROM read_csv('{csv_file}');").as_str()) |
| 58 | + .await?; |
| 59 | + df.show().await?; |
| 60 | + |
| 61 | + let df2 = ctx |
| 62 | + .sql(format!("SELECT * FROM read_csv_stream('{csv_file}');").as_str()) |
| 63 | + .await?; |
| 64 | + df2.show().await?; |
| 65 | + |
| 66 | + Ok(()) |
| 67 | +} |
| 68 | + |
| 69 | +// Option1: (full implmentation of a TableProvider) |
| 70 | +// Define your own TableProvider and make a function return Arc<dyn Provider> |
| 71 | +struct LocalCsvTable { |
| 72 | + schema: SchemaRef, |
| 73 | + batches: Vec<RecordBatch>, |
| 74 | +} |
| 75 | + |
| 76 | +#[async_trait] |
| 77 | +impl TableProvider for LocalCsvTable { |
| 78 | + fn as_any(&self) -> &dyn std::any::Any { |
| 79 | + self |
| 80 | + } |
| 81 | + |
| 82 | + fn schema(&self) -> SchemaRef { |
| 83 | + self.schema.clone() |
| 84 | + } |
| 85 | + |
| 86 | + fn table_type(&self) -> TableType { |
| 87 | + TableType::Base |
| 88 | + } |
| 89 | + |
| 90 | + async fn scan( |
| 91 | + &self, |
| 92 | + _state: &SessionState, |
| 93 | + projection: Option<&Vec<usize>>, |
| 94 | + _filters: &[Expr], |
| 95 | + _limit: Option<usize>, |
| 96 | + ) -> Result<Arc<dyn ExecutionPlan>> { |
| 97 | + Ok(Arc::new(MemoryExec::try_new( |
| 98 | + &[self.batches.clone()], |
| 99 | + TableProvider::schema(self), |
| 100 | + projection.cloned(), |
| 101 | + )?)) |
| 102 | + } |
| 103 | +} |
| 104 | + |
| 105 | +fn read_csv(args: &[String]) -> Result<Arc<dyn TableProvider>> { |
| 106 | + let (schema, batches) = read_csv_batches(args[0].clone())?; |
| 107 | + let table = LocalCsvTable { schema, batches }; |
| 108 | + Ok(Arc::new(table)) |
| 109 | +} |
| 110 | + |
| 111 | +// Option2: (use StreamingTable to make it simpler) |
| 112 | +// Implement PartitionStream and make a function return it with StreamTable |
| 113 | +impl PartitionStream for LocalCsvTable { |
| 114 | + fn schema(&self) -> &SchemaRef { |
| 115 | + &self.schema |
| 116 | + } |
| 117 | + |
| 118 | + fn execute( |
| 119 | + &self, |
| 120 | + _ctx: Arc<datafusion::execution::TaskContext>, |
| 121 | + ) -> datafusion::physical_plan::SendableRecordBatchStream { |
| 122 | + Box::pin(RecordBatchStreamAdapter::new( |
| 123 | + self.schema.clone(), |
| 124 | + // You can even read data from network or else anywhere, using async is also ok |
| 125 | + // In Fact, you can even implement your own SendableRecordBatchStream |
| 126 | + // by implementing Stream<Item = ArrowResult<RecordBatch>> + Send + Sync + 'static |
| 127 | + futures::stream::iter(self.batches.clone().into_iter().map(Ok)), |
| 128 | + )) |
| 129 | + } |
| 130 | +} |
| 131 | + |
| 132 | +fn read_csv_stream(args: &[String]) -> Result<Arc<dyn TableProvider>> { |
| 133 | + let (schema, batches) = read_csv_batches(args[0].clone())?; |
| 134 | + let stream = LocalCsvTable { |
| 135 | + schema: schema.clone(), |
| 136 | + batches, |
| 137 | + }; |
| 138 | + let table = StreamingTable::try_new(schema, vec![Arc::new(stream)])?; |
| 139 | + Ok(Arc::new(table)) |
| 140 | +} |
| 141 | + |
| 142 | +fn read_csv_batches(csv_path: impl AsRef<Path>) -> Result<(SchemaRef, Vec<RecordBatch>)> { |
| 143 | + let mut file = File::open(csv_path)?; |
| 144 | + let (schema, _) = Format::default().infer_schema(&mut file, None)?; |
| 145 | + file.rewind()?; |
| 146 | + |
| 147 | + let reader = ReaderBuilder::new(Arc::new(schema.clone())) |
| 148 | + .with_header(true) |
| 149 | + .build(file)?; |
| 150 | + let mut batches = vec![]; |
| 151 | + for bacth in reader { |
| 152 | + batches.push(bacth?); |
| 153 | + } |
| 154 | + let schema = Arc::new(schema); |
| 155 | + Ok((schema, batches)) |
| 156 | +} |
0 commit comments