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

Rework the python bindings [WIP] #856

Closed
wants to merge 12 commits into from
Closed
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
2 changes: 1 addition & 1 deletion datafusion/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ unicode_expressions = ["unicode-segmentation"]

[dependencies]
ahash = "0.7"
hashbrown = "0.11"
hashbrown = { version = "0.11", features = ["raw"] }
arrow = { version = "5.1", features = ["prettyprint"] }
parquet = { version = "5.1", features = ["arrow"] }
sqlparser = "0.9.0"
Expand Down
6 changes: 3 additions & 3 deletions python/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -30,16 +30,16 @@ edition = "2018"
libc = "0.2"
tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync"] }
rand = "0.7"
pyo3 = { version = "0.14.1", features = ["extension-module"] }
pyo3 = { version = "0.14.2", features = ["extension-module"] }
datafusion = { git = "https://github.com/apache/arrow-datafusion.git", rev = "4d61196dee8526998aee7e7bb10ea88422e5f9e1" }

[lib]
name = "datafusion"
name = "internals"
crate-type = ["cdylib"]

[package.metadata.maturin]
name = "datafusion.internals"
requires-dist = ["pyarrow>=1"]

classifier = [
"Development Status :: 2 - Pre-Alpha",
"Intended Audience :: Developers",
Expand Down
28 changes: 28 additions & 0 deletions python/datafusion/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
# 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.

from .internals import PyDataFrame as DataFrame
from .internals import PyExecutionContext as ExecutionContext
from .internals import PyExpr as Expr
from .internals import functions

__all__ = [
Copy link
Member Author

Choose a reason for hiding this comment

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

@jorgecarleitao the symbols are exported without the Py prefix.

"DataFrame",
"ExecutionContext",
"Expr",
"functions"
]
File renamed without changes.
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import pyarrow as pa
import pytest

from datafusion import ExecutionContext
from datafusion import functions as f

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import numpy as np
import pyarrow as pa
import pytest

from datafusion import ExecutionContext
from datafusion import functions as f

Expand Down
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import pytest

from datafusion import ExecutionContext

from . import generic as helpers


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import pyarrow as pa
import pytest

from datafusion import ExecutionContext
from datafusion import functions as f

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import pyarrow as pa
import pyarrow.compute as pc
import pytest

from datafusion import ExecutionContext
from datafusion import functions as f

Expand Down
9 changes: 9 additions & 0 deletions python/pyproject.toml
Original file line number Diff line number Diff line change
Expand Up @@ -18,3 +18,12 @@
[build-system]
requires = ["maturin>=0.11,<0.12"]
build-backend = "maturin"

[project]
name = "datafusion"
dependencies = [
"pyarrow"
]

[tool.isort]
profile = "black"
2 changes: 1 addition & 1 deletion python/rust-toolchain
Original file line number Diff line number Diff line change
@@ -1 +1 @@
nightly-2021-05-10
stable
82 changes: 42 additions & 40 deletions python/src/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,79 +24,79 @@ use rand::Rng;
use pyo3::exceptions::PyValueError;
use pyo3::prelude::*;

use datafusion::arrow::datatypes::Schema;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::datasource::MemTable;
use datafusion::execution::context::ExecutionContext as _ExecutionContext;
use datafusion::execution::context::ExecutionContext;
use datafusion::prelude::CsvReadOptions;

use crate::dataframe;
use crate::errors;
use crate::dataframe::PyDataFrame;
use crate::errors::DataFusionError;
use crate::functions;
use crate::to_rust;
use crate::types::PyDataType;
use crate::pyarrow::PyArrowConvert;

/// `ExecutionContext` is able to plan and execute DataFusion plans.
/// `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(unsendable)]
pub(crate) struct ExecutionContext {
ctx: _ExecutionContext,
pub(crate) struct PyExecutionContext {
ctx: ExecutionContext,
}

#[pymethods]
impl ExecutionContext {
impl PyExecutionContext {
#[new]
fn new() -> Self {
ExecutionContext {
ctx: _ExecutionContext::new(),
PyExecutionContext {
ctx: ExecutionContext::new(),
}
}

/// Returns a DataFrame whose plan corresponds to the SQL statement.
fn sql(&mut self, query: &str) -> PyResult<dataframe::DataFrame> {
let df = self
.ctx
.sql(query)
.map_err(|e| -> errors::DataFusionError { e.into() })?;
Ok(dataframe::DataFrame::new(
/// Returns a PyDataFrame whose plan corresponds to the SQL statement.
fn sql(&mut self, query: &str) -> PyResult<PyDataFrame> {
let df = self.ctx.sql(query).map_err(DataFusionError::from)?;
Ok(PyDataFrame::new(
self.ctx.state.clone(),
df.to_logical_plan(),
))
}

fn create_dataframe(
&mut self,
partitions: Vec<Vec<PyObject>>,
py: Python,
) -> PyResult<dataframe::DataFrame> {
partitions: Vec<Vec<&PyAny>>,
) -> PyResult<PyDataFrame> {
let partitions: Vec<Vec<RecordBatch>> = partitions
.iter()
.into_iter()
.map(|batches| {
batches
.iter()
.map(|batch| to_rust::to_rust_batch(batch.as_ref(py)))
.collect()
.into_iter()
.map(RecordBatch::from_pyarrow)
.collect::<PyResult<_>>()
})
.collect::<PyResult<_>>()?;

let table =
errors::wrap(MemTable::try_new(partitions[0][0].schema(), partitions))?;
let table = MemTable::try_new(partitions[0][0].schema(), partitions)
.map_err(DataFusionError::from)?;

// generate a random (unique) name for this table
let name = rand::thread_rng()
.sample_iter(&Alphanumeric)
.take(10)
.collect::<String>();

errors::wrap(self.ctx.register_table(&*name, Arc::new(table)))?;
Ok(dataframe::DataFrame::new(
self.ctx.state.clone(),
errors::wrap(self.ctx.table(&*name))?.to_logical_plan(),
))
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(self.ctx.state.clone(), table.to_logical_plan());
Ok(df)
}

fn register_parquet(&mut self, name: &str, path: &str) -> PyResult<()> {
errors::wrap(self.ctx.register_parquet(name, path))?;
self.ctx
.register_parquet(name, path)
.map_err(DataFusionError::from)?;
Ok(())
}

Expand All @@ -121,7 +121,7 @@ impl ExecutionContext {
.to_str()
.ok_or(PyValueError::new_err("Unable to convert path to a string"))?;
let schema = match schema {
Some(s) => Some(to_rust::to_rust_schema(s)?),
Some(s) => Some(Schema::from_pyarrow(s)?),
None => None,
};
let delimiter = delimiter.as_bytes();
Expand All @@ -138,20 +138,22 @@ impl ExecutionContext {
.file_extension(file_extension);
options.schema = schema.as_ref();

errors::wrap(self.ctx.register_csv(name, path, options))?;
self.ctx
.register_csv(name, path, options)
.map_err(DataFusionError::from)?;
Ok(())
}

fn register_udf(
&mut self,
name: &str,
func: PyObject,
args_types: Vec<PyDataType>,
return_type: PyDataType,
) {
let function = functions::create_udf(func, args_types, return_type, name);

args_types: Vec<&PyAny>,
return_type: &PyAny,
) -> PyResult<()> {
let function = functions::create_udf(func, args_types, return_type, name)?;
self.ctx.register_udf(function.function);
Ok(())
}

fn tables(&self) -> HashSet<String> {
Expand Down
Loading