Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add ability to execute ExecutionPlan and get a stream of RecordBatch #186

Merged
merged 4 commits into from
Feb 16, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 9 additions & 0 deletions datafusion/tests/test_dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -388,6 +388,15 @@ def test_execution_plan(aggregate_df):
assert "RepartitionExec:" in indent
assert "CsvExec:" in indent

ctx = SessionContext()
stream = ctx.execute(plan, 0)
# get the one and only batch
batch = stream.next()
assert batch is not None
# there should be no more batches
batch = stream.next()
assert batch is None


def test_repartition(df):
df.repartition(2)
Expand Down
33 changes: 31 additions & 2 deletions src/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,9 @@ use pyo3::prelude::*;
use crate::catalog::{PyCatalog, PyTable};
use crate::dataframe::PyDataFrame;
use crate::dataset::Dataset;
use crate::errors::DataFusionError;
use crate::errors::{py_datafusion_err, DataFusionError};
use crate::physical_plan::PyExecutionPlan;
use crate::record_batch::PyRecordBatchStream;
use crate::sql::logical::PyLogicalPlan;
use crate::store::StorageContexts;
use crate::udaf::PyAggregateUDF;
Expand All @@ -39,14 +41,17 @@ use datafusion::arrow::pyarrow::PyArrowType;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::datasource::datasource::TableProvider;
use datafusion::datasource::MemTable;
use datafusion::execution::context::{SessionConfig, SessionContext};
use datafusion::execution::context::{SessionConfig, SessionContext, TaskContext};
use datafusion::execution::disk_manager::DiskManagerConfig;
use datafusion::execution::memory_pool::{FairSpillPool, GreedyMemoryPool, UnboundedMemoryPool};
use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv};
use datafusion::physical_plan::SendableRecordBatchStream;
use datafusion::prelude::{
AvroReadOptions, CsvReadOptions, DataFrame, NdJsonReadOptions, ParquetReadOptions,
};
use datafusion_common::ScalarValue;
use tokio::runtime::Runtime;
use tokio::task::JoinHandle;

#[pyclass(name = "SessionConfig", module = "datafusion", subclass, unsendable)]
#[derive(Clone, Default)]
Expand Down Expand Up @@ -579,6 +584,30 @@ impl PySessionContext {
Err(err) => Ok(format!("Error: {:?}", err.to_string())),
}
}

/// Execute a partition of an execution plan and return a stream of record batches
pub fn execute(
&self,
plan: PyExecutionPlan,
part: usize,
py: Python,
) -> PyResult<PyRecordBatchStream> {
let ctx = Arc::new(TaskContext::new(
"task_id".to_string(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Created an issue for more general configuration setup #187

"session_id".to_string(),
HashMap::new(),
HashMap::new(),
HashMap::new(),
Arc::new(RuntimeEnv::default()),
));
// create a Tokio runtime to run the async code
let rt = Runtime::new().unwrap();
let plan = plan.plan.clone();
let fut: JoinHandle<datafusion_common::Result<SendableRecordBatchStream>> =
rt.spawn(async move { plan.execute(part, ctx) });
let stream = wait_for_future(py, fut).map_err(|e| py_datafusion_err(e))?;
Ok(PyRecordBatchStream::new(stream?))
}
}

impl PySessionContext {
Expand Down
1 change: 1 addition & 0 deletions src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ mod expr;
mod functions;
pub mod physical_plan;
mod pyarrow_filter_expression;
mod record_batch;
pub mod sql;
pub mod store;
pub mod substrait;
Expand Down
64 changes: 64 additions & 0 deletions src/record_batch.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
// 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 crate::utils::wait_for_future;
use datafusion::arrow::pyarrow::PyArrowConvert;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::physical_plan::SendableRecordBatchStream;
use futures::StreamExt;
use pyo3::{pyclass, pymethods, PyObject, PyResult, Python};

#[pyclass(name = "RecordBatch", module = "datafusion", subclass)]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Eventually I want to add a datafusion.arrow module but I don't quite have all the pieces in place for that yet.

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)]
pub struct PyRecordBatchStream {
stream: SendableRecordBatchStream,
}

impl PyRecordBatchStream {
pub fn new(stream: SendableRecordBatchStream) -> Self {
Self { stream }
}
}

#[pymethods]
impl PyRecordBatchStream {
fn next(&mut self, py: Python) -> PyResult<Option<PyRecordBatch>> {
let result = self.stream.next();
match wait_for_future(py, result) {
None => Ok(None),
Some(Ok(b)) => Ok(Some(b.into())),
Some(Err(e)) => Err(e.into()),
}
}
}