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 1 commit
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
17 changes: 17 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,19 @@ 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])
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
115 changes: 115 additions & 0 deletions src/dataset.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
// 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()?;
match dataset.is_instance(ds_type)? {
true => Ok(Dataset {
kylebrooks-8451 marked this conversation as resolved.
Show resolved Hide resolved
dataset: dataset.into(),
}),
false => 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