forked from apache/datafusion-python
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathrecord_batch.rs
More file actions
105 lines (92 loc) · 3.22 KB
/
record_batch.rs
File metadata and controls
105 lines (92 loc) · 3.22 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
// 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::sync::Arc;
use crate::errors::PyDataFusionError;
use crate::utils::wait_for_future;
use datafusion::arrow::pyarrow::ToPyArrow;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::physical_plan::SendableRecordBatchStream;
use futures::StreamExt;
use pyo3::exceptions::{PyStopAsyncIteration, PyStopIteration};
use pyo3::prelude::*;
use pyo3::{pyclass, pymethods, PyObject, PyResult, Python};
use tokio::sync::Mutex;
#[pyclass(name = "RecordBatch", module = "datafusion", subclass, frozen)]
pub struct PyRecordBatch {
batch: RecordBatch,
}
#[pymethods]
impl PyRecordBatch {
fn to_pyarrow(&self, py: Python) -> PyResult<PyObject> {
self.batch.to_pyarrow(py)
}
}
impl From<RecordBatch> for PyRecordBatch {
fn from(batch: RecordBatch) -> Self {
Self { batch }
}
}
#[pyclass(name = "RecordBatchStream", module = "datafusion", subclass, frozen)]
pub struct PyRecordBatchStream {
stream: Arc<Mutex<SendableRecordBatchStream>>,
}
impl PyRecordBatchStream {
pub fn new(stream: SendableRecordBatchStream) -> Self {
Self {
stream: Arc::new(Mutex::new(stream)),
}
}
}
#[pymethods]
impl PyRecordBatchStream {
fn next(&self, py: Python) -> PyResult<PyRecordBatch> {
let stream = self.stream.clone();
wait_for_future(py, next_stream(stream, true))?
}
fn __next__(&self, py: Python) -> PyResult<PyRecordBatch> {
self.next(py)
}
fn __anext__<'py>(&'py self, py: Python<'py>) -> PyResult<Bound<'py, PyAny>> {
let stream = self.stream.clone();
pyo3_async_runtimes::tokio::future_into_py(py, next_stream(stream, false))
}
fn __iter__(slf: PyRef<'_, Self>) -> PyRef<'_, Self> {
slf
}
fn __aiter__(slf: PyRef<'_, Self>) -> PyRef<'_, Self> {
slf
}
}
async fn next_stream(
stream: Arc<Mutex<SendableRecordBatchStream>>,
sync: bool,
) -> PyResult<PyRecordBatch> {
let mut stream = stream.lock().await;
match stream.next().await {
Some(Ok(batch)) => Ok(batch.into()),
Some(Err(e)) => Err(PyDataFusionError::from(e))?,
None => {
// Depending on whether the iteration is sync or not, we raise either a
// StopIteration or a StopAsyncIteration
if sync {
Err(PyStopIteration::new_err("stream exhausted"))
} else {
Err(PyStopAsyncIteration::new_err("stream exhausted"))
}
}
}
}