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

chore: set validation and type hint for ffi tableprovider #983

Open
wants to merge 1 commit 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
13 changes: 12 additions & 1 deletion python/datafusion/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,15 @@ def __arrow_c_array__( # noqa: D105
) -> tuple[object, object]: ...


class TableProviderExportable(Protocol):
"""Type hint for object that has __datafusion_table_provider__ PyCapsule.

https://datafusion.apache.org/python/user-guide/io/table_provider.html
"""

def __datafusion_table_provider__(self) -> object: ... # noqa: D105


class SessionConfig:
"""Session configuration options."""

Expand Down Expand Up @@ -685,7 +694,9 @@ def deregister_table(self, name: str) -> None:
"""Remove a table from the session."""
self.ctx.deregister_table(name)

def register_table_provider(self, name: str, provider: Any) -> None:
def register_table_provider(
self, name: str, provider: TableProviderExportable
) -> None:
"""Register a table provider.

This table provider must have a method called ``__datafusion_table_provider__``
Expand Down
4 changes: 2 additions & 2 deletions src/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ use crate::store::StorageContexts;
use crate::udaf::PyAggregateUDF;
use crate::udf::PyScalarUDF;
use crate::udwf::PyWindowUDF;
use crate::utils::{get_tokio_runtime, wait_for_future};
use crate::utils::{get_tokio_runtime, validate_pycapsule, wait_for_future};
use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef};
use datafusion::arrow::pyarrow::PyArrowType;
use datafusion::arrow::record_batch::RecordBatch;
Expand Down Expand Up @@ -576,7 +576,7 @@ impl PySessionContext {
if provider.hasattr("__datafusion_table_provider__")? {
let capsule = provider.getattr("__datafusion_table_provider__")?.call0()?;
let capsule = capsule.downcast::<PyCapsule>()?;
// validate_pycapsule(capsule, "arrow_array_stream")?;
validate_pycapsule(capsule, "datafusion_table_provider")?;

let provider = unsafe { capsule.reference::<FFI_TableProvider>() };
let provider: ForeignTableProvider = provider.into();
Expand Down
21 changes: 1 addition & 20 deletions src/dataframe.rs
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ use crate::expr::sort_expr::to_sort_expressions;
use crate::physical_plan::PyExecutionPlan;
use crate::record_batch::PyRecordBatchStream;
use crate::sql::logical::PyLogicalPlan;
use crate::utils::{get_tokio_runtime, wait_for_future};
use crate::utils::{get_tokio_runtime, validate_pycapsule, wait_for_future};
use crate::{
errors::DataFusionError,
expr::{sort_expr::PySortExpr, PyExpr},
Expand Down Expand Up @@ -724,22 +724,3 @@ fn record_batch_into_schema(

RecordBatch::try_new(schema, data_arrays)
}

fn validate_pycapsule(capsule: &Bound<PyCapsule>, name: &str) -> PyResult<()> {
let capsule_name = capsule.name()?;
if capsule_name.is_none() {
return Err(PyValueError::new_err(
"Expected schema PyCapsule to have name set.",
));
}

let capsule_name = capsule_name.unwrap().to_str()?;
if capsule_name != name {
return Err(PyValueError::new_err(format!(
"Expected name '{}' in PyCapsule, instead got '{}'",
name, capsule_name
)));
}

Ok(())
}
21 changes: 21 additions & 0 deletions src/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@
use crate::errors::DataFusionError;
use crate::TokioRuntime;
use datafusion::logical_expr::Volatility;
use pyo3::exceptions::PyValueError;
use pyo3::prelude::*;
use pyo3::types::PyCapsule;
use std::future::Future;
use std::sync::OnceLock;
use tokio::runtime::Runtime;
Expand Down Expand Up @@ -58,3 +60,22 @@ pub(crate) fn parse_volatility(value: &str) -> Result<Volatility, DataFusionErro
}
})
}

pub(crate) fn validate_pycapsule(capsule: &Bound<PyCapsule>, name: &str) -> PyResult<()> {
let capsule_name = capsule.name()?;
if capsule_name.is_none() {
return Err(PyValueError::new_err(
"Expected schema PyCapsule to have name set.",
));
}

let capsule_name = capsule_name.unwrap().to_str()?;
if capsule_name != name {
return Err(PyValueError::new_err(format!(
"Expected name '{}' in PyCapsule, instead got '{}'",
name, capsule_name
)));
}

Ok(())
}
Loading