Skip to content

Commit fb91556

Browse files
andygrovetustvoldfrancis-du
authored
DataFusion 13.0.0 (#61)
* Update for changes in apache/arrow-rs#2711 * Clippy * Use Vec conversion * Update to DF 13 (#59) * [DataFrame] - Add write_csv/write_parquet/write_json to DataFrame (#58) * [SessionContext] - Add read_csv/read_parquet/read_avro functions to SessionContext (#57) Co-authored-by: Francis Du <[email protected]> * remove patch from cargo toml * add notes on git submodule for test data Co-authored-by: Raphael Taylor-Davies <[email protected]> Co-authored-by: Francis Du <[email protected]>
1 parent 55909a8 commit fb91556

10 files changed

+520
-277
lines changed

Cargo.lock

Lines changed: 437 additions & 225 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

Cargo.toml

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -34,9 +34,9 @@ default = ["mimalloc"]
3434
tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync"] }
3535
rand = "0.7"
3636
pyo3 = { version = "~0.17.1", features = ["extension-module", "abi3", "abi3-py37"] }
37-
datafusion = { version = "^12.0.0", features = ["pyarrow", "avro"] }
38-
datafusion-expr = { version = "^12.0.0" }
39-
datafusion-common = { version = "^12.0.0", features = ["pyarrow"] }
37+
datafusion = { version = "^13.0.0", features = ["pyarrow", "avro"] }
38+
datafusion-expr = { version = "^13.0.0" }
39+
datafusion-common = { version = "^13.0.0", features = ["pyarrow"] }
4040
uuid = { version = "0.8", features = ["v4"] }
4141
mimalloc = { version = "*", optional = true, default-features = false }
4242
async-trait = "0.1"
@@ -51,4 +51,4 @@ name = "datafusion._internal"
5151

5252
[profile.release]
5353
lto = true
54-
codegen-units = 1
54+
codegen-units = 1

README.md

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -163,6 +163,13 @@ python -m pip install -U pip
163163
python -m pip install -r requirements-310.txt
164164
```
165165

166+
The tests rely on test data in git submodules.
167+
168+
```bash
169+
git submodule init
170+
git submodule update
171+
```
172+
166173
Whenever rust code changes (your changes or via `git pull`):
167174

168175
```bash

src/context.rs

Lines changed: 25 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ use pyo3::exceptions::{PyKeyError, PyValueError};
2424
use pyo3::prelude::*;
2525

2626
use datafusion::arrow::datatypes::Schema;
27+
use datafusion::arrow::pyarrow::PyArrowType;
2728
use datafusion::arrow::record_batch::RecordBatch;
2829
use datafusion::datasource::datasource::TableProvider;
2930
use datafusion::datasource::MemTable;
@@ -99,9 +100,12 @@ impl PySessionContext {
99100
Ok(PyDataFrame::new(df))
100101
}
101102

102-
fn create_dataframe(&mut self, partitions: Vec<Vec<RecordBatch>>) -> PyResult<PyDataFrame> {
103-
let table = MemTable::try_new(partitions[0][0].schema(), partitions)
104-
.map_err(DataFusionError::from)?;
103+
fn create_dataframe(
104+
&mut self,
105+
partitions: PyArrowType<Vec<Vec<RecordBatch>>>,
106+
) -> PyResult<PyDataFrame> {
107+
let schema = partitions.0[0][0].schema();
108+
let table = MemTable::try_new(schema, partitions.0).map_err(DataFusionError::from)?;
105109

106110
// generate a random (unique) name for this table
107111
// table name cannot start with numeric digit
@@ -136,10 +140,10 @@ impl PySessionContext {
136140
fn register_record_batches(
137141
&mut self,
138142
name: &str,
139-
partitions: Vec<Vec<RecordBatch>>,
143+
partitions: PyArrowType<Vec<Vec<RecordBatch>>>,
140144
) -> PyResult<()> {
141-
let schema = partitions[0][0].schema();
142-
let table = MemTable::try_new(schema, partitions)?;
145+
let schema = partitions.0[0][0].schema();
146+
let table = MemTable::try_new(schema, partitions.0)?;
143147
self.ctx
144148
.register_table(name, Arc::new(table))
145149
.map_err(DataFusionError::from)?;
@@ -182,7 +186,7 @@ impl PySessionContext {
182186
&mut self,
183187
name: &str,
184188
path: PathBuf,
185-
schema: Option<Schema>,
189+
schema: Option<PyArrowType<Schema>>,
186190
has_header: bool,
187191
delimiter: &str,
188192
schema_infer_max_records: usize,
@@ -204,7 +208,7 @@ impl PySessionContext {
204208
.delimiter(delimiter[0])
205209
.schema_infer_max_records(schema_infer_max_records)
206210
.file_extension(file_extension);
207-
options.schema = schema.as_ref();
211+
options.schema = schema.as_ref().map(|x| &x.0);
208212

209213
let result = self.ctx.register_csv(name, path, options);
210214
wait_for_future(py, result).map_err(DataFusionError::from)?;
@@ -277,7 +281,7 @@ impl PySessionContext {
277281
fn read_csv(
278282
&self,
279283
path: PathBuf,
280-
schema: Option<Schema>,
284+
schema: Option<PyArrowType<Schema>>,
281285
has_header: bool,
282286
delimiter: &str,
283287
schema_infer_max_records: usize,
@@ -302,12 +306,17 @@ impl PySessionContext {
302306
.schema_infer_max_records(schema_infer_max_records)
303307
.file_extension(file_extension)
304308
.table_partition_cols(table_partition_cols);
305-
options.schema = schema.as_ref();
306309

307-
let result = self.ctx.read_csv(path, options);
308-
let df = PyDataFrame::new(wait_for_future(py, result).map_err(DataFusionError::from)?);
309-
310-
Ok(df)
310+
if let Some(py_schema) = schema {
311+
options.schema = Some(&py_schema.0);
312+
let result = self.ctx.read_csv(path, options);
313+
let df = PyDataFrame::new(wait_for_future(py, result).map_err(DataFusionError::from)?);
314+
Ok(df)
315+
} else {
316+
let result = self.ctx.read_csv(path, options);
317+
let df = PyDataFrame::new(wait_for_future(py, result).map_err(DataFusionError::from)?);
318+
Ok(df)
319+
}
311320
}
312321

313322
#[allow(clippy::too_many_arguments)]
@@ -346,14 +355,14 @@ impl PySessionContext {
346355
fn read_avro(
347356
&self,
348357
path: &str,
349-
schema: Option<Schema>,
358+
schema: Option<PyArrowType<Schema>>,
350359
table_partition_cols: Vec<String>,
351360
file_extension: &str,
352361
py: Python,
353362
) -> PyResult<PyDataFrame> {
354363
let mut options = AvroReadOptions::default().table_partition_cols(table_partition_cols);
355364
options.file_extension = file_extension;
356-
options.schema = schema.map(Arc::new);
365+
options.schema = schema.map(|s| Arc::new(s.0));
357366

358367
let result = self.ctx.read_avro(path, options);
359368
let df = PyDataFrame::new(wait_for_future(py, result).map_err(DataFusionError::from)?);

src/dataframe.rs

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
use crate::utils::wait_for_future;
1919
use crate::{errors::DataFusionError, expression::PyExpr};
2020
use datafusion::arrow::datatypes::Schema;
21-
use datafusion::arrow::pyarrow::PyArrowConvert;
21+
use datafusion::arrow::pyarrow::{PyArrowConvert, PyArrowException, PyArrowType};
2222
use datafusion::arrow::util::pretty;
2323
use datafusion::dataframe::DataFrame;
2424
use datafusion::prelude::*;
@@ -65,8 +65,8 @@ impl PyDataFrame {
6565
}
6666

6767
/// Returns the schema from the logical plan
68-
fn schema(&self) -> Schema {
69-
self.df.schema().into()
68+
fn schema(&self) -> PyArrowType<Schema> {
69+
PyArrowType(self.df.schema().into())
7070
}
7171

7272
#[args(args = "*")]
@@ -144,7 +144,7 @@ impl PyDataFrame {
144144
fn show(&self, py: Python, num: usize) -> PyResult<()> {
145145
let df = self.df.limit(0, Some(num))?;
146146
let batches = wait_for_future(py, df.collect())?;
147-
Ok(pretty::print_batches(&batches)?)
147+
pretty::print_batches(&batches).map_err(|err| PyArrowException::new_err(err.to_string()))
148148
}
149149

150150
/// Filter out duplicate rows
@@ -186,7 +186,7 @@ impl PyDataFrame {
186186
fn explain(&self, py: Python, verbose: bool, analyze: bool) -> PyResult<()> {
187187
let df = self.df.explain(verbose, analyze)?;
188188
let batches = wait_for_future(py, df.collect())?;
189-
Ok(pretty::print_batches(&batches)?)
189+
pretty::print_batches(&batches).map_err(|err| PyArrowException::new_err(err.to_string()))
190190
}
191191

192192
/// Repartition a `DataFrame` based on a logical partitioning scheme.

src/dataset.rs

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ use std::sync::Arc;
2727
use async_trait::async_trait;
2828

2929
use datafusion::arrow::datatypes::SchemaRef;
30+
use datafusion::arrow::pyarrow::PyArrowType;
3031
use datafusion::datasource::datasource::TableProviderFilterPushDown;
3132
use datafusion::datasource::{TableProvider, TableType};
3233
use datafusion::error::{DataFusionError, Result as DFResult};
@@ -74,7 +75,14 @@ impl TableProvider for Dataset {
7475
Python::with_gil(|py| {
7576
let dataset = self.dataset.as_ref(py);
7677
// This can panic but since we checked that self.dataset is a pyarrow.dataset.Dataset it should never
77-
Arc::new(dataset.getattr("schema").unwrap().extract().unwrap())
78+
Arc::new(
79+
dataset
80+
.getattr("schema")
81+
.unwrap()
82+
.extract::<PyArrowType<_>>()
83+
.unwrap()
84+
.0,
85+
)
7886
})
7987
}
8088

src/dataset_exec.rs

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ use futures::stream;
2828
use datafusion::arrow::datatypes::SchemaRef;
2929
use datafusion::arrow::error::ArrowError;
3030
use datafusion::arrow::error::Result as ArrowResult;
31+
use datafusion::arrow::pyarrow::PyArrowType;
3132
use datafusion::arrow::record_batch::RecordBatch;
3233
use datafusion::error::{DataFusionError as InnerDataFusionError, Result as DFResult};
3334
use datafusion::execution::context::TaskContext;
@@ -54,7 +55,7 @@ impl Iterator for PyArrowBatchesAdapter {
5455
Some(
5556
batches
5657
.next()?
57-
.and_then(|batch| batch.extract())
58+
.and_then(|batch| Ok(batch.extract::<PyArrowType<_>>()?.0))
5859
.map_err(|err| ArrowError::ExternalError(Box::new(err))),
5960
)
6061
})
@@ -109,7 +110,12 @@ impl DatasetExec {
109110

110111
let scanner = dataset.call_method("scanner", (), Some(kwargs))?;
111112

112-
let schema = Arc::new(scanner.getattr("projected_schema")?.extract()?);
113+
let schema = Arc::new(
114+
scanner
115+
.getattr("projected_schema")?
116+
.extract::<PyArrowType<_>>()?
117+
.0,
118+
);
113119

114120
let builtins = Python::import(py, "builtins")?;
115121
let pylist = builtins.getattr("list")?;
@@ -211,7 +217,7 @@ impl ExecutionPlan for DatasetExec {
211217
let schema: SchemaRef = Arc::new(
212218
scanner
213219
.getattr("projected_schema")
214-
.and_then(|schema| schema.extract())
220+
.and_then(|schema| Ok(schema.extract::<PyArrowType<_>>()?.0))
215221
.map_err(|err| InnerDataFusionError::External(Box::new(err)))?,
216222
);
217223
let record_batches: &PyIterator = scanner

src/expression.rs

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ use pyo3::{basic::CompareOp, prelude::*};
1919
use std::convert::{From, Into};
2020

2121
use datafusion::arrow::datatypes::DataType;
22+
use datafusion::arrow::pyarrow::PyArrowType;
2223
use datafusion_expr::{col, lit, Expr};
2324

2425
use datafusion::scalar::ScalarValue;
@@ -125,12 +126,12 @@ impl PyExpr {
125126
self.expr.clone().is_null().into()
126127
}
127128

128-
pub fn cast(&self, to: DataType) -> PyExpr {
129+
pub fn cast(&self, to: PyArrowType<DataType>) -> PyExpr {
129130
// self.expr.cast_to() requires DFSchema to validate that the cast
130131
// is supported, omit that for now
131132
let expr = Expr::Cast {
132133
expr: Box::new(self.expr.clone()),
133-
data_type: to,
134+
data_type: to.0,
134135
};
135136
expr.into()
136137
}

src/udaf.rs

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,9 @@ use std::sync::Arc;
1919

2020
use pyo3::{prelude::*, types::PyTuple};
2121

22-
use datafusion::arrow::array::ArrayRef;
22+
use datafusion::arrow::array::{Array, ArrayRef};
2323
use datafusion::arrow::datatypes::DataType;
24-
use datafusion::arrow::pyarrow::PyArrowConvert;
24+
use datafusion::arrow::pyarrow::{PyArrowConvert, PyArrowType};
2525
use datafusion::common::ScalarValue;
2626
use datafusion::error::{DataFusionError, Result};
2727
use datafusion_expr::{
@@ -82,6 +82,7 @@ impl Accumulator for RustAccumulator {
8282

8383
// 1. cast states to Pyarrow array
8484
let state = state
85+
.data()
8586
.to_pyarrow(py)
8687
.map_err(|e| DataFusionError::Execution(format!("{}", e)))?;
8788

@@ -120,18 +121,18 @@ impl PyAggregateUDF {
120121
fn new(
121122
name: &str,
122123
accumulator: PyObject,
123-
input_type: DataType,
124-
return_type: DataType,
125-
state_type: Vec<DataType>,
124+
input_type: PyArrowType<DataType>,
125+
return_type: PyArrowType<DataType>,
126+
state_type: PyArrowType<Vec<DataType>>,
126127
volatility: &str,
127128
) -> PyResult<Self> {
128129
let function = create_udaf(
129130
name,
130-
input_type,
131-
Arc::new(return_type),
131+
input_type.0,
132+
Arc::new(return_type.0),
132133
parse_volatility(volatility)?,
133134
to_rust_accumulator(accumulator),
134-
Arc::new(state_type),
135+
Arc::new(state_type.0),
135136
);
136137
Ok(Self { function })
137138
}

src/udf.rs

Lines changed: 12 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,9 @@ use std::sync::Arc;
1919

2020
use pyo3::{prelude::*, types::PyTuple};
2121

22-
use datafusion::arrow::array::ArrayRef;
22+
use datafusion::arrow::array::{make_array, Array, ArrayData, ArrayRef};
2323
use datafusion::arrow::datatypes::DataType;
24-
use datafusion::arrow::pyarrow::PyArrowConvert;
24+
use datafusion::arrow::pyarrow::{PyArrowConvert, PyArrowType};
2525
use datafusion::error::DataFusionError;
2626
use datafusion::physical_plan::functions::make_scalar_function;
2727
use datafusion::physical_plan::udf::ScalarUDF;
@@ -46,15 +46,14 @@ fn to_rust_function(func: PyObject) -> ScalarFunctionImplementation {
4646
let py_args = PyTuple::new(py, py_args);
4747

4848
// 2. call function
49-
let value = func.as_ref(py).call(py_args, None);
50-
let value = match value {
51-
Ok(n) => Ok(n),
52-
Err(error) => Err(DataFusionError::Execution(format!("{:?}", error))),
53-
}?;
49+
let value = func
50+
.as_ref(py)
51+
.call(py_args, None)
52+
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?;
5453

5554
// 3. cast to arrow::array::Array
56-
let array = ArrayRef::from_pyarrow(value).unwrap();
57-
Ok(array)
55+
let array_data = ArrayData::from_pyarrow(value).unwrap();
56+
Ok(make_array(array_data))
5857
})
5958
},
6059
)
@@ -73,14 +72,14 @@ impl PyScalarUDF {
7372
fn new(
7473
name: &str,
7574
func: PyObject,
76-
input_types: Vec<DataType>,
77-
return_type: DataType,
75+
input_types: PyArrowType<Vec<DataType>>,
76+
return_type: PyArrowType<DataType>,
7877
volatility: &str,
7978
) -> PyResult<Self> {
8079
let function = create_udf(
8180
name,
82-
input_types,
83-
Arc::new(return_type),
81+
input_types.0,
82+
Arc::new(return_type.0),
8483
parse_volatility(volatility)?,
8584
to_rust_function(func),
8685
);

0 commit comments

Comments
 (0)