forked from datafusion-contrib/datafusion-python
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathcontext.rs
More file actions
178 lines (154 loc) · 5.81 KB
/
context.rs
File metadata and controls
178 lines (154 loc) · 5.81 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
// 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 std::path::PathBuf;
use std::{collections::HashSet, sync::Arc};
use uuid::Uuid;
use pyo3::exceptions::{PyKeyError, PyValueError};
use pyo3::prelude::*;
use datafusion::arrow::datatypes::Schema;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::datasource::MemTable;
use datafusion::execution::context::ExecutionContext;
use datafusion::prelude::CsvReadOptions;
use crate::catalog::PyCatalog;
use crate::dataframe::PyDataFrame;
use crate::dataset::PyArrowDatasetTable;
use crate::errors::DataFusionError;
use crate::udf::PyScalarUDF;
use crate::utils::wait_for_future;
/// `PyExecutionContext` is able to plan and execute DataFusion plans.
/// It has a powerful optimizer, a physical planner for local execution, and a
/// multi-threaded execution engine to perform the execution.
#[pyclass(name = "ExecutionContext", module = "datafusion", subclass, unsendable)]
pub(crate) struct PyExecutionContext {
ctx: ExecutionContext,
}
#[pymethods]
impl PyExecutionContext {
// TODO(kszucs): should expose the configuration options as keyword arguments
#[new]
fn new() -> Self {
PyExecutionContext {
ctx: ExecutionContext::new(),
}
}
/// Returns a PyDataFrame whose plan corresponds to the SQL statement.
fn sql(&mut self, query: &str, py: Python) -> PyResult<PyDataFrame> {
let result = self.ctx.sql(query);
let df = wait_for_future(py, result).map_err(DataFusionError::from)?;
Ok(PyDataFrame::new(df))
}
fn create_dataframe(&mut self, partitions: Vec<Vec<RecordBatch>>) -> PyResult<PyDataFrame> {
let table = MemTable::try_new(partitions[0][0].schema(), partitions)
.map_err(DataFusionError::from)?;
// generate a random (unique) name for this table
// table name cannot start with numeric digit
let name = "c".to_owned()
+ &Uuid::new_v4()
.to_simple()
.encode_lower(&mut Uuid::encode_buffer());
self.ctx
.register_table(&*name, Arc::new(table))
.map_err(DataFusionError::from)?;
let table = self.ctx.table(&*name).map_err(DataFusionError::from)?;
let df = PyDataFrame::new(table);
Ok(df)
}
fn register_record_batches(
&mut self,
name: &str,
partitions: Vec<Vec<RecordBatch>>,
) -> PyResult<()> {
let schema = partitions[0][0].schema();
let table = MemTable::try_new(schema, partitions)?;
self.ctx
.register_table(name, Arc::new(table))
.map_err(DataFusionError::from)?;
Ok(())
}
fn register_parquet(&mut self, name: &str, path: &str, py: Python) -> PyResult<()> {
let result = self.ctx.register_parquet(name, path);
wait_for_future(py, result).map_err(DataFusionError::from)?;
Ok(())
}
#[args(
schema = "None",
has_header = "true",
delimiter = "\",\"",
schema_infer_max_records = "1000",
file_extension = "\".csv\""
)]
fn register_csv(
&mut self,
name: &str,
path: PathBuf,
schema: Option<Schema>,
has_header: bool,
delimiter: &str,
schema_infer_max_records: usize,
file_extension: &str,
py: Python,
) -> PyResult<()> {
let path = path
.to_str()
.ok_or(PyValueError::new_err("Unable to convert path to a string"))?;
let delimiter = delimiter.as_bytes();
if delimiter.len() != 1 {
return Err(PyValueError::new_err(
"Delimiter must be a single character",
));
}
let mut options = CsvReadOptions::new()
.has_header(has_header)
.delimiter(delimiter[0])
.schema_infer_max_records(schema_infer_max_records)
.file_extension(file_extension);
options.schema = schema.as_ref();
let result = self.ctx.register_csv(name, path, options);
wait_for_future(py, result).map_err(DataFusionError::from)?;
Ok(())
}
fn register_dataset(&mut self, name: &str, dataset: PyArrowDatasetTable) -> PyResult<()> {
self.ctx
.register_table(name, Arc::new(dataset))
.map_err(DataFusionError::from)?;
Ok(())
}
fn register_udf(&mut self, udf: PyScalarUDF) -> PyResult<()> {
self.ctx.register_udf(udf.function);
Ok(())
}
#[args(name = "\"datafusion\"")]
fn catalog(&self, name: &str) -> PyResult<PyCatalog> {
match self.ctx.catalog(name) {
Some(catalog) => Ok(PyCatalog::new(catalog)),
None => Err(PyKeyError::new_err(format!(
"Catalog with name {} doesn't exist.",
&name
))),
}
}
fn tables(&self) -> HashSet<String> {
self.ctx.tables().unwrap()
}
fn table(&self, name: &str) -> PyResult<PyDataFrame> {
Ok(PyDataFrame::new(self.ctx.table(name)?))
}
fn empty_table(&self) -> PyResult<PyDataFrame> {
Ok(PyDataFrame::new(self.ctx.read_empty()?))
}
}