Skip to content
This repository has been archived by the owner on Jul 25, 2022. It is now read-only.

Implement PyArrow Dataset TableProvider #59

Open
wants to merge 3 commits into
base: main
Choose a base branch
from
Open
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: 2 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@ datafusion-expr = { version = "^8.0.0" }
datafusion-common = { version = "^8.0.0", features = ["pyarrow"] }
uuid = { version = "0.8", features = ["v4"] }
mimalloc = { version = "*", optional = true, default-features = false }
async-trait = "0.1"
futures = "0.3"

[lib]
name = "datafusion_python"
Expand Down
33 changes: 33 additions & 0 deletions datafusion/tests/test_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
# under the License.

import pyarrow as pa
import pyarrow.dataset as ds


def test_register_record_batches(ctx):
Expand Down Expand Up @@ -72,3 +73,35 @@ def test_deregister_table(ctx, database):

ctx.deregister_table("csv")
assert public.names() == {"csv1", "csv2"}

def test_register_dataset(ctx):
# create a RecordBatch and register it as a pyarrow.dataset.Dataset
batch = pa.RecordBatch.from_arrays(
[pa.array([1, 2, 3]), pa.array([4, 5, 6])],
names=["a", "b"],
)
dataset = ds.dataset([batch])
ctx.register_dataset("t", dataset)

assert ctx.tables() == {"t"}

result = ctx.sql("SELECT a+b, a-b FROM t").collect()
kylebrooks-8451 marked this conversation as resolved.
Show resolved Hide resolved

assert result[0].column(0) == pa.array([5, 7, 9])
assert result[0].column(1) == pa.array([-3, -3, -3])

def test_dataset_filter(ctx):
# create a RecordBatch and register it as a pyarrow.dataset.Dataset
batch = pa.RecordBatch.from_arrays(
[pa.array([1, 2, 3]), pa.array([4, 5, 6])],
names=["a", "b"],
)
dataset = ds.dataset([batch])
ctx.register_dataset("t", dataset)

assert ctx.tables() == {"t"}

result = ctx.sql("SELECT a+b, a-b FROM t WHERE a BETWEEN 2 and 3 AND b > 5").collect()

assert result[0].column(0) == pa.array([9])
assert result[0].column(1) == pa.array([-3])
12 changes: 12 additions & 0 deletions datafusion/tests/test_sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import numpy as np
import pyarrow as pa
import pyarrow.dataset as ds
import pytest

from datafusion import udf
Expand Down Expand Up @@ -121,6 +122,17 @@ def test_register_parquet_partitioned(ctx, tmp_path):
rd = result.to_pydict()
assert dict(zip(rd["grp"], rd["cnt"])) == {"a": 3, "b": 1}

def test_register_dataset(ctx, tmp_path):
path = helpers.write_parquet(tmp_path / "a.parquet", helpers.data())
dataset = ds.dataset(path, format="parquet")

ctx.register_dataset("t", dataset)
assert ctx.tables() == {"t"}

result = ctx.sql("SELECT COUNT(a) AS cnt FROM t").collect()
result = pa.Table.from_batches(result)
assert result.to_pydict() == {"cnt": [100]}


def test_execute(ctx, tmp_path):
data = [1, 1, 2, 2, 3, 11, 12]
Expand Down
13 changes: 13 additions & 0 deletions src/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,14 @@ use pyo3::prelude::*;

use datafusion::arrow::datatypes::Schema;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::datasource::datasource::TableProvider;
use datafusion::datasource::MemTable;
use datafusion::execution::context::SessionContext;
use datafusion::prelude::{CsvReadOptions, ParquetReadOptions};

use crate::catalog::{PyCatalog, PyTable};
use crate::dataframe::PyDataFrame;
use crate::dataset::Dataset;
use crate::errors::DataFusionError;
use crate::udf::PyScalarUDF;
use crate::utils::wait_for_future;
Expand Down Expand Up @@ -173,6 +175,17 @@ impl PySessionContext {
Ok(())
}

// Registers a PyArrow.Dataset
fn register_dataset(&self, name: &str, dataset: &PyAny, py: Python) -> PyResult<()> {
let table: Arc<dyn TableProvider> = Arc::new(Dataset::new(dataset, py)?);

self.ctx
.register_table(name, table)
.map_err(DataFusionError::from)?;

Ok(())
}

fn register_udf(&mut self, udf: PyScalarUDF) -> PyResult<()> {
self.ctx.register_udf(udf.function);
Ok(())
Expand Down
116 changes: 116 additions & 0 deletions src/dataset.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
// 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 pyo3::exceptions::PyValueError;
/// Implements a Datafusion TableProvider that delegates to a PyArrow Dataset
/// This allows us to use PyArrow Datasets as Datafusion tables while pushing down projections and filters
use pyo3::prelude::*;
use pyo3::types::PyType;

use std::any::Any;
use std::sync::Arc;

use async_trait::async_trait;

use datafusion::arrow::datatypes::SchemaRef;
use datafusion::datasource::datasource::TableProviderFilterPushDown;
use datafusion::datasource::{TableProvider, TableType};
use datafusion::error::{DataFusionError, Result as DFResult};
use datafusion::logical_plan::*;
use datafusion::physical_plan::ExecutionPlan;

use crate::dataset_exec::DatasetExec;
use crate::pyarrow_filter_expression::PyArrowFilterExpression;

// Wraps a pyarrow.dataset.Dataset class and implements a Datafusion TableProvider around it
#[derive(Debug, Clone)]
pub(crate) struct Dataset {
dataset: PyObject,
}

impl Dataset {
// Creates a Python PyArrow.Dataset
pub fn new(dataset: &PyAny, py: Python) -> PyResult<Self> {
// Ensure that we were passed an instance of pyarrow.dataset.Dataset
let ds = PyModule::import(py, "pyarrow.dataset")?;
let ds_type: &PyType = ds.getattr("Dataset")?.downcast()?;
if dataset.is_instance(ds_type)? {
Ok(Dataset {
dataset: dataset.into(),
})
} else {
Err(PyValueError::new_err(
"dataset argument must be a pyarrow.dataset.Dataset object",
))
}
}
}

#[async_trait]
impl TableProvider for Dataset {
/// Returns the table provider as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any {
self
}

/// Get a reference to the schema for this table
fn schema(&self) -> SchemaRef {
Python::with_gil(|py| {
let dataset = self.dataset.as_ref(py);
// This can panic but since we checked that self.dataset is a pyarrow.dataset.Dataset it should never
Arc::new(dataset.getattr("schema").unwrap().extract().unwrap())
})
}

/// Get the type of this table for metadata/catalog purposes.
fn table_type(&self) -> TableType {
TableType::Base
}

/// Create an ExecutionPlan that will scan the table.
/// The table provider will be usually responsible of grouping
/// the source data into partitions that can be efficiently
/// parallelized or distributed.
async fn scan(
&self,
projection: &Option<Vec<usize>>,
filters: &[Expr],
// limit can be used to reduce the amount scanned
// from the datasource as a performance optimization.
// If set, it contains the amount of rows needed by the `LogicalPlan`,
// The datasource should return *at least* this number of rows if available.
_limit: Option<usize>,
) -> DFResult<Arc<dyn ExecutionPlan>> {
Python::with_gil(|py| {
let plan: Arc<dyn ExecutionPlan> = Arc::new(
DatasetExec::new(py, self.dataset.as_ref(py), projection.clone(), filters)
.map_err(|err| DataFusionError::External(Box::new(err)))?,
);
Ok(plan)
})
}

/// Tests whether the table provider can make use of a filter expression
/// to optimise data retrieval.
fn supports_filter_pushdown(&self, filter: &Expr) -> DFResult<TableProviderFilterPushDown> {
match PyArrowFilterExpression::try_from(filter) {
Ok(_) => Ok(TableProviderFilterPushDown::Exact),
_ => Ok(TableProviderFilterPushDown::Unsupported),
}
}
}
Loading