From 51cb939785b10bfe749294f9834031b8f721ed35 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 19 Oct 2025 08:57:24 -0400 Subject: [PATCH 01/69] Starting to flush out physical expr --- datafusion/ffi/src/expr/columnar_value.rs | 20 ++ datafusion/ffi/src/expr/distribution.rs | 20 ++ datafusion/ffi/src/expr/expr_properties.rs | 20 ++ datafusion/ffi/src/expr/interval.rs | 20 ++ datafusion/ffi/src/expr/mod.rs | 4 + datafusion/ffi/src/lib.rs | 2 + datafusion/ffi/src/physical_expr/mod.rs | 261 +++++++++++++++++++++ datafusion/ffi/src/util.rs | 4 +- 8 files changed, 350 insertions(+), 1 deletion(-) create mode 100644 datafusion/ffi/src/expr/columnar_value.rs create mode 100644 datafusion/ffi/src/expr/distribution.rs create mode 100644 datafusion/ffi/src/expr/expr_properties.rs create mode 100644 datafusion/ffi/src/expr/interval.rs create mode 100644 datafusion/ffi/src/expr/mod.rs create mode 100644 datafusion/ffi/src/physical_expr/mod.rs diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs new file mode 100644 index 000000000000..f05b525a4f25 --- /dev/null +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -0,0 +1,20 @@ +use abi_stable::StableAbi; +use datafusion_expr::ColumnarValue; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_ColumnarValue { +} + +impl From for FFI_ColumnarValue { + fn from(value: ColumnarValue) -> Self { + todo!() + } +} + +impl From for ColumnarValue { + fn from(value: FFI_ColumnarValue) -> Self { + todo!() + } +} \ No newline at end of file diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs new file mode 100644 index 000000000000..d92255c6dc38 --- /dev/null +++ b/datafusion/ffi/src/expr/distribution.rs @@ -0,0 +1,20 @@ +use abi_stable::StableAbi; +use datafusion_expr::statistics::Distribution; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_Distribution { +} + +impl From for FFI_Distribution { + fn from(value: Distribution) -> Self { + todo!() + } +} + +impl From for Distribution { + fn from(value: FFI_Distribution) -> Self { + todo!() + } +} \ No newline at end of file diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs new file mode 100644 index 000000000000..f8133a065b05 --- /dev/null +++ b/datafusion/ffi/src/expr/expr_properties.rs @@ -0,0 +1,20 @@ +use abi_stable::StableAbi; +use datafusion_expr::sort_properties::ExprProperties; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_ExprProperties { +} + +impl From for FFI_ExprProperties { + fn from(value: ExprProperties) -> Self { + todo!() + } +} + +impl From for ExprProperties { + fn from(value: FFI_ExprProperties) -> Self { + todo!() + } +} \ No newline at end of file diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs new file mode 100644 index 000000000000..496482bff449 --- /dev/null +++ b/datafusion/ffi/src/expr/interval.rs @@ -0,0 +1,20 @@ +use abi_stable::StableAbi; +use datafusion_expr::interval_arithmetic::Interval; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_Interval { +} + +impl From for FFI_Interval { + fn from(value: Interval) -> Self { + todo!() + } +} + +impl From for Interval { + fn from(value: FFI_Interval) -> Self { + todo!() + } +} \ No newline at end of file diff --git a/datafusion/ffi/src/expr/mod.rs b/datafusion/ffi/src/expr/mod.rs new file mode 100644 index 000000000000..d2dcee31bfef --- /dev/null +++ b/datafusion/ffi/src/expr/mod.rs @@ -0,0 +1,4 @@ +pub mod columnar_value; +pub mod distribution; +pub mod expr_properties; +pub mod interval; diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index 0c2340e8ce7b..82afc4d2e456 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -27,7 +27,9 @@ pub mod arrow_wrappers; pub mod catalog_provider; pub mod execution_plan; +pub mod expr; pub mod insert_op; +pub mod physical_expr; pub mod plan_properties; pub mod record_batch_stream; pub mod schema_provider; diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs new file mode 100644 index 000000000000..d4b48c79a891 --- /dev/null +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -0,0 +1,261 @@ +// 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 std::{any::Any, ffi::c_void, sync::Arc}; +use std::fmt::Formatter; +use abi_stable::{ + std_types::{ROption, FFIResult, RString, RVec}, + StableAbi, +}; +use abi_stable::pmr::RSlice; +use arrow::array::{BooleanArray, RecordBatch}; +use arrow::datatypes::SchemaRef; +use arrow::ffi::FFI_ArrowArray; +use arrow_schema::{DataType, FieldRef, Schema}; +use async_ffi::{FfiFuture, FutureExt}; +use async_trait::async_trait; +use datafusion_proto::{ + logical_plan::{ + from_proto::parse_exprs, to_proto::serialize_exprs, DefaultLogicalExtensionCodec, + }, + protobuf::LogicalExprList, +}; +use prost::Message; +use tokio::runtime::Handle; +use datafusion::logical_expr::ColumnarValue; +use datafusion::logical_expr_common::interval_arithmetic::Interval; +use datafusion::logical_expr_common::sort_properties::ExprProperties; +use datafusion::logical_expr_common::statistics::Distribution; +use crate::session::{FFI_Session, ForeignSession}; +use crate::{ + arrow_wrappers::WrappedSchema, + df_result, + execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}, + insert_op::FFI_InsertOp, + rresult_return, + table_source::FFI_TableType, +}; +use datafusion_catalog::Session; +use datafusion_common::DataFusionError; +use datafusion_common::Result; +use datafusion_execution::TaskContext; +use datafusion_expr::dml::InsertOp; +use datafusion_expr::registry::MemoryFunctionRegistry; +use datafusion_expr::{Expr, TableType}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::ExecutionPlan; +use crate::arrow_wrappers::WrappedArray; +use crate::expr::columnar_value::FFI_ColumnarValue; +use crate::expr::distribution::FFI_Distribution; +use crate::expr::expr_properties::FFI_ExprProperties; +use crate::expr::interval::FFI_Interval; +use crate::table_provider::FFI_TableProvider; +use crate::util::FFIResult; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_PhysicalExpr { + pub data_type: unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, + + pub nullable: unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, + + pub evaluate: unsafe extern "C" fn(&Self, batch: WrappedArray) -> FFIResult, + + pub return_field: unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, + + pub aggregate: unsafe extern "C" fn(&Self, batch: WrappedArray, selection: FFI_ArrowArray) -> FFIResult, + + pub children: unsafe extern "C" fn(&Self) -> FFIResult>, + + pub new_with_children: unsafe extern "C" fn(&Self, children: RVec) -> FFIResult, + + pub evalutate_bounds: unsafe extern "C" fn(&Self, children: RSlice) -> FFIResult, + + pub propagate_constraints: unsafe extern "C" fn(&Self, interval: &FFI_Interval, children: RSlice) -> FFIResult>>, + + pub evaluate_statistics: unsafe extern "C" fn(&Self, children: &[&FFI_Distribution]) -> FFIResult, + + pub propagate_statistics: unsafe extern "C" fn(&Self, parent: &FFI_Distribution, children: &[&FFI_Distribution]) -> FFIResult>>, + + pub get_properties: unsafe extern "C" fn(&Self, children: &[FFI_ExprProperties]) -> FFIResult, + + pub fmt_sql: unsafe extern "C" fn(&Self, f: &mut Formatter<'_>) -> FFIResult, + + pub snapshot: unsafe extern "C" fn(&Self) -> FFIResult>, + + pub snapshot_generation: unsafe extern "C" fn(&Self) -> u64, + + pub is_volatile_node: unsafe extern "C" fn(&Self) -> bool, + + /// Used to create a clone on the provider of the execution plan. This should + /// only need to be called by the receiver of the plan. + pub clone: unsafe extern "C" fn(plan: &Self) -> Self, + + /// Release the memory of the private data when it is no longer being used. + pub release: unsafe extern "C" fn(arg: &mut Self), + + /// Return the major DataFusion version number of this provider. + pub version: unsafe extern "C" fn() -> u64, + + /// Internal data. This is only to be accessed by the provider of the plan. + /// A [`ForeignExecutionPlan`] should never attempt to access this data. + pub private_data: *mut c_void, +} + +unsafe impl Send for FFI_PhysicalExpr {} +unsafe impl Sync for FFI_PhysicalExpr {} + +struct PhysicalExprPrivateData { + expr: Arc, +} + + +unsafe extern "C" fn release_fn_wrapper(expr: &mut FFI_PhysicalExpr) { + let private_data = Box::from_raw(expr.private_data as *mut PhysicalExprPrivateData); + drop(private_data); +} + +unsafe extern "C" fn clone_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFI_PhysicalExpr { + let old_private_data = expr.private_data as *const PhysicalExprPrivateData; + + let private_data = Box::into_raw(Box::new(PhysicalExprPrivateData { + expr: Arc::clone(&(*old_private_data).expr), + })) as *mut c_void; + + FFI_PhysicalExpr { + clone: clone_fn_wrapper, + release: release_fn_wrapper, + version: super::version, + private_data, + } +} + +impl Drop for FFI_PhysicalExpr { + fn drop(&mut self) { + unsafe { (self.release)(self) } + } +} + +impl FFI_PhysicalExpr { + /// Creates a new [`FFI_PhysicalExpr`]. + pub fn new( + expr: Arc, + ) -> Self { + let private_data = Box::new(PhysicalExprPrivateData { expr }); + + Self { + + clone: clone_fn_wrapper, + release: release_fn_wrapper, + version: super::version, + private_data: Box::into_raw(private_data) as *mut c_void, + } + } +} + +/// This wrapper struct exists on the receiver side of the FFI interface, so it has +/// no guarantees about being able to access the data in `private_data`. Any functions +/// defined on this struct must only use the stable functions provided in +/// FFI_PhysicalExpr to interact with the foreign table provider. +#[derive(Debug)] +pub struct ForeignPhysicalExpr(pub FFI_PhysicalExpr); + +unsafe impl Send for ForeignPhysicalExpr {} +unsafe impl Sync for ForeignPhysicalExpr {} + +impl From<&FFI_PhysicalExpr> for ForeignPhysicalExpr { + fn from(provider: &FFI_PhysicalExpr) -> Self { + Self(provider.clone()) + } +} + +impl Clone for FFI_PhysicalExpr { + fn clone(&self) -> Self { + unsafe { (self.clone)(self) } + } +} + +impl PhysicalExpr for ForeignPhysicalExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> Result { + todo!() + } + + fn nullable(&self, input_schema: &Schema) -> Result { + todo!() + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + todo!() + } + + fn return_field(&self, input_schema: &Schema) -> Result { + todo!() + } + + fn evaluate_selection(&self, batch: &RecordBatch, selection: &BooleanArray) -> Result { + todo!() + } + + fn children(&self) -> Vec<&Arc> { + todo!() + } + + fn with_new_children(self: Arc, children: Vec>) -> Result> { + todo!() + } + + fn evaluate_bounds(&self, _children: &[&Interval]) -> Result { + todo!() + } + + fn propagate_constraints(&self, _interval: &Interval, _children: &[&Interval]) -> Result>> { + todo!() + } + + fn evaluate_statistics(&self, children: &[&Distribution]) -> Result { + todo!() + } + + fn propagate_statistics(&self, parent: &Distribution, children: &[&Distribution]) -> Result>> { + todo!() + } + + fn get_properties(&self, _children: &[ExprProperties]) -> Result { + todo!() + } + + fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + todo!() + } + + fn snapshot(&self) -> Result>> { + todo!() + } + + fn snapshot_generation(&self) -> u64 { + todo!() + } + + fn is_volatile_node(&self) -> bool { + todo!() + } +} diff --git a/datafusion/ffi/src/util.rs b/datafusion/ffi/src/util.rs index 151464dc9745..9809544bdb65 100644 --- a/datafusion/ffi/src/util.rs +++ b/datafusion/ffi/src/util.rs @@ -16,12 +16,14 @@ // under the License. use crate::arrow_wrappers::WrappedSchema; -use abi_stable::std_types::RVec; +use abi_stable::std_types::{RResult, RString, RVec}; use arrow::datatypes::Field; use arrow::{datatypes::DataType, ffi::FFI_ArrowSchema}; use arrow_schema::FieldRef; use std::sync::Arc; +pub type FFIResult = RResult; + /// This macro is a helpful conversion utility to convert from an abi_stable::RResult to a /// DataFusion result. #[macro_export] From d61a1ed19c45e5a507ec318c05aecba735e47a6a Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 19 Oct 2025 10:37:46 -0400 Subject: [PATCH 02/69] Intermediate work flushing out physicalexpr ffi --- datafusion/ffi/src/physical_expr/mod.rs | 458 ++++++++++++++++++---- datafusion/ffi/src/record_batch_stream.rs | 4 +- 2 files changed, 375 insertions(+), 87 deletions(-) diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index d4b48c79a891..7e98b7c5777e 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -15,19 +15,45 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, ffi::c_void, sync::Arc}; -use std::fmt::Formatter; +use crate::arrow_wrappers::WrappedArray; +use crate::execution_plan::ExecutionPlanPrivateData; +use crate::expr::columnar_value::FFI_ColumnarValue; +use crate::expr::distribution::FFI_Distribution; +use crate::expr::expr_properties::FFI_ExprProperties; +use crate::expr::interval::FFI_Interval; +use crate::record_batch_stream::{ + record_batch_to_wrapped_array, wrapped_array_to_record_batch, +}; +use crate::table_provider::FFI_TableProvider; +use crate::util::FFIResult; +use crate::{ + arrow_wrappers::WrappedSchema, + df_result, + execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}, + insert_op::FFI_InsertOp, + rresult, rresult_return, + table_source::FFI_TableType, +}; +use abi_stable::pmr::RSlice; +use abi_stable::std_types::RResult; use abi_stable::{ - std_types::{ROption, FFIResult, RString, RVec}, + std_types::{ROption, RString, RVec}, StableAbi, }; -use abi_stable::pmr::RSlice; -use arrow::array::{BooleanArray, RecordBatch}; +use arrow::array::{ArrayRef, BooleanArray, RecordBatch}; use arrow::datatypes::SchemaRef; use arrow::ffi::FFI_ArrowArray; use arrow_schema::{DataType, FieldRef, Schema}; use async_ffi::{FfiFuture, FutureExt}; use async_trait::async_trait; +use datafusion::logical_expr::ColumnarValue; +use datafusion::logical_expr_common::interval_arithmetic::Interval; +use datafusion::logical_expr_common::sort_properties::ExprProperties; +use datafusion::logical_expr_common::statistics::Distribution; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_expr_common::physical_expr::fmt_sql; +use datafusion_common::DataFusionError; +use datafusion_common::Result; use datafusion_proto::{ logical_plan::{ from_proto::parse_exprs, to_proto::serialize_exprs, DefaultLogicalExtensionCodec, @@ -35,66 +61,71 @@ use datafusion_proto::{ protobuf::LogicalExprList, }; use prost::Message; +use std::fmt::Formatter; +use std::{any::Any, ffi::c_void, sync::Arc}; use tokio::runtime::Handle; -use datafusion::logical_expr::ColumnarValue; -use datafusion::logical_expr_common::interval_arithmetic::Interval; -use datafusion::logical_expr_common::sort_properties::ExprProperties; -use datafusion::logical_expr_common::statistics::Distribution; -use crate::session::{FFI_Session, ForeignSession}; -use crate::{ - arrow_wrappers::WrappedSchema, - df_result, - execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}, - insert_op::FFI_InsertOp, - rresult_return, - table_source::FFI_TableType, -}; -use datafusion_catalog::Session; -use datafusion_common::DataFusionError; -use datafusion_common::Result; -use datafusion_execution::TaskContext; -use datafusion_expr::dml::InsertOp; -use datafusion_expr::registry::MemoryFunctionRegistry; -use datafusion_expr::{Expr, TableType}; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::ExecutionPlan; -use crate::arrow_wrappers::WrappedArray; -use crate::expr::columnar_value::FFI_ColumnarValue; -use crate::expr::distribution::FFI_Distribution; -use crate::expr::expr_properties::FFI_ExprProperties; -use crate::expr::interval::FFI_Interval; -use crate::table_provider::FFI_TableProvider; -use crate::util::FFIResult; #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] pub struct FFI_PhysicalExpr { - pub data_type: unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, - - pub nullable: unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, - - pub evaluate: unsafe extern "C" fn(&Self, batch: WrappedArray) -> FFIResult, - - pub return_field: unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, - - pub aggregate: unsafe extern "C" fn(&Self, batch: WrappedArray, selection: FFI_ArrowArray) -> FFIResult, - - pub children: unsafe extern "C" fn(&Self) -> FFIResult>, - - pub new_with_children: unsafe extern "C" fn(&Self, children: RVec) -> FFIResult, - - pub evalutate_bounds: unsafe extern "C" fn(&Self, children: RSlice) -> FFIResult, - - pub propagate_constraints: unsafe extern "C" fn(&Self, interval: &FFI_Interval, children: RSlice) -> FFIResult>>, - - pub evaluate_statistics: unsafe extern "C" fn(&Self, children: &[&FFI_Distribution]) -> FFIResult, - - pub propagate_statistics: unsafe extern "C" fn(&Self, parent: &FFI_Distribution, children: &[&FFI_Distribution]) -> FFIResult>>, - - pub get_properties: unsafe extern "C" fn(&Self, children: &[FFI_ExprProperties]) -> FFIResult, - - pub fmt_sql: unsafe extern "C" fn(&Self, f: &mut Formatter<'_>) -> FFIResult, + pub data_type: unsafe extern "C" fn( + &Self, + input_schema: WrappedSchema, + ) -> FFIResult, + + pub nullable: + unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, + + pub evaluate: + unsafe extern "C" fn(&Self, batch: WrappedArray) -> FFIResult, + + pub return_field: unsafe extern "C" fn( + &Self, + input_schema: WrappedSchema, + ) -> FFIResult, + + pub evaluate_selection: unsafe extern "C" fn( + &Self, + batch: WrappedArray, + selection: FFI_ArrowArray, + ) -> FFIResult, + + pub children: unsafe extern "C" fn(&Self) -> RVec, + + pub new_with_children: + unsafe extern "C" fn(&Self, children: RVec) -> FFIResult, + + pub evalutate_bounds: unsafe extern "C" fn( + &Self, + children: RSlice, + ) -> FFIResult, + + pub propagate_constraints: + unsafe extern "C" fn( + &Self, + interval: &FFI_Interval, + children: RSlice, + ) -> FFIResult>>, + + pub evaluate_statistics: unsafe extern "C" fn( + &Self, + children: &[&FFI_Distribution], + ) -> FFIResult, + + pub propagate_statistics: + unsafe extern "C" fn( + &Self, + parent: &FFI_Distribution, + children: &[&FFI_Distribution], + ) -> FFIResult>>, + + pub get_properties: unsafe extern "C" fn( + &Self, + children: &[FFI_ExprProperties], + ) -> FFIResult, + + pub fmt_sql: unsafe extern "C" fn(&Self) -> FFIResult, pub snapshot: unsafe extern "C" fn(&Self) -> FFIResult>, @@ -120,10 +151,171 @@ pub struct FFI_PhysicalExpr { unsafe impl Send for FFI_PhysicalExpr {} unsafe impl Sync for FFI_PhysicalExpr {} +impl FFI_PhysicalExpr { + fn inner(&self) -> &Arc { + unsafe { + let private_data = self.private_data as *const PhysicalExprPrivateData; + &(*private_data).expr + } + } +} + struct PhysicalExprPrivateData { expr: Arc, } +unsafe extern "C" fn data_type_fn_wrapper( + expr: &FFI_PhysicalExpr, + input_schema: WrappedSchema, +) -> FFIResult { + let expr = expr.inner(); + let schema = input_schema.into(); + let data_type = expr.data_type(schema); + rresult!(data_type.map(|dt| dt.into())) +} + +unsafe extern "C" fn nullable_fn_wrapper( + expr: &FFI_PhysicalExpr, + input_schema: WrappedSchema, +) -> FFIResult { + let expr = expr.inner(); + let schema = input_schema.into(); + rresult!(expr.nullable(schema)) +} + +unsafe extern "C" fn evaluate_fn_wrapper( + expr: &FFI_PhysicalExpr, + batch: WrappedArray, +) -> FFIResult { + let batch = rresult_return!(wrapped_array_to_record_batch(batch)); + let value = rresult_return!(expr.inner().evaluate(&batch)); + RResult::ROk(value.into()) +} + +unsafe extern "C" fn return_field_fn_wrapper( + expr: &FFI_PhysicalExpr, + input_schema: WrappedSchema, +) -> FFIResult { + let expr = expr.inner(); + let schema = input_schema.into(); + rresult_return!(expr.return_field(schema).map(|field| field.into())) +} + +unsafe extern "C" fn evaluate_selection_fn_wrapper( + expr: &FFI_PhysicalExpr, + batch: WrappedArray, + selection: FFI_ArrowArray, +) -> FFIResult { + let batch = rresult_return!(wrapped_array_to_record_batch(batch)); + let selection: ArrayRef = selection.into(); + let selection = rresult_return!(selection.as_any().downcast_ref::()); + let value = rresult_return!(expr.inner().evaluate_selection(&batch, selection)); + RResult::ROk(value.into()) +} + +unsafe extern "C" fn children_fn_wrapper( + expr: &FFI_PhysicalExpr, +) -> RVec { + let expr = expr.inner(); + let children = expr.children(); + children.into_iter().map(|child| child.into()).collect() +} + +unsafe extern "C" fn new_with_children_fn_wrapper( + expr: &FFI_PhysicalExpr, + children: RVec, +) -> FFIResult { + let expr = expr.inner(); + let children = children.into_iter().map(|child| child.into()).collect(); + rresult!(expr.new_with_children(children).map(|child| child.into())) +} + +unsafe extern "C" fn evalutate_bounds_fn_wrapper( + expr: &FFI_PhysicalExpr, + children: RSlice, +) -> FFIResult { + let expr = expr.inner(); + let children = children + .iter() + .map(|child| child.into()) + .collect::>(); + + rresult!(expr.evaluate_bounds(&children)) +} + +unsafe extern "C" fn propagate_constraints_fn_wrapper( + expr: &FFI_PhysicalExpr, + interval: &FFI_Interval, + children: RSlice, +) -> FFIResult>> { + let expr = expr.inner(); + let interval = interval.into(); + let children = children + .iter() + .map(|child| child.into()) + .collect::>(); + + rresult!(expr.propagate_constraints(&interval, &children).into()) +} + +unsafe extern "C" fn evaluate_statistics_fn_wrapper( + expr: &FFI_PhysicalExpr, + children: &[&FFI_Distribution], +) -> FFIResult { + let expr = expr.inner(); + let children = children + .iter() + .map(|child| child.into()) + .collect::>(); + rresult!(expr.evaluate_statistics(&children).into()) +} + +unsafe extern "C" fn propagate_statistics_fn_wrapper( + expr: &FFI_PhysicalExpr, + parent: &FFI_Distribution, + children: &[&FFI_Distribution], +) -> FFIResult>> { + let expr = expr.inner(); + let parent = parent.into(); + let children = children + .iter() + .map(|child| child.into()) + .collect::>(); + + rresult!(expr.propagate_statistics(&parent, &children).into()) +} + +unsafe extern "C" fn get_properties_fn_wrapper( + expr: &FFI_PhysicalExpr, + children: &[FFI_ExprProperties], +) -> FFIResult { + let expr = expr.inner(); + let children = children.iter().map(|child| child.into()).collect(); + rresult!(expr.get_properties(&children).into()) +} + +unsafe extern "C" fn fmt_sql_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFIResult { + let expr = expr.inner(); + let result = fmt_sql(expr).to_string(); + RResult::ROk(result.into()) +} + +unsafe extern "C" fn snapshot_fn_wrapper( + expr: &FFI_PhysicalExpr, +) -> FFIResult> { + let expr = expr.inner(); + rresult!(expr.snapshot().map(|snapshot| snapshot.into())) +} + +unsafe extern "C" fn snapshot_generation_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 { + let expr = expr.inner(); + expr.snapshot_generation() +} + +unsafe extern "C" fn is_volatile_node_fn_wrapper(expr: &FFI_PhysicalExpr) -> bool { + let expr = expr.inner(); + expr.is_volatile_node() +} unsafe extern "C" fn release_fn_wrapper(expr: &mut FFI_PhysicalExpr) { let private_data = Box::from_raw(expr.private_data as *mut PhysicalExprPrivateData); @@ -138,6 +330,22 @@ unsafe extern "C" fn clone_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFI_PhysicalEx })) as *mut c_void; FFI_PhysicalExpr { + data_type: data_type_fn_wrapper, + nullable: nullable_fn_wrapper, + evaluate: evaluate_fn_wrapper, + return_field: return_field_fn_wrapper, + evaluate_selection: evaluate_selection_fn_wrapper, + children: children_fn_wrapper, + new_with_children: new_with_children_fn_wrapper, + evalutate_bounds: evalutate_bounds_fn_wrapper, + propagate_constraints: propagate_constraints_fn_wrapper, + evaluate_statistics: evaluate_statistics_fn_wrapper, + propagate_statistics: propagate_statistics_fn_wrapper, + get_properties: get_properties_fn_wrapper, + fmt_sql: fmt_sql_fn_wrapper, + snapshot: snapshot_fn_wrapper, + snapshot_generation: snapshot_generation_fn_wrapper, + is_volatile_node: is_volatile_node_fn_wrapper, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -153,13 +361,26 @@ impl Drop for FFI_PhysicalExpr { impl FFI_PhysicalExpr { /// Creates a new [`FFI_PhysicalExpr`]. - pub fn new( - expr: Arc, - ) -> Self { + pub fn new(expr: Arc) -> Self { let private_data = Box::new(PhysicalExprPrivateData { expr }); Self { - + data_type: data_type_fn_wrapper, + nullable: nullable_fn_wrapper, + evaluate: evaluate_fn_wrapper, + return_field: return_field_fn_wrapper, + evaluate_selection: evaluate_selection_fn_wrapper, + children: children_fn_wrapper, + new_with_children: new_with_children_fn_wrapper, + evalutate_bounds: evalutate_bounds_fn_wrapper, + propagate_constraints: propagate_constraints_fn_wrapper, + evaluate_statistics: evaluate_statistics_fn_wrapper, + propagate_statistics: propagate_statistics_fn_wrapper, + get_properties: get_properties_fn_wrapper, + fmt_sql: fmt_sql_fn_wrapper, + snapshot: snapshot_fn_wrapper, + snapshot_generation: snapshot_generation_fn_wrapper, + is_volatile_node: is_volatile_node_fn_wrapper, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -196,66 +417,133 @@ impl PhysicalExpr for ForeignPhysicalExpr { } fn data_type(&self, input_schema: &Schema) -> Result { - todo!() + unsafe { + let schema = input_schema.into(); + df_result!((self.0.data_type)(&self.0, schema).into()) + } } fn nullable(&self, input_schema: &Schema) -> Result { - todo!() + unsafe { + let schema = input_schema.into(); + df_result!((self.0.nullable)(&self.0, schema)) + } } fn evaluate(&self, batch: &RecordBatch) -> Result { - todo!() + unsafe { + let batch = record_batch_to_wrapped_array(batch.clone())?; + df_result!((self.0.evaluate)(&self.0, batch).map(|v| v.into())) + } } fn return_field(&self, input_schema: &Schema) -> Result { - todo!() + unsafe { + let schema = input_schema.into(); + df_result!((self.0.return_field)(&self.0, schema).map(|f| f.into())) + } } - fn evaluate_selection(&self, batch: &RecordBatch, selection: &BooleanArray) -> Result { - todo!() + fn evaluate_selection( + &self, + batch: &RecordBatch, + selection: &BooleanArray, + ) -> Result { + unsafe { + let batch = record_batch_to_wrapped_array(batch)?; + let selection = selection.into(); + df_result!( + (self.0.evaluate_selection)(&self.0, batch, selection).map(|f| f.into()) + ) + } } fn children(&self) -> Vec<&Arc> { - todo!() + unsafe { + (self.0.children)(&self.0) + .into_iter() + .map(Into::into) + .collect() + } } - fn with_new_children(self: Arc, children: Vec>) -> Result> { - todo!() + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + unsafe { + let children = children.into_iter().map(Into::into).collect(); + df_result!((self.0.new_with_children)(&self.0, children).map(Into::into)) + } } - fn evaluate_bounds(&self, _children: &[&Interval]) -> Result { - todo!() + fn evaluate_bounds(&self, children: &[&Interval]) -> Result { + unsafe { + let children = children.iter().map(Into::into).collect(); + df_result!((self.0.evalutate_bounds)(&self.0, children)) + } } - fn propagate_constraints(&self, _interval: &Interval, _children: &[&Interval]) -> Result>> { - todo!() + fn propagate_constraints( + &self, + interval: &Interval, + children: &[&Interval], + ) -> Result>> { + unsafe { + let interval = interval.into(); + let children = children.iter().map(Into::into).collect(); + df_result!((self.0.propagate_constraints)(&self.0, interval, children) + .map(Into::into) + .into()) + } } fn evaluate_statistics(&self, children: &[&Distribution]) -> Result { - todo!() + unsafe { + let children = children.iter().map(Into::into).collect::>(); + df_result!((self.0.evaluate_statistics)(&self.0, &children).map(Into::into)) + } } - fn propagate_statistics(&self, parent: &Distribution, children: &[&Distribution]) -> Result>> { - todo!() + fn propagate_statistics( + &self, + parent: &Distribution, + children: &[&Distribution], + ) -> Result>> { + unsafe { + let parent = parent.into(); + let children = children.iter().map(Into::into).collect(); + df_result!( + (self.0.propagate_statistics)(&self.0, parent, children).map(Into::into) + ) + } } - fn get_properties(&self, _children: &[ExprProperties]) -> Result { - todo!() + fn get_properties(&self, children: &[ExprProperties]) -> Result { + unsafe { + let children = children.iter().map(Into::into).collect(); + df_result!((self.0.get_properties)(&self.0, children).map(Into::into)) + } } fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - todo!() + unsafe { + match (self.0.fmt_sql)(&self.0) { + RResult::ROk(sql) => write!(f, "{sql}"), + RResult::RErr(_) => Err(std::fmt::Error), + } + } } fn snapshot(&self) -> Result>> { - todo!() + unsafe { df_result!((self.0.snapshot)(&self.0).map(Into::into)) } } fn snapshot_generation(&self) -> u64 { - todo!() + unsafe { (self.0.snapshot_generation)(&self.0) } } fn is_volatile_node(&self) -> bool { - todo!() + unsafe { (self.0.is_volatile_node)(&self.0) } } } diff --git a/datafusion/ffi/src/record_batch_stream.rs b/datafusion/ffi/src/record_batch_stream.rs index 1739235d1703..046c61cea36b 100644 --- a/datafusion/ffi/src/record_batch_stream.rs +++ b/datafusion/ffi/src/record_batch_stream.rs @@ -107,7 +107,7 @@ unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_RecordBatchStream) { drop(private_data); } -fn record_batch_to_wrapped_array( +pub fn record_batch_to_wrapped_array( record_batch: RecordBatch, ) -> RResult { let struct_array = StructArray::from(record_batch); @@ -157,7 +157,7 @@ impl RecordBatchStream for FFI_RecordBatchStream { } } -fn wrapped_array_to_record_batch(array: WrappedArray) -> Result { +pub fn wrapped_array_to_record_batch(array: WrappedArray) -> Result { let array_data = unsafe { from_ffi(array.array, &array.schema.0).map_err(DataFusionError::from)? }; let array = make_array(array_data); From 607ee5b3ab64bdcd0418137cd768e9d16ea32cd6 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 20 Oct 2025 08:29:22 -0400 Subject: [PATCH 03/69] physical expr minimally compiles but not all supporting structs implemented --- datafusion/ffi/src/expr/columnar_value.rs | 2 +- datafusion/ffi/src/expr/distribution.rs | 16 +- datafusion/ffi/src/expr/expr_properties.rs | 16 +- datafusion/ffi/src/expr/interval.rs | 15 +- datafusion/ffi/src/physical_expr/mod.rs | 320 +++++++++++++++------ datafusion/ffi/src/record_batch_stream.rs | 2 + 6 files changed, 270 insertions(+), 101 deletions(-) diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs index f05b525a4f25..fead61cc466f 100644 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -1,5 +1,5 @@ use abi_stable::StableAbi; -use datafusion_expr::ColumnarValue; +use datafusion::logical_expr::ColumnarValue; #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs index d92255c6dc38..ebaea24efc82 100644 --- a/datafusion/ffi/src/expr/distribution.rs +++ b/datafusion/ffi/src/expr/distribution.rs @@ -1,5 +1,5 @@ use abi_stable::StableAbi; -use datafusion_expr::statistics::Distribution; +use datafusion::logical_expr::statistics::Distribution; #[repr(C)] #[derive(Debug, StableAbi)] @@ -7,14 +7,26 @@ use datafusion_expr::statistics::Distribution; pub struct FFI_Distribution { } +impl From<&Distribution> for FFI_Distribution { + fn from(value: &Distribution) -> Self { + todo!() + } +} + impl From for FFI_Distribution { fn from(value: Distribution) -> Self { + FFI_Distribution::from(&value) + } +} + +impl From<&FFI_Distribution> for Distribution { + fn from(value: &FFI_Distribution) -> Self { todo!() } } impl From for Distribution { fn from(value: FFI_Distribution) -> Self { - todo!() + Distribution::from(&value) } } \ No newline at end of file diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs index f8133a065b05..d75c6eab8a8a 100644 --- a/datafusion/ffi/src/expr/expr_properties.rs +++ b/datafusion/ffi/src/expr/expr_properties.rs @@ -1,5 +1,5 @@ use abi_stable::StableAbi; -use datafusion_expr::sort_properties::ExprProperties; +use datafusion::logical_expr::sort_properties::ExprProperties; #[repr(C)] #[derive(Debug, StableAbi)] @@ -7,14 +7,26 @@ use datafusion_expr::sort_properties::ExprProperties; pub struct FFI_ExprProperties { } +impl From<&ExprProperties> for FFI_ExprProperties { + fn from(value: &ExprProperties) -> Self { + todo!() + } +} + impl From for FFI_ExprProperties { fn from(value: ExprProperties) -> Self { + FFI_ExprProperties::from(&value) + } +} + +impl From<&FFI_ExprProperties> for ExprProperties { + fn from(value: &FFI_ExprProperties) -> Self { todo!() } } impl From for ExprProperties { fn from(value: FFI_ExprProperties) -> Self { - todo!() + ExprProperties::from(&value) } } \ No newline at end of file diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs index 496482bff449..8030821acaa7 100644 --- a/datafusion/ffi/src/expr/interval.rs +++ b/datafusion/ffi/src/expr/interval.rs @@ -1,5 +1,5 @@ use abi_stable::StableAbi; -use datafusion_expr::interval_arithmetic::Interval; +use datafusion::logical_expr::interval_arithmetic::Interval; #[repr(C)] #[derive(Debug, StableAbi)] @@ -7,14 +7,25 @@ use datafusion_expr::interval_arithmetic::Interval; pub struct FFI_Interval { } +impl From<&Interval> for FFI_Interval { + fn from(value: &Interval) -> Self { + todo!() + } +} impl From for FFI_Interval { fn from(value: Interval) -> Self { + FFI_Interval::from(&value) + } +} + +impl From<&FFI_Interval> for Interval { + fn from(value: &FFI_Interval) -> Self { todo!() } } impl From for Interval { fn from(value: FFI_Interval) -> Self { - todo!() + Interval::from(&value) } } \ No newline at end of file diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index 7e98b7c5777e..07dd65d3a4cb 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -25,6 +25,7 @@ use crate::record_batch_stream::{ record_batch_to_wrapped_array, wrapped_array_to_record_batch, }; use crate::table_provider::FFI_TableProvider; +use crate::udaf::ForeignAggregateUDF; use crate::util::FFIResult; use crate::{ arrow_wrappers::WrappedSchema, @@ -43,7 +44,8 @@ use abi_stable::{ use arrow::array::{ArrayRef, BooleanArray, RecordBatch}; use arrow::datatypes::SchemaRef; use arrow::ffi::FFI_ArrowArray; -use arrow_schema::{DataType, FieldRef, Schema}; +use arrow_schema::ffi::FFI_ArrowSchema; +use arrow_schema::{DataType, Field, FieldRef, Schema}; use async_ffi::{FfiFuture, FutureExt}; use async_trait::async_trait; use datafusion::logical_expr::ColumnarValue; @@ -51,9 +53,10 @@ use datafusion::logical_expr_common::interval_arithmetic::Interval; use datafusion::logical_expr_common::sort_properties::ExprProperties; use datafusion::logical_expr_common::statistics::Distribution; use datafusion::physical_expr::PhysicalExpr; -use datafusion::physical_expr_common::physical_expr::fmt_sql; -use datafusion_common::DataFusionError; +use datafusion::physical_expr_common::physical_expr::{fmt_sql, DynEq}; +use datafusion::physical_plan::ExecutionPlan; use datafusion_common::Result; +use datafusion_common::{exec_datafusion_err, DataFusionError}; use datafusion_proto::{ logical_plan::{ from_proto::parse_exprs, to_proto::serialize_exprs, DefaultLogicalExtensionCodec, @@ -61,9 +64,11 @@ use datafusion_proto::{ protobuf::LogicalExprList, }; use prost::Message; -use std::fmt::Formatter; +use std::fmt::{Display, Formatter}; +use std::hash::{DefaultHasher, Hash, Hasher}; use std::{any::Any, ffi::c_void, sync::Arc}; use tokio::runtime::Handle; +use datafusion::logical_expr::expr_rewriter::unalias; #[repr(C)] #[derive(Debug, StableAbi)] @@ -88,41 +93,41 @@ pub struct FFI_PhysicalExpr { pub evaluate_selection: unsafe extern "C" fn( &Self, batch: WrappedArray, - selection: FFI_ArrowArray, + selection: WrappedArray, ) -> FFIResult, pub children: unsafe extern "C" fn(&Self) -> RVec, pub new_with_children: - unsafe extern "C" fn(&Self, children: RVec) -> FFIResult, + unsafe extern "C" fn(&Self, children: &RVec) -> FFIResult, - pub evalutate_bounds: unsafe extern "C" fn( + pub evaluate_bounds: unsafe extern "C" fn( &Self, - children: RSlice, - ) -> FFIResult, + children: &RVec, + ) -> FFIResult, pub propagate_constraints: unsafe extern "C" fn( &Self, interval: &FFI_Interval, - children: RSlice, + children: &RVec, ) -> FFIResult>>, pub evaluate_statistics: unsafe extern "C" fn( &Self, - children: &[&FFI_Distribution], + children: &RVec, ) -> FFIResult, pub propagate_statistics: unsafe extern "C" fn( &Self, parent: &FFI_Distribution, - children: &[&FFI_Distribution], + children: &RVec, ) -> FFIResult>>, pub get_properties: unsafe extern "C" fn( &Self, - children: &[FFI_ExprProperties], + children: &RVec, ) -> FFIResult, pub fmt_sql: unsafe extern "C" fn(&Self) -> FFIResult, @@ -133,6 +138,12 @@ pub struct FFI_PhysicalExpr { pub is_volatile_node: unsafe extern "C" fn(&Self) -> bool, + // Display trait + pub display: unsafe extern "C" fn(&Self) -> RString, + + // Hash trait + pub hash: unsafe extern "C" fn(&Self) -> u64, + /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -161,7 +172,7 @@ impl FFI_PhysicalExpr { } struct PhysicalExprPrivateData { - expr: Arc, + expr: Arc, } unsafe extern "C" fn data_type_fn_wrapper( @@ -169,9 +180,12 @@ unsafe extern "C" fn data_type_fn_wrapper( input_schema: WrappedSchema, ) -> FFIResult { let expr = expr.inner(); - let schema = input_schema.into(); - let data_type = expr.data_type(schema); - rresult!(data_type.map(|dt| dt.into())) + let schema: SchemaRef = rresult_return!(input_schema.try_into()); + let data_type = expr + .data_type(&schema) + .and_then(|dt| FFI_ArrowSchema::try_from(dt).map_err(Into::into)) + .map(WrappedSchema); + rresult!(data_type) } unsafe extern "C" fn nullable_fn_wrapper( @@ -179,8 +193,8 @@ unsafe extern "C" fn nullable_fn_wrapper( input_schema: WrappedSchema, ) -> FFIResult { let expr = expr.inner(); - let schema = input_schema.into(); - rresult!(expr.nullable(schema)) + let schema: SchemaRef = rresult_return!(input_schema.try_into()); + rresult!(expr.nullable(&schema)) } unsafe extern "C" fn evaluate_fn_wrapper( @@ -197,18 +211,24 @@ unsafe extern "C" fn return_field_fn_wrapper( input_schema: WrappedSchema, ) -> FFIResult { let expr = expr.inner(); - let schema = input_schema.into(); - rresult_return!(expr.return_field(schema).map(|field| field.into())) + let schema: SchemaRef = rresult_return!(input_schema.try_into()); + rresult!(expr + .return_field(&schema) + .and_then(|f| FFI_ArrowSchema::try_from(&f).map_err(Into::into)) + .map(WrappedSchema)) } unsafe extern "C" fn evaluate_selection_fn_wrapper( expr: &FFI_PhysicalExpr, batch: WrappedArray, - selection: FFI_ArrowArray, + selection: WrappedArray, ) -> FFIResult { let batch = rresult_return!(wrapped_array_to_record_batch(batch)); - let selection: ArrayRef = selection.into(); - let selection = rresult_return!(selection.as_any().downcast_ref::()); + let selection: ArrayRef = rresult_return!(selection.try_into()); + let selection = rresult_return!(selection + .as_any() + .downcast_ref::() + .ok_or(exec_datafusion_err!("Unexpected selection array type"))); let value = rresult_return!(expr.inner().evaluate_selection(&batch, selection)); RResult::ROk(value.into()) } @@ -218,35 +238,43 @@ unsafe extern "C" fn children_fn_wrapper( ) -> RVec { let expr = expr.inner(); let children = expr.children(); - children.into_iter().map(|child| child.into()).collect() + children + .into_iter() + .map(|child| FFI_PhysicalExpr::from(Arc::clone(child))) + .collect() } unsafe extern "C" fn new_with_children_fn_wrapper( expr: &FFI_PhysicalExpr, - children: RVec, + children: &RVec, ) -> FFIResult { - let expr = expr.inner(); - let children = children.into_iter().map(|child| child.into()).collect(); - rresult!(expr.new_with_children(children).map(|child| child.into())) + let expr = Arc::clone(expr.inner()); + let children = children + .iter() + .map(|e| ForeignPhysicalExpr::from(e.clone())) + .map(|e| Arc::new(e) as Arc) + .collect::>(); + rresult!(expr.with_new_children(children).map(FFI_PhysicalExpr::from)) } -unsafe extern "C" fn evalutate_bounds_fn_wrapper( +unsafe extern "C" fn evaluate_bounds_fn_wrapper( expr: &FFI_PhysicalExpr, - children: RSlice, -) -> FFIResult { + children: &RVec, +) -> FFIResult { let expr = expr.inner(); let children = children .iter() .map(|child| child.into()) .collect::>(); + let children_borrowed = children.iter().collect::>(); - rresult!(expr.evaluate_bounds(&children)) + rresult!(expr.evaluate_bounds(&children_borrowed).map(Into::into)) } unsafe extern "C" fn propagate_constraints_fn_wrapper( expr: &FFI_PhysicalExpr, interval: &FFI_Interval, - children: RSlice, + children: &RVec, ) -> FFIResult>> { let expr = expr.inner(); let interval = interval.into(); @@ -254,49 +282,65 @@ unsafe extern "C" fn propagate_constraints_fn_wrapper( .iter() .map(|child| child.into()) .collect::>(); + let children_borrowed = children.iter().collect::>(); - rresult!(expr.propagate_constraints(&interval, &children).into()) + let result = + rresult_return!(expr.propagate_constraints(&interval, &children_borrowed)); + + RResult::ROk( + result + .map(|i| i.into_iter().map(Into::into).collect()) + .into(), + ) } unsafe extern "C" fn evaluate_statistics_fn_wrapper( expr: &FFI_PhysicalExpr, - children: &[&FFI_Distribution], + children: &RVec, ) -> FFIResult { let expr = expr.inner(); let children = children .iter() .map(|child| child.into()) .collect::>(); - rresult!(expr.evaluate_statistics(&children).into()) + let children_borrowed = children.iter().collect::>(); + rresult!(expr.evaluate_statistics(&children_borrowed).map(Into::into)) } unsafe extern "C" fn propagate_statistics_fn_wrapper( expr: &FFI_PhysicalExpr, parent: &FFI_Distribution, - children: &[&FFI_Distribution], + children: &RVec, ) -> FFIResult>> { let expr = expr.inner(); let parent = parent.into(); - let children = children - .iter() - .map(|child| child.into()) - .collect::>(); + let children = children.iter().map(Distribution::from).collect::>(); + let children_borrowed = children.iter().collect::>(); + + let result = rresult_return!(expr.propagate_statistics(&parent, &children_borrowed)); - rresult!(expr.propagate_statistics(&parent, &children).into()) + RResult::ROk( + result + .map(|v| v.into_iter().map(Into::into).collect()) + .into(), + ) } unsafe extern "C" fn get_properties_fn_wrapper( expr: &FFI_PhysicalExpr, - children: &[FFI_ExprProperties], + children: &RVec, ) -> FFIResult { let expr = expr.inner(); - let children = children.iter().map(|child| child.into()).collect(); - rresult!(expr.get_properties(&children).into()) + let children = children + .iter() + .map(|child| child.into()) + .collect::>(); + rresult!(expr.get_properties(&children).map(Into::into)) } unsafe extern "C" fn fmt_sql_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFIResult { let expr = expr.inner(); - let result = fmt_sql(expr).to_string(); + let result = fmt_sql(expr.as_ref()).to_string(); RResult::ROk(result.into()) } @@ -304,7 +348,9 @@ unsafe extern "C" fn snapshot_fn_wrapper( expr: &FFI_PhysicalExpr, ) -> FFIResult> { let expr = expr.inner(); - rresult!(expr.snapshot().map(|snapshot| snapshot.into())) + rresult!(expr + .snapshot() + .map(|snapshot| snapshot.map(FFI_PhysicalExpr::from).into())) } unsafe extern "C" fn snapshot_generation_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 { @@ -316,6 +362,18 @@ unsafe extern "C" fn is_volatile_node_fn_wrapper(expr: &FFI_PhysicalExpr) -> boo let expr = expr.inner(); expr.is_volatile_node() } +unsafe extern "C" fn display_fn_wrapper(expr: &FFI_PhysicalExpr) -> RString { + let expr = expr.inner(); + format!("{expr}").into() +} + +unsafe extern "C" fn hash_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 +{ + let mut expr = expr.inner(); + let mut hasher = DefaultHasher::new(); + expr.hash(&mut hasher); + hasher.finish() +} unsafe extern "C" fn release_fn_wrapper(expr: &mut FFI_PhysicalExpr) { let private_data = Box::from_raw(expr.private_data as *mut PhysicalExprPrivateData); @@ -337,7 +395,7 @@ unsafe extern "C" fn clone_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFI_PhysicalEx evaluate_selection: evaluate_selection_fn_wrapper, children: children_fn_wrapper, new_with_children: new_with_children_fn_wrapper, - evalutate_bounds: evalutate_bounds_fn_wrapper, + evaluate_bounds: evaluate_bounds_fn_wrapper, propagate_constraints: propagate_constraints_fn_wrapper, evaluate_statistics: evaluate_statistics_fn_wrapper, propagate_statistics: propagate_statistics_fn_wrapper, @@ -346,6 +404,8 @@ unsafe extern "C" fn clone_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFI_PhysicalEx snapshot: snapshot_fn_wrapper, snapshot_generation: snapshot_generation_fn_wrapper, is_volatile_node: is_volatile_node_fn_wrapper, + display: display_fn_wrapper, + hash: hash_fn_wrapper, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -359,9 +419,9 @@ impl Drop for FFI_PhysicalExpr { } } -impl FFI_PhysicalExpr { +impl From> for FFI_PhysicalExpr { /// Creates a new [`FFI_PhysicalExpr`]. - pub fn new(expr: Arc) -> Self { + fn from(expr: Arc) -> Self { let private_data = Box::new(PhysicalExprPrivateData { expr }); Self { @@ -372,7 +432,7 @@ impl FFI_PhysicalExpr { evaluate_selection: evaluate_selection_fn_wrapper, children: children_fn_wrapper, new_with_children: new_with_children_fn_wrapper, - evalutate_bounds: evalutate_bounds_fn_wrapper, + evaluate_bounds: evaluate_bounds_fn_wrapper, propagate_constraints: propagate_constraints_fn_wrapper, evaluate_statistics: evaluate_statistics_fn_wrapper, propagate_statistics: propagate_statistics_fn_wrapper, @@ -381,6 +441,8 @@ impl FFI_PhysicalExpr { snapshot: snapshot_fn_wrapper, snapshot_generation: snapshot_generation_fn_wrapper, is_volatile_node: is_volatile_node_fn_wrapper, + display: display_fn_wrapper, + hash: hash_fn_wrapper, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -394,14 +456,26 @@ impl FFI_PhysicalExpr { /// defined on this struct must only use the stable functions provided in /// FFI_PhysicalExpr to interact with the foreign table provider. #[derive(Debug)] -pub struct ForeignPhysicalExpr(pub FFI_PhysicalExpr); +pub struct ForeignPhysicalExpr { + pub expr: FFI_PhysicalExpr, + children: Vec>, +} unsafe impl Send for ForeignPhysicalExpr {} unsafe impl Sync for ForeignPhysicalExpr {} -impl From<&FFI_PhysicalExpr> for ForeignPhysicalExpr { - fn from(provider: &FFI_PhysicalExpr) -> Self { - Self(provider.clone()) +impl From for ForeignPhysicalExpr { + fn from(expr: FFI_PhysicalExpr) -> Self { + let children = unsafe { + (expr.children)(&expr) + .into_iter() + .map(|child| { + Arc::new(ForeignPhysicalExpr::from(child)) as Arc + }) + .collect() + }; + + Self { expr, children } } } @@ -418,29 +492,31 @@ impl PhysicalExpr for ForeignPhysicalExpr { fn data_type(&self, input_schema: &Schema) -> Result { unsafe { - let schema = input_schema.into(); - df_result!((self.0.data_type)(&self.0, schema).into()) + let schema = WrappedSchema::from(Arc::new(input_schema.clone())); + df_result!((self.expr.data_type)(&self.expr, schema)) + .and_then(|d| DataType::try_from(&d.0).map_err(Into::into)) } } fn nullable(&self, input_schema: &Schema) -> Result { unsafe { - let schema = input_schema.into(); - df_result!((self.0.nullable)(&self.0, schema)) + let schema = WrappedSchema::from(Arc::new(input_schema.clone())); + df_result!((self.expr.nullable)(&self.expr, schema)) } } fn evaluate(&self, batch: &RecordBatch) -> Result { unsafe { - let batch = record_batch_to_wrapped_array(batch.clone())?; - df_result!((self.0.evaluate)(&self.0, batch).map(|v| v.into())) + let batch = df_result!(record_batch_to_wrapped_array(batch.clone()))?; + df_result!((self.expr.evaluate)(&self.expr, batch).map(|v| v.into())) } } fn return_field(&self, input_schema: &Schema) -> Result { unsafe { - let schema = input_schema.into(); - df_result!((self.0.return_field)(&self.0, schema).map(|f| f.into())) + let schema = WrappedSchema::from(Arc::new(input_schema.clone())); + let result = df_result!((self.expr.return_field)(&self.expr, schema))?; + Field::try_from(&result.0).map(Arc::new).map_err(Into::into) } } @@ -450,21 +526,19 @@ impl PhysicalExpr for ForeignPhysicalExpr { selection: &BooleanArray, ) -> Result { unsafe { - let batch = record_batch_to_wrapped_array(batch)?; - let selection = selection.into(); - df_result!( - (self.0.evaluate_selection)(&self.0, batch, selection).map(|f| f.into()) - ) + let batch = df_result!(record_batch_to_wrapped_array(batch.clone()))?; + // This is not ideal - we are cloning the selection array + // This is not terrible since it will be a small array. + // The other alternative is to modify the trait signature. + let selection: ArrayRef = Arc::new(selection.clone()); + let selection = WrappedArray::try_from(&selection)?; + df_result!((self.expr.evaluate_selection)(&self.expr, batch, selection) + .map(|f| f.into())) } } fn children(&self) -> Vec<&Arc> { - unsafe { - (self.0.children)(&self.0) - .into_iter() - .map(Into::into) - .collect() - } + self.children.iter().collect() } fn with_new_children( @@ -472,15 +546,22 @@ impl PhysicalExpr for ForeignPhysicalExpr { children: Vec>, ) -> Result> { unsafe { - let children = children.into_iter().map(Into::into).collect(); - df_result!((self.0.new_with_children)(&self.0, children).map(Into::into)) + let children = children + .into_iter() + .map(|expr| FFI_PhysicalExpr::from(expr)) + .collect(); + df_result!((self.expr.new_with_children)(&self.expr, &children) + .map(|expr| Arc::new(ForeignPhysicalExpr::from(expr)))) } } fn evaluate_bounds(&self, children: &[&Interval]) -> Result { unsafe { - let children = children.iter().map(Into::into).collect(); - df_result!((self.0.evalutate_bounds)(&self.0, children)) + let children = children + .iter() + .map(|interval| FFI_Interval::from(*interval)) + .collect::>(); + df_result!((self.expr.evaluate_bounds)(&self.expr, &children)).map(Into::into) } } @@ -491,17 +572,31 @@ impl PhysicalExpr for ForeignPhysicalExpr { ) -> Result>> { unsafe { let interval = interval.into(); - let children = children.iter().map(Into::into).collect(); - df_result!((self.0.propagate_constraints)(&self.0, interval, children) - .map(Into::into) + let children = children + .iter() + .map(|interval| FFI_Interval::from(*interval)) + .collect::>(); + let result = df_result!((self.expr.propagate_constraints)( + &self.expr, &interval, &children + ))?; + + Ok(result + .map(|intervals| { + intervals.into_iter().map(Into::into).collect::>() + }) .into()) } } fn evaluate_statistics(&self, children: &[&Distribution]) -> Result { unsafe { - let children = children.iter().map(Into::into).collect::>(); - df_result!((self.0.evaluate_statistics)(&self.0, &children).map(Into::into)) + let children = children + .iter() + .map(|dist| FFI_Distribution::from(*dist)) + .collect::>(); + df_result!( + (self.expr.evaluate_statistics)(&self.expr, &children).map(Into::into) + ) } } @@ -512,23 +607,30 @@ impl PhysicalExpr for ForeignPhysicalExpr { ) -> Result>> { unsafe { let parent = parent.into(); - let children = children.iter().map(Into::into).collect(); - df_result!( - (self.0.propagate_statistics)(&self.0, parent, children).map(Into::into) - ) + let children = children + .iter() + .map(|dist| FFI_Distribution::from(*dist)) + .collect::>(); + let result = df_result!((self.expr.propagate_statistics)( + &self.expr, &parent, &children + ))?; + + Ok(result + .map(|dists| dists.into_iter().map(Into::into).collect()) + .into()) } } fn get_properties(&self, children: &[ExprProperties]) -> Result { unsafe { - let children = children.iter().map(Into::into).collect(); - df_result!((self.0.get_properties)(&self.0, children).map(Into::into)) + let children = children.iter().map(Into::into).collect::>(); + df_result!((self.expr.get_properties)(&self.expr, &children).map(Into::into)) } } fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { unsafe { - match (self.0.fmt_sql)(&self.0) { + match (self.expr.fmt_sql)(&self.expr) { RResult::ROk(sql) => write!(f, "{sql}"), RResult::RErr(_) => Err(std::fmt::Error), } @@ -536,14 +638,44 @@ impl PhysicalExpr for ForeignPhysicalExpr { } fn snapshot(&self) -> Result>> { - unsafe { df_result!((self.0.snapshot)(&self.0).map(Into::into)) } + unsafe { + let result = df_result!((self.expr.snapshot)(&self.expr))?; + Ok(result + .map(|ffi_expr| { + Arc::new(ForeignPhysicalExpr::from(ffi_expr)) as Arc + }) + .into()) + } } fn snapshot_generation(&self) -> u64 { - unsafe { (self.0.snapshot_generation)(&self.0) } + unsafe { (self.expr.snapshot_generation)(&self.expr) } } fn is_volatile_node(&self) -> bool { - unsafe { (self.0.is_volatile_node)(&self.0) } + unsafe { (self.expr.is_volatile_node)(&self.expr) } } } + +impl Eq for ForeignPhysicalExpr {} +impl PartialEq for ForeignPhysicalExpr { + fn eq(&self, other: &Self) -> bool { + // FFI_PhysicalExpr cannot be compared, so identity equality is the best we can do. + std::ptr::eq(self, other) + } +} +impl Hash for ForeignPhysicalExpr { + fn hash(&self, state: &mut H) { + let value = unsafe { (self.expr.hash)(&self.expr) }; + value.hash(state) + } +} + +impl Display for ForeignPhysicalExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + let display = unsafe { + (self.expr.display)(&self.expr) + }; + write!(f, "{display}") + } +} \ No newline at end of file diff --git a/datafusion/ffi/src/record_batch_stream.rs b/datafusion/ffi/src/record_batch_stream.rs index 046c61cea36b..89fe6df617cf 100644 --- a/datafusion/ffi/src/record_batch_stream.rs +++ b/datafusion/ffi/src/record_batch_stream.rs @@ -107,6 +107,8 @@ unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_RecordBatchStream) { drop(private_data); } +// TODO(tsaucer) switch to Result +// and put the rresult handling to the caller pub fn record_batch_to_wrapped_array( record_batch: RecordBatch, ) -> RResult { From 3877c19e4bee2bfb37b635695ce19065c096245e Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 20 Oct 2025 17:56:01 -0400 Subject: [PATCH 04/69] Implement ffi_interval --- datafusion/ffi/src/expr/interval.rs | 51 +++++++++++++++----- datafusion/ffi/src/physical_expr/mod.rs | 63 +++++++++++++------------ 2 files changed, 72 insertions(+), 42 deletions(-) diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs index 8030821acaa7..7963539766ce 100644 --- a/datafusion/ffi/src/expr/interval.rs +++ b/datafusion/ffi/src/expr/interval.rs @@ -1,31 +1,56 @@ +use abi_stable::std_types::RVec; use abi_stable::StableAbi; +use arrow::ffi::FFI_ArrowArray; +use arrow_schema::ArrowError; use datafusion::logical_expr::interval_arithmetic::Interval; +use datafusion_common::DataFusionError; +use prost::Message; #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] pub struct FFI_Interval { + lower: RVec, + upper: RVec, } -impl From<&Interval> for FFI_Interval { - fn from(value: &Interval) -> Self { - todo!() +impl TryFrom<&Interval> for FFI_Interval { + type Error = DataFusionError; + fn try_from(value: &Interval) -> Result { + let upper: datafusion_proto_common::ScalarValue = value.upper().try_into()?; + let lower: datafusion_proto_common::ScalarValue = value.lower().try_into()?; + + let upper = upper.encode_to_vec().into(); + let lower = lower.encode_to_vec().into(); + + Ok(FFI_Interval { upper, lower }) } } -impl From for FFI_Interval { - fn from(value: Interval) -> Self { - FFI_Interval::from(&value) +impl TryFrom for FFI_Interval { + type Error = DataFusionError; + fn try_from(value: Interval) -> Result { + FFI_Interval::try_from(&value) } } -impl From<&FFI_Interval> for Interval { - fn from(value: &FFI_Interval) -> Self { - todo!() +impl TryFrom<&FFI_Interval> for Interval { + type Error = DataFusionError; + fn try_from(value: &FFI_Interval) -> Result { + let upper = datafusion_proto_common::ScalarValue::decode(value.upper.as_ref()) + .map_err(|err| DataFusionError::Execution(err.to_string()))?; + let lower = datafusion_proto_common::ScalarValue::decode(value.lower.as_ref()) + .map_err(|err| DataFusionError::Execution(err.to_string()))?; + + let upper = (&upper).try_into()?; + let lower = (&lower).try_into()?; + + Interval::try_new(lower, upper) } } -impl From for Interval { - fn from(value: FFI_Interval) -> Self { - Interval::from(&value) +impl TryFrom for Interval { + type Error = DataFusionError; + fn try_from(value: FFI_Interval) -> Result { + Interval::try_from(&value) } -} \ No newline at end of file +} diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index 07dd65d3a4cb..1ec105166ce8 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -48,6 +48,7 @@ use arrow_schema::ffi::FFI_ArrowSchema; use arrow_schema::{DataType, Field, FieldRef, Schema}; use async_ffi::{FfiFuture, FutureExt}; use async_trait::async_trait; +use datafusion::logical_expr::expr_rewriter::unalias; use datafusion::logical_expr::ColumnarValue; use datafusion::logical_expr_common::interval_arithmetic::Interval; use datafusion::logical_expr_common::sort_properties::ExprProperties; @@ -68,7 +69,6 @@ use std::fmt::{Display, Formatter}; use std::hash::{DefaultHasher, Hash, Hasher}; use std::{any::Any, ffi::c_void, sync::Arc}; use tokio::runtime::Handle; -use datafusion::logical_expr::expr_rewriter::unalias; #[repr(C)] #[derive(Debug, StableAbi)] @@ -262,13 +262,13 @@ unsafe extern "C" fn evaluate_bounds_fn_wrapper( children: &RVec, ) -> FFIResult { let expr = expr.inner(); - let children = children + let children = rresult_return!( children .iter() - .map(|child| child.into()) - .collect::>(); + .map(Interval::try_from) + .collect::>>()); let children_borrowed = children.iter().collect::>(); - rresult!(expr.evaluate_bounds(&children_borrowed).map(Into::into)) + rresult!(expr.evaluate_bounds(&children_borrowed).and_then(FFI_Interval::try_from)) } unsafe extern "C" fn propagate_constraints_fn_wrapper( @@ -277,21 +277,24 @@ unsafe extern "C" fn propagate_constraints_fn_wrapper( children: &RVec, ) -> FFIResult>> { let expr = expr.inner(); - let interval = interval.into(); - let children = children + let interval = rresult_return!(Interval::try_from(interval)); + let children = rresult_return!(children .iter() - .map(|child| child.into()) - .collect::>(); + .map(Interval::try_from) + .collect::>>()); let children_borrowed = children.iter().collect::>(); let result = rresult_return!(expr.propagate_constraints(&interval, &children_borrowed)); - RResult::ROk( - result - .map(|i| i.into_iter().map(Into::into).collect()) - .into(), - ) + let result = rresult_return!(result + .map(|intervals| intervals + .into_iter() + .map(FFI_Interval::try_from) + .collect::>>()) + .transpose()); + + RResult::ROk(result.into()) } unsafe extern "C" fn evaluate_statistics_fn_wrapper( @@ -367,8 +370,7 @@ unsafe extern "C" fn display_fn_wrapper(expr: &FFI_PhysicalExpr) -> RString { format!("{expr}").into() } -unsafe extern "C" fn hash_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 -{ +unsafe extern "C" fn hash_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 { let mut expr = expr.inner(); let mut hasher = DefaultHasher::new(); expr.hash(&mut hasher); @@ -559,9 +561,10 @@ impl PhysicalExpr for ForeignPhysicalExpr { unsafe { let children = children .iter() - .map(|interval| FFI_Interval::from(*interval)) - .collect::>(); - df_result!((self.expr.evaluate_bounds)(&self.expr, &children)).map(Into::into) + .map(|interval| FFI_Interval::try_from(*interval)) + .collect::>>()?; + df_result!((self.expr.evaluate_bounds)(&self.expr, &children)) + .and_then(Interval::try_from) } } @@ -571,20 +574,24 @@ impl PhysicalExpr for ForeignPhysicalExpr { children: &[&Interval], ) -> Result>> { unsafe { - let interval = interval.into(); + let interval = interval.try_into()?; let children = children .iter() - .map(|interval| FFI_Interval::from(*interval)) - .collect::>(); + .map(|interval| FFI_Interval::try_from(*interval)) + .collect::>>()?; let result = df_result!((self.expr.propagate_constraints)( &self.expr, &interval, &children ))?; - Ok(result + let result: Option<_> = result .map(|intervals| { - intervals.into_iter().map(Into::into).collect::>() + intervals + .into_iter() + .map(Interval::try_from) + .collect::>>() }) - .into()) + .into(); + result.transpose() } } @@ -673,9 +680,7 @@ impl Hash for ForeignPhysicalExpr { impl Display for ForeignPhysicalExpr { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - let display = unsafe { - (self.expr.display)(&self.expr) - }; + let display = unsafe { (self.expr.display)(&self.expr) }; write!(f, "{display}") } -} \ No newline at end of file +} From 65aa587d85affb652ee548fd414b3aaa4ca527b9 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 20 Oct 2025 20:07:46 -0400 Subject: [PATCH 05/69] implement ffi_columnarvalue --- datafusion/ffi/src/expr/columnar_value.rs | 41 ++++++++++++++++++----- datafusion/ffi/src/physical_expr/mod.rs | 12 +++---- 2 files changed, 38 insertions(+), 15 deletions(-) diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs index fead61cc466f..2feec9742d9d 100644 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -1,20 +1,45 @@ +use crate::arrow_wrappers::WrappedArray; +use abi_stable::std_types::RVec; use abi_stable::StableAbi; +use arrow::ffi::FFI_ArrowArray; +use arrow_schema::ArrowError; use datafusion::logical_expr::ColumnarValue; +use datafusion_common::DataFusionError; +use prost::Message; #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] -pub struct FFI_ColumnarValue { +pub enum FFI_ColumnarValue { + Array(WrappedArray), + Scalar(RVec), } -impl From for FFI_ColumnarValue { - fn from(value: ColumnarValue) -> Self { - todo!() +impl TryFrom for FFI_ColumnarValue { + type Error = DataFusionError; + fn try_from(value: ColumnarValue) -> Result { + Ok(match value { + ColumnarValue::Array(v) => { + FFI_ColumnarValue::Array({ WrappedArray::try_from(&v)? }) + } + ColumnarValue::Scalar(v) => FFI_ColumnarValue::Scalar({ + let v: datafusion_proto_common::ScalarValue = (&v).try_into()?; + v.encode_to_vec().into() + }), + }) } } -impl From for ColumnarValue { - fn from(value: FFI_ColumnarValue) -> Self { - todo!() +impl TryFrom for ColumnarValue { + type Error = DataFusionError; + fn try_from(value: FFI_ColumnarValue) -> Result { + Ok(match value { + FFI_ColumnarValue::Array(v) => ColumnarValue::Array({ v.try_into()? }), + FFI_ColumnarValue::Scalar(v) => ColumnarValue::Scalar({ + let v = datafusion_proto_common::ScalarValue::decode(v.as_ref()) + .map_err(|err| DataFusionError::Execution(err.to_string()))?; + (&v).try_into()? + }), + }) } -} \ No newline at end of file +} diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index 1ec105166ce8..c0f4beebe07d 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -202,8 +202,7 @@ unsafe extern "C" fn evaluate_fn_wrapper( batch: WrappedArray, ) -> FFIResult { let batch = rresult_return!(wrapped_array_to_record_batch(batch)); - let value = rresult_return!(expr.inner().evaluate(&batch)); - RResult::ROk(value.into()) + rresult!(expr.inner().evaluate(&batch).and_then(FFI_ColumnarValue::try_from)) } unsafe extern "C" fn return_field_fn_wrapper( @@ -229,8 +228,7 @@ unsafe extern "C" fn evaluate_selection_fn_wrapper( .as_any() .downcast_ref::() .ok_or(exec_datafusion_err!("Unexpected selection array type"))); - let value = rresult_return!(expr.inner().evaluate_selection(&batch, selection)); - RResult::ROk(value.into()) + rresult!(expr.inner().evaluate_selection(&batch, selection).and_then(FFI_ColumnarValue::try_from)) } unsafe extern "C" fn children_fn_wrapper( @@ -510,7 +508,7 @@ impl PhysicalExpr for ForeignPhysicalExpr { fn evaluate(&self, batch: &RecordBatch) -> Result { unsafe { let batch = df_result!(record_batch_to_wrapped_array(batch.clone()))?; - df_result!((self.expr.evaluate)(&self.expr, batch).map(|v| v.into())) + df_result!((self.expr.evaluate)(&self.expr, batch)).and_then(ColumnarValue::try_from) } } @@ -534,8 +532,8 @@ impl PhysicalExpr for ForeignPhysicalExpr { // The other alternative is to modify the trait signature. let selection: ArrayRef = Arc::new(selection.clone()); let selection = WrappedArray::try_from(&selection)?; - df_result!((self.expr.evaluate_selection)(&self.expr, batch, selection) - .map(|f| f.into())) + df_result!((self.expr.evaluate_selection)(&self.expr, batch, selection)) + .and_then(ColumnarValue::try_from) } } From 041f754ccb65bb1c072f7ec70bbbcd9e3fd333ac Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 21 Oct 2025 07:29:34 -0400 Subject: [PATCH 06/69] Implement FFI_Distribution --- datafusion/ffi/src/expr/columnar_value.rs | 8 +- datafusion/ffi/src/expr/distribution.rs | 198 ++++++++++++++++++++-- datafusion/ffi/src/expr/interval.rs | 17 +- datafusion/ffi/src/expr/mod.rs | 1 + datafusion/ffi/src/expr/util.rs | 16 ++ datafusion/ffi/src/physical_expr/mod.rs | 76 ++++++--- 6 files changed, 257 insertions(+), 59 deletions(-) create mode 100644 datafusion/ffi/src/expr/util.rs diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs index 2feec9742d9d..eb46540591f6 100644 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -6,6 +6,7 @@ use arrow_schema::ArrowError; use datafusion::logical_expr::ColumnarValue; use datafusion_common::DataFusionError; use prost::Message; +use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; #[repr(C)] #[derive(Debug, StableAbi)] @@ -23,8 +24,7 @@ impl TryFrom for FFI_ColumnarValue { FFI_ColumnarValue::Array({ WrappedArray::try_from(&v)? }) } ColumnarValue::Scalar(v) => FFI_ColumnarValue::Scalar({ - let v: datafusion_proto_common::ScalarValue = (&v).try_into()?; - v.encode_to_vec().into() + scalar_value_to_rvec_u8(&v)? }), }) } @@ -36,9 +36,7 @@ impl TryFrom for ColumnarValue { Ok(match value { FFI_ColumnarValue::Array(v) => ColumnarValue::Array({ v.try_into()? }), FFI_ColumnarValue::Scalar(v) => ColumnarValue::Scalar({ - let v = datafusion_proto_common::ScalarValue::decode(v.as_ref()) - .map_err(|err| DataFusionError::Execution(err.to_string()))?; - (&v).try_into()? + rvec_u8_to_scalar_value(&v)? }), }) } diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs index ebaea24efc82..e6144b69569f 100644 --- a/datafusion/ffi/src/expr/distribution.rs +++ b/datafusion/ffi/src/expr/distribution.rs @@ -1,32 +1,200 @@ +use crate::expr::interval::FFI_Interval; +use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; +use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion::logical_expr::statistics::Distribution; +use datafusion::logical_expr::interval_arithmetic::Interval; +use datafusion::logical_expr::statistics::{ + BernoulliDistribution, Distribution, ExponentialDistribution, GaussianDistribution, + GenericDistribution, UniformDistribution, +}; +use datafusion_common::{DataFusionError, ScalarValue}; +use prost::Message; #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] -pub struct FFI_Distribution { +pub enum FFI_Distribution { + Uniform(FFI_UniformDistribution), + Exponential(FFI_ExponentialDistribution), + Gaussian(FFI_GaussianDistribution), + Bernoulli(FFI_BernoulliDistribution), + Generic(FFI_GenericDistribution), } -impl From<&Distribution> for FFI_Distribution { - fn from(value: &Distribution) -> Self { - todo!() +impl TryFrom<&Distribution> for FFI_Distribution { + type Error = DataFusionError; + fn try_from(value: &Distribution) -> Result { + match value { + Distribution::Uniform(d) => Ok(FFI_Distribution::Uniform(d.try_into()?)), + Distribution::Exponential(d) => { + Ok(FFI_Distribution::Exponential(d.try_into()?)) + } + Distribution::Gaussian(d) => Ok(FFI_Distribution::Gaussian(d.try_into()?)), + Distribution::Bernoulli(d) => Ok(FFI_Distribution::Bernoulli(d.try_into()?)), + Distribution::Generic(d) => Ok(FFI_Distribution::Generic(d.try_into()?)), + } } } -impl From for FFI_Distribution { - fn from(value: Distribution) -> Self { - FFI_Distribution::from(&value) +impl TryFrom<&FFI_Distribution> for Distribution { + type Error = DataFusionError; + fn try_from(value: &FFI_Distribution) -> Result { + match value { + FFI_Distribution::Uniform(d) => d.try_into(), + FFI_Distribution::Exponential(d) => d.try_into(), + FFI_Distribution::Gaussian(d) => d.try_into(), + FFI_Distribution::Bernoulli(d) => d.try_into(), + FFI_Distribution::Generic(d) => d.try_into(), + } } } -impl From<&FFI_Distribution> for Distribution { - fn from(value: &FFI_Distribution) -> Self { - todo!() +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_UniformDistribution { + interval: FFI_Interval, +} + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_ExponentialDistribution { + rate: RVec, + offset: RVec, + positive_tail: bool, +} + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_GaussianDistribution { + mean: RVec, + variance: RVec, +} + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_BernoulliDistribution { + p: RVec, +} + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_GenericDistribution { + mean: RVec, + median: RVec, + variance: RVec, + range: FFI_Interval, +} + +impl TryFrom<&UniformDistribution> for FFI_UniformDistribution { + type Error = DataFusionError; + fn try_from(value: &UniformDistribution) -> Result { + Ok(Self { + interval: value.range().try_into()?, + }) + } +} + +impl TryFrom<&ExponentialDistribution> for FFI_ExponentialDistribution { + type Error = DataFusionError; + fn try_from(value: &ExponentialDistribution) -> Result { + let rate = scalar_value_to_rvec_u8(value.rate())?; + let offset = scalar_value_to_rvec_u8(value.offset())?; + + Ok(Self { + rate, + offset, + positive_tail: value.positive_tail(), + }) + } +} + +impl TryFrom<&GaussianDistribution> for FFI_GaussianDistribution { + type Error = DataFusionError; + fn try_from(value: &GaussianDistribution) -> Result { + let mean = scalar_value_to_rvec_u8(value.mean())?; + let variance = scalar_value_to_rvec_u8(value.variance())?; + + Ok(Self { mean, variance }) + } +} + +impl TryFrom<&BernoulliDistribution> for FFI_BernoulliDistribution { + type Error = DataFusionError; + fn try_from(value: &BernoulliDistribution) -> Result { + let p = scalar_value_to_rvec_u8(value.p_value())?; + + Ok(Self { p }) + } +} + +impl TryFrom<&GenericDistribution> for FFI_GenericDistribution { + type Error = DataFusionError; + fn try_from(value: &GenericDistribution) -> Result { + let mean = scalar_value_to_rvec_u8(value.mean())?; + let median = scalar_value_to_rvec_u8(value.median())?; + let variance = scalar_value_to_rvec_u8(value.variance())?; + + Ok(Self { + mean, + median, + variance, + range: value.range().try_into()?, + }) + } +} + + + +impl TryFrom<&FFI_UniformDistribution> for Distribution { + type Error = DataFusionError; + fn try_from(value: &FFI_UniformDistribution) -> Result { + let interval = (&value.interval).try_into()?; + Distribution::new_uniform(interval) + } +} + +impl TryFrom<&FFI_ExponentialDistribution> for Distribution { + type Error = DataFusionError; + fn try_from(value: &FFI_ExponentialDistribution) -> Result { + let rate = rvec_u8_to_scalar_value(&value.rate)?; + let offset = rvec_u8_to_scalar_value(&value.offset)?; + + Distribution::new_exponential(rate, offset, value.positive_tail) } } -impl From for Distribution { - fn from(value: FFI_Distribution) -> Self { - Distribution::from(&value) +impl TryFrom<&FFI_GaussianDistribution> for Distribution { + type Error = DataFusionError; + fn try_from(value: &FFI_GaussianDistribution) -> Result { + let mean = rvec_u8_to_scalar_value(&value.mean)?; + let variance = rvec_u8_to_scalar_value(&value.variance)?; + + Distribution::new_gaussian(mean, variance) } -} \ No newline at end of file +} + +impl TryFrom<&FFI_BernoulliDistribution> for Distribution { + type Error = DataFusionError; + fn try_from(value: &FFI_BernoulliDistribution) -> Result { + let p = rvec_u8_to_scalar_value(&value.p)?; + + Distribution::new_bernoulli(p) + } +} + +impl TryFrom<&FFI_GenericDistribution> for Distribution { + type Error = DataFusionError; + fn try_from(value: &FFI_GenericDistribution) -> Result { + let mean = rvec_u8_to_scalar_value(&value.mean)?; + let median = rvec_u8_to_scalar_value(&value.median)?; + let variance = rvec_u8_to_scalar_value(&value.variance)?; + let range = (&value.range).try_into()?; + + Distribution::new_generic(mean, median, variance, range) + } +} diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs index 7963539766ce..1a43ea514050 100644 --- a/datafusion/ffi/src/expr/interval.rs +++ b/datafusion/ffi/src/expr/interval.rs @@ -5,6 +5,7 @@ use arrow_schema::ArrowError; use datafusion::logical_expr::interval_arithmetic::Interval; use datafusion_common::DataFusionError; use prost::Message; +use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; #[repr(C)] #[derive(Debug, StableAbi)] @@ -17,11 +18,8 @@ pub struct FFI_Interval { impl TryFrom<&Interval> for FFI_Interval { type Error = DataFusionError; fn try_from(value: &Interval) -> Result { - let upper: datafusion_proto_common::ScalarValue = value.upper().try_into()?; - let lower: datafusion_proto_common::ScalarValue = value.lower().try_into()?; - - let upper = upper.encode_to_vec().into(); - let lower = lower.encode_to_vec().into(); + let upper = scalar_value_to_rvec_u8(value.upper())?; + let lower = scalar_value_to_rvec_u8(value.lower())?; Ok(FFI_Interval { upper, lower }) } @@ -36,13 +34,8 @@ impl TryFrom for FFI_Interval { impl TryFrom<&FFI_Interval> for Interval { type Error = DataFusionError; fn try_from(value: &FFI_Interval) -> Result { - let upper = datafusion_proto_common::ScalarValue::decode(value.upper.as_ref()) - .map_err(|err| DataFusionError::Execution(err.to_string()))?; - let lower = datafusion_proto_common::ScalarValue::decode(value.lower.as_ref()) - .map_err(|err| DataFusionError::Execution(err.to_string()))?; - - let upper = (&upper).try_into()?; - let lower = (&lower).try_into()?; + let upper = rvec_u8_to_scalar_value(&value.upper)?; + let lower = rvec_u8_to_scalar_value(&value.lower)?; Interval::try_new(lower, upper) } diff --git a/datafusion/ffi/src/expr/mod.rs b/datafusion/ffi/src/expr/mod.rs index d2dcee31bfef..9bfdc58c40c0 100644 --- a/datafusion/ffi/src/expr/mod.rs +++ b/datafusion/ffi/src/expr/mod.rs @@ -2,3 +2,4 @@ pub mod columnar_value; pub mod distribution; pub mod expr_properties; pub mod interval; +pub(crate) mod util; diff --git a/datafusion/ffi/src/expr/util.rs b/datafusion/ffi/src/expr/util.rs new file mode 100644 index 000000000000..575fe7fcd9f4 --- /dev/null +++ b/datafusion/ffi/src/expr/util.rs @@ -0,0 +1,16 @@ +use abi_stable::std_types::RVec; +use prost::Message; +use datafusion_common::{exec_datafusion_err, DataFusionError, Result, ScalarValue}; + +pub fn scalar_value_to_rvec_u8(value: &ScalarValue) -> Result> { + let value: datafusion_proto_common::ScalarValue = value.try_into()?; + Ok(value.encode_to_vec().into()) +} + +pub fn rvec_u8_to_scalar_value(value: &RVec) -> Result { + + let value = datafusion_proto_common::ScalarValue::decode(value.as_ref()) + .map_err(|err| exec_datafusion_err!("{err}"))?; + + (&value).try_into().map_err(|err| exec_datafusion_err!("{err}")) +} \ No newline at end of file diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index c0f4beebe07d..abb3abb544f6 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -202,7 +202,10 @@ unsafe extern "C" fn evaluate_fn_wrapper( batch: WrappedArray, ) -> FFIResult { let batch = rresult_return!(wrapped_array_to_record_batch(batch)); - rresult!(expr.inner().evaluate(&batch).and_then(FFI_ColumnarValue::try_from)) + rresult!(expr + .inner() + .evaluate(&batch) + .and_then(FFI_ColumnarValue::try_from)) } unsafe extern "C" fn return_field_fn_wrapper( @@ -228,7 +231,10 @@ unsafe extern "C" fn evaluate_selection_fn_wrapper( .as_any() .downcast_ref::() .ok_or(exec_datafusion_err!("Unexpected selection array type"))); - rresult!(expr.inner().evaluate_selection(&batch, selection).and_then(FFI_ColumnarValue::try_from)) + rresult!(expr + .inner() + .evaluate_selection(&batch, selection) + .and_then(FFI_ColumnarValue::try_from)) } unsafe extern "C" fn children_fn_wrapper( @@ -260,13 +266,15 @@ unsafe extern "C" fn evaluate_bounds_fn_wrapper( children: &RVec, ) -> FFIResult { let expr = expr.inner(); - let children = rresult_return!( children + let children = rresult_return!(children .iter() .map(Interval::try_from) .collect::>>()); let children_borrowed = children.iter().collect::>(); - rresult!(expr.evaluate_bounds(&children_borrowed).and_then(FFI_Interval::try_from)) + rresult!(expr + .evaluate_bounds(&children_borrowed) + .and_then(FFI_Interval::try_from)) } unsafe extern "C" fn propagate_constraints_fn_wrapper( @@ -300,12 +308,12 @@ unsafe extern "C" fn evaluate_statistics_fn_wrapper( children: &RVec, ) -> FFIResult { let expr = expr.inner(); - let children = children + let children = rresult_return!(children .iter() - .map(|child| child.into()) - .collect::>(); + .map(Distribution::try_from) + .collect::>>()); let children_borrowed = children.iter().collect::>(); - rresult!(expr.evaluate_statistics(&children_borrowed).map(Into::into)) + rresult!(expr.evaluate_statistics(&children_borrowed).and_then(|dist| FFI_Distribution::try_from(&dist))) } unsafe extern "C" fn propagate_statistics_fn_wrapper( @@ -314,17 +322,22 @@ unsafe extern "C" fn propagate_statistics_fn_wrapper( children: &RVec, ) -> FFIResult>> { let expr = expr.inner(); - let parent = parent.into(); - let children = children.iter().map(Distribution::from).collect::>(); + let parent = rresult_return!(Distribution::try_from(parent)); + let children = rresult_return!(children + .iter() + .map(Distribution::try_from) + .collect::>>()); let children_borrowed = children.iter().collect::>(); let result = rresult_return!(expr.propagate_statistics(&parent, &children_borrowed)); + let result = rresult_return!(result + .map(|dists| dists + .iter() + .map(FFI_Distribution::try_from) + .collect::>>()) + .transpose()); - RResult::ROk( - result - .map(|v| v.into_iter().map(Into::into).collect()) - .into(), - ) + RResult::ROk(result.into()) } unsafe extern "C" fn get_properties_fn_wrapper( @@ -508,7 +521,8 @@ impl PhysicalExpr for ForeignPhysicalExpr { fn evaluate(&self, batch: &RecordBatch) -> Result { unsafe { let batch = df_result!(record_batch_to_wrapped_array(batch.clone()))?; - df_result!((self.expr.evaluate)(&self.expr, batch)).and_then(ColumnarValue::try_from) + df_result!((self.expr.evaluate)(&self.expr, batch)) + .and_then(ColumnarValue::try_from) } } @@ -597,11 +611,12 @@ impl PhysicalExpr for ForeignPhysicalExpr { unsafe { let children = children .iter() - .map(|dist| FFI_Distribution::from(*dist)) - .collect::>(); - df_result!( - (self.expr.evaluate_statistics)(&self.expr, &children).map(Into::into) - ) + .map(|dist| FFI_Distribution::try_from(*dist)) + .collect::>>()?; + + let result = + df_result!((self.expr.evaluate_statistics)(&self.expr, &children))?; + Distribution::try_from(&result) } } @@ -611,18 +626,25 @@ impl PhysicalExpr for ForeignPhysicalExpr { children: &[&Distribution], ) -> Result>> { unsafe { - let parent = parent.into(); + let parent = FFI_Distribution::try_from(parent)?; let children = children .iter() - .map(|dist| FFI_Distribution::from(*dist)) - .collect::>(); + .map(|dist| FFI_Distribution::try_from(*dist)) + .collect::>>()?; let result = df_result!((self.expr.propagate_statistics)( &self.expr, &parent, &children ))?; - Ok(result - .map(|dists| dists.into_iter().map(Into::into).collect()) - .into()) + let result: Option>> = result + .map(|dists| { + dists + .iter() + .map(Distribution::try_from) + .collect::>>() + }) + .into(); + + result.transpose() } } From b0d6872ee6f26acd8afd08ee0fef6180df1dc979 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 21 Oct 2025 07:43:22 -0400 Subject: [PATCH 07/69] implement FFI_ExprProperties --- datafusion/ffi/src/expr/expr_properties.rs | 93 +++++++++++++++++++--- datafusion/ffi/src/physical_expr/mod.rs | 12 +-- 2 files changed, 86 insertions(+), 19 deletions(-) diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs index d75c6eab8a8a..6f3280d2c93d 100644 --- a/datafusion/ffi/src/expr/expr_properties.rs +++ b/datafusion/ffi/src/expr/expr_properties.rs @@ -1,32 +1,99 @@ use abi_stable::StableAbi; -use datafusion::logical_expr::sort_properties::ExprProperties; +use arrow_schema::SortOptions; +use datafusion::logical_expr::interval_arithmetic::Interval; +use datafusion::logical_expr::sort_properties::{ExprProperties, SortProperties}; +use datafusion_common::DataFusionError; +use crate::expr::interval::FFI_Interval; #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] pub struct FFI_ExprProperties { + pub sort_properties: FFI_SortProperties, + pub range: FFI_Interval, + pub preserves_lex_ordering: bool, } -impl From<&ExprProperties> for FFI_ExprProperties { - fn from(value: &ExprProperties) -> Self { - todo!() +impl TryFrom<&ExprProperties> for FFI_ExprProperties { + type Error = DataFusionError; + fn try_from(value: &ExprProperties) -> Result { + let sort_properties = (&value.sort_properties).into(); + let range = (&value.range).try_into()?; + + Ok(FFI_ExprProperties { + sort_properties, + range, + preserves_lex_ordering: value.preserves_lex_ordering, + }) } } -impl From for FFI_ExprProperties { - fn from(value: ExprProperties) -> Self { - FFI_ExprProperties::from(&value) +impl TryFrom<&FFI_ExprProperties> for ExprProperties { + type Error = DataFusionError; + fn try_from(value: &FFI_ExprProperties) -> Result { + let sort_properties = (&value.sort_properties).into(); + let range = (&value.range).try_into()?; + Ok(ExprProperties { + sort_properties, + range, + preserves_lex_ordering: value.preserves_lex_ordering, + }) + } +} + + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub enum FFI_SortProperties { + Ordered(FFI_SortOptions), + Unordered, + Singleton, +} + +impl From<&SortProperties> for FFI_SortProperties { + fn from(value: &SortProperties) -> Self { + match value { + SortProperties::Unordered => FFI_SortProperties::Unordered, + SortProperties::Singleton => FFI_SortProperties::Singleton, + SortProperties::Ordered(o) => FFI_SortProperties::Ordered(o.into()), + } } } -impl From<&FFI_ExprProperties> for ExprProperties { - fn from(value: &FFI_ExprProperties) -> Self { - todo!() +impl From<&FFI_SortProperties> for SortProperties { + fn from(value: &FFI_SortProperties) -> Self { + match value { + FFI_SortProperties::Unordered => SortProperties::Unordered, + FFI_SortProperties::Singleton => SortProperties::Singleton, + FFI_SortProperties::Ordered(o) => SortProperties::Ordered(o.into()), + } + } +} + + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_SortOptions { + pub descending: bool, + pub nulls_first: bool, +} + +impl From<&SortOptions> for FFI_SortOptions { + fn from(value: &SortOptions) -> Self { + Self { + descending: value.descending, + nulls_first: value.nulls_first, + } } } -impl From for ExprProperties { - fn from(value: FFI_ExprProperties) -> Self { - ExprProperties::from(&value) +impl From<&FFI_SortOptions> for SortOptions { + fn from(value: &FFI_SortOptions) -> Self { + Self { + descending: value.descending, + nulls_first: value.nulls_first, + } } } \ No newline at end of file diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index abb3abb544f6..68f818689386 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -345,11 +345,11 @@ unsafe extern "C" fn get_properties_fn_wrapper( children: &RVec, ) -> FFIResult { let expr = expr.inner(); - let children = children + let children = rresult_return!(children .iter() - .map(|child| child.into()) - .collect::>(); - rresult!(expr.get_properties(&children).map(Into::into)) + .map(ExprProperties::try_from) + .collect::>>()); + rresult!(expr.get_properties(&children).and_then(|p| FFI_ExprProperties::try_from(&p))) } unsafe extern "C" fn fmt_sql_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFIResult { @@ -650,8 +650,8 @@ impl PhysicalExpr for ForeignPhysicalExpr { fn get_properties(&self, children: &[ExprProperties]) -> Result { unsafe { - let children = children.iter().map(Into::into).collect::>(); - df_result!((self.expr.get_properties)(&self.expr, &children).map(Into::into)) + let children = children.into_iter().map(FFI_ExprProperties::try_from).collect::>>()?; + df_result!((self.expr.get_properties)(&self.expr, &children)).and_then(|p| ExprProperties::try_from(&p)) } } From 9d1294e3eadefd1a95d9140f49d6a5771745e864 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 21 Oct 2025 07:58:12 -0400 Subject: [PATCH 08/69] FFI_PhysicalExpr implemented with all dependencies --- datafusion/ffi/src/expr/columnar_value.rs | 21 ++++---- datafusion/ffi/src/expr/distribution.rs | 14 ++--- datafusion/ffi/src/expr/expr_properties.rs | 7 +-- datafusion/ffi/src/expr/interval.rs | 5 +- datafusion/ffi/src/expr/util.rs | 9 ++-- datafusion/ffi/src/physical_expr/mod.rs | 59 ++++++++-------------- 6 files changed, 42 insertions(+), 73 deletions(-) diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs index eb46540591f6..626dd37dd836 100644 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -1,12 +1,9 @@ use crate::arrow_wrappers::WrappedArray; +use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use arrow::ffi::FFI_ArrowArray; -use arrow_schema::ArrowError; use datafusion::logical_expr::ColumnarValue; use datafusion_common::DataFusionError; -use prost::Message; -use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; #[repr(C)] #[derive(Debug, StableAbi)] @@ -21,11 +18,11 @@ impl TryFrom for FFI_ColumnarValue { fn try_from(value: ColumnarValue) -> Result { Ok(match value { ColumnarValue::Array(v) => { - FFI_ColumnarValue::Array({ WrappedArray::try_from(&v)? }) + FFI_ColumnarValue::Array(WrappedArray::try_from(&v)?) + } + ColumnarValue::Scalar(v) => { + FFI_ColumnarValue::Scalar(scalar_value_to_rvec_u8(&v)?) } - ColumnarValue::Scalar(v) => FFI_ColumnarValue::Scalar({ - scalar_value_to_rvec_u8(&v)? - }), }) } } @@ -34,10 +31,10 @@ impl TryFrom for ColumnarValue { type Error = DataFusionError; fn try_from(value: FFI_ColumnarValue) -> Result { Ok(match value { - FFI_ColumnarValue::Array(v) => ColumnarValue::Array({ v.try_into()? }), - FFI_ColumnarValue::Scalar(v) => ColumnarValue::Scalar({ - rvec_u8_to_scalar_value(&v)? - }), + FFI_ColumnarValue::Array(v) => ColumnarValue::Array(v.try_into()?), + FFI_ColumnarValue::Scalar(v) => { + ColumnarValue::Scalar(rvec_u8_to_scalar_value(&v)?) + } }) } } diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs index e6144b69569f..d5a924b6b399 100644 --- a/datafusion/ffi/src/expr/distribution.rs +++ b/datafusion/ffi/src/expr/distribution.rs @@ -2,13 +2,11 @@ use crate::expr::interval::FFI_Interval; use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion::logical_expr::interval_arithmetic::Interval; use datafusion::logical_expr::statistics::{ BernoulliDistribution, Distribution, ExponentialDistribution, GaussianDistribution, GenericDistribution, UniformDistribution, }; -use datafusion_common::{DataFusionError, ScalarValue}; -use prost::Message; +use datafusion_common::DataFusionError; #[repr(C)] #[derive(Debug, StableAbi)] @@ -41,10 +39,10 @@ impl TryFrom<&FFI_Distribution> for Distribution { fn try_from(value: &FFI_Distribution) -> Result { match value { FFI_Distribution::Uniform(d) => d.try_into(), - FFI_Distribution::Exponential(d) => d.try_into(), - FFI_Distribution::Gaussian(d) => d.try_into(), - FFI_Distribution::Bernoulli(d) => d.try_into(), - FFI_Distribution::Generic(d) => d.try_into(), + FFI_Distribution::Exponential(d) => d.try_into(), + FFI_Distribution::Gaussian(d) => d.try_into(), + FFI_Distribution::Bernoulli(d) => d.try_into(), + FFI_Distribution::Generic(d) => d.try_into(), } } } @@ -148,8 +146,6 @@ impl TryFrom<&GenericDistribution> for FFI_GenericDistribution { } } - - impl TryFrom<&FFI_UniformDistribution> for Distribution { type Error = DataFusionError; fn try_from(value: &FFI_UniformDistribution) -> Result { diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs index 6f3280d2c93d..c30dc0762265 100644 --- a/datafusion/ffi/src/expr/expr_properties.rs +++ b/datafusion/ffi/src/expr/expr_properties.rs @@ -1,9 +1,8 @@ +use crate::expr::interval::FFI_Interval; use abi_stable::StableAbi; use arrow_schema::SortOptions; -use datafusion::logical_expr::interval_arithmetic::Interval; use datafusion::logical_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_common::DataFusionError; -use crate::expr::interval::FFI_Interval; #[repr(C)] #[derive(Debug, StableAbi)] @@ -41,7 +40,6 @@ impl TryFrom<&FFI_ExprProperties> for ExprProperties { } } - #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] @@ -71,7 +69,6 @@ impl From<&FFI_SortProperties> for SortProperties { } } - #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] @@ -96,4 +93,4 @@ impl From<&FFI_SortOptions> for SortOptions { nulls_first: value.nulls_first, } } -} \ No newline at end of file +} diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs index 1a43ea514050..04f653269ec8 100644 --- a/datafusion/ffi/src/expr/interval.rs +++ b/datafusion/ffi/src/expr/interval.rs @@ -1,11 +1,8 @@ +use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use arrow::ffi::FFI_ArrowArray; -use arrow_schema::ArrowError; use datafusion::logical_expr::interval_arithmetic::Interval; use datafusion_common::DataFusionError; -use prost::Message; -use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/expr/util.rs b/datafusion/ffi/src/expr/util.rs index 575fe7fcd9f4..d383f257d618 100644 --- a/datafusion/ffi/src/expr/util.rs +++ b/datafusion/ffi/src/expr/util.rs @@ -1,6 +1,6 @@ use abi_stable::std_types::RVec; +use datafusion_common::{exec_datafusion_err, Result, ScalarValue}; use prost::Message; -use datafusion_common::{exec_datafusion_err, DataFusionError, Result, ScalarValue}; pub fn scalar_value_to_rvec_u8(value: &ScalarValue) -> Result> { let value: datafusion_proto_common::ScalarValue = value.try_into()?; @@ -8,9 +8,10 @@ pub fn scalar_value_to_rvec_u8(value: &ScalarValue) -> Result> { } pub fn rvec_u8_to_scalar_value(value: &RVec) -> Result { - let value = datafusion_proto_common::ScalarValue::decode(value.as_ref()) .map_err(|err| exec_datafusion_err!("{err}"))?; - (&value).try_into().map_err(|err| exec_datafusion_err!("{err}")) -} \ No newline at end of file + (&value) + .try_into() + .map_err(|err| exec_datafusion_err!("{err}")) +} diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index 68f818689386..788351a8f725 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -16,7 +16,6 @@ // under the License. use crate::arrow_wrappers::WrappedArray; -use crate::execution_plan::ExecutionPlanPrivateData; use crate::expr::columnar_value::FFI_ColumnarValue; use crate::expr::distribution::FFI_Distribution; use crate::expr::expr_properties::FFI_ExprProperties; @@ -24,18 +23,8 @@ use crate::expr::interval::FFI_Interval; use crate::record_batch_stream::{ record_batch_to_wrapped_array, wrapped_array_to_record_batch, }; -use crate::table_provider::FFI_TableProvider; -use crate::udaf::ForeignAggregateUDF; use crate::util::FFIResult; -use crate::{ - arrow_wrappers::WrappedSchema, - df_result, - execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}, - insert_op::FFI_InsertOp, - rresult, rresult_return, - table_source::FFI_TableType, -}; -use abi_stable::pmr::RSlice; +use crate::{arrow_wrappers::WrappedSchema, df_result, rresult, rresult_return}; use abi_stable::std_types::RResult; use abi_stable::{ std_types::{ROption, RString, RVec}, @@ -43,32 +32,19 @@ use abi_stable::{ }; use arrow::array::{ArrayRef, BooleanArray, RecordBatch}; use arrow::datatypes::SchemaRef; -use arrow::ffi::FFI_ArrowArray; use arrow_schema::ffi::FFI_ArrowSchema; use arrow_schema::{DataType, Field, FieldRef, Schema}; -use async_ffi::{FfiFuture, FutureExt}; -use async_trait::async_trait; -use datafusion::logical_expr::expr_rewriter::unalias; use datafusion::logical_expr::ColumnarValue; use datafusion::logical_expr_common::interval_arithmetic::Interval; use datafusion::logical_expr_common::sort_properties::ExprProperties; use datafusion::logical_expr_common::statistics::Distribution; use datafusion::physical_expr::PhysicalExpr; -use datafusion::physical_expr_common::physical_expr::{fmt_sql, DynEq}; -use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_expr_common::physical_expr::fmt_sql; +use datafusion_common::exec_datafusion_err; use datafusion_common::Result; -use datafusion_common::{exec_datafusion_err, DataFusionError}; -use datafusion_proto::{ - logical_plan::{ - from_proto::parse_exprs, to_proto::serialize_exprs, DefaultLogicalExtensionCodec, - }, - protobuf::LogicalExprList, -}; -use prost::Message; use std::fmt::{Display, Formatter}; use std::hash::{DefaultHasher, Hash, Hasher}; use std::{any::Any, ffi::c_void, sync::Arc}; -use tokio::runtime::Handle; #[repr(C)] #[derive(Debug, StableAbi)] @@ -180,7 +156,7 @@ unsafe extern "C" fn data_type_fn_wrapper( input_schema: WrappedSchema, ) -> FFIResult { let expr = expr.inner(); - let schema: SchemaRef = rresult_return!(input_schema.try_into()); + let schema: SchemaRef = input_schema.into(); let data_type = expr .data_type(&schema) .and_then(|dt| FFI_ArrowSchema::try_from(dt).map_err(Into::into)) @@ -193,7 +169,7 @@ unsafe extern "C" fn nullable_fn_wrapper( input_schema: WrappedSchema, ) -> FFIResult { let expr = expr.inner(); - let schema: SchemaRef = rresult_return!(input_schema.try_into()); + let schema: SchemaRef = input_schema.into(); rresult!(expr.nullable(&schema)) } @@ -213,7 +189,7 @@ unsafe extern "C" fn return_field_fn_wrapper( input_schema: WrappedSchema, ) -> FFIResult { let expr = expr.inner(); - let schema: SchemaRef = rresult_return!(input_schema.try_into()); + let schema: SchemaRef = input_schema.into(); rresult!(expr .return_field(&schema) .and_then(|f| FFI_ArrowSchema::try_from(&f).map_err(Into::into)) @@ -313,7 +289,9 @@ unsafe extern "C" fn evaluate_statistics_fn_wrapper( .map(Distribution::try_from) .collect::>>()); let children_borrowed = children.iter().collect::>(); - rresult!(expr.evaluate_statistics(&children_borrowed).and_then(|dist| FFI_Distribution::try_from(&dist))) + rresult!(expr + .evaluate_statistics(&children_borrowed) + .and_then(|dist| FFI_Distribution::try_from(&dist))) } unsafe extern "C" fn propagate_statistics_fn_wrapper( @@ -349,7 +327,9 @@ unsafe extern "C" fn get_properties_fn_wrapper( .iter() .map(ExprProperties::try_from) .collect::>>()); - rresult!(expr.get_properties(&children).and_then(|p| FFI_ExprProperties::try_from(&p))) + rresult!(expr + .get_properties(&children) + .and_then(|p| FFI_ExprProperties::try_from(&p))) } unsafe extern "C" fn fmt_sql_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFIResult { @@ -382,7 +362,7 @@ unsafe extern "C" fn display_fn_wrapper(expr: &FFI_PhysicalExpr) -> RString { } unsafe extern "C" fn hash_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 { - let mut expr = expr.inner(); + let expr = expr.inner(); let mut hasher = DefaultHasher::new(); expr.hash(&mut hasher); hasher.finish() @@ -560,10 +540,7 @@ impl PhysicalExpr for ForeignPhysicalExpr { children: Vec>, ) -> Result> { unsafe { - let children = children - .into_iter() - .map(|expr| FFI_PhysicalExpr::from(expr)) - .collect(); + let children = children.into_iter().map(FFI_PhysicalExpr::from).collect(); df_result!((self.expr.new_with_children)(&self.expr, &children) .map(|expr| Arc::new(ForeignPhysicalExpr::from(expr)))) } @@ -650,8 +627,12 @@ impl PhysicalExpr for ForeignPhysicalExpr { fn get_properties(&self, children: &[ExprProperties]) -> Result { unsafe { - let children = children.into_iter().map(FFI_ExprProperties::try_from).collect::>>()?; - df_result!((self.expr.get_properties)(&self.expr, &children)).and_then(|p| ExprProperties::try_from(&p)) + let children = children + .iter() + .map(FFI_ExprProperties::try_from) + .collect::>>()?; + df_result!((self.expr.get_properties)(&self.expr, &children)) + .and_then(|p| ExprProperties::try_from(&p)) } } From 5344889daf9841fd9d8f12911f6568ca50707c90 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 21 Oct 2025 08:33:37 -0400 Subject: [PATCH 09/69] Switch accumulator to use ffi physical expr --- datafusion/ffi/src/physical_expr/mod.rs | 2 + datafusion/ffi/src/physical_expr/sort.rs | 32 +++++++++ datafusion/ffi/src/udaf/accumulator_args.rs | 79 ++++++++------------- 3 files changed, 65 insertions(+), 48 deletions(-) create mode 100644 datafusion/ffi/src/physical_expr/sort.rs diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index 788351a8f725..df08f2f8c5ab 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +pub(crate) mod sort; + use crate::arrow_wrappers::WrappedArray; use crate::expr::columnar_value::FFI_ColumnarValue; use crate::expr::distribution::FFI_Distribution; diff --git a/datafusion/ffi/src/physical_expr/sort.rs b/datafusion/ffi/src/physical_expr/sort.rs new file mode 100644 index 000000000000..f357f77f609a --- /dev/null +++ b/datafusion/ffi/src/physical_expr/sort.rs @@ -0,0 +1,32 @@ +use crate::expr::expr_properties::FFI_SortOptions; +use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; +use abi_stable::StableAbi; +use arrow_schema::SortOptions; +use datafusion::physical_expr::PhysicalSortExpr; +use std::sync::Arc; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_PhysicalSortExpr { + pub expr: FFI_PhysicalExpr, + pub options: FFI_SortOptions, +} + +impl From<&PhysicalSortExpr> for FFI_PhysicalSortExpr { + fn from(value: &PhysicalSortExpr) -> Self { + let expr = FFI_PhysicalExpr::from(value.clone().expr); + let options = FFI_SortOptions::from(&value.options); + + Self { expr, options } + } +} + +impl From<&FFI_PhysicalSortExpr> for PhysicalSortExpr { + fn from(value: &FFI_PhysicalSortExpr) -> Self { + let expr = Arc::new(ForeignPhysicalExpr::from(value.expr.clone())); + let options = SortOptions::from(&value.options); + + Self { expr, options } + } +} diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 6ac0a0b21d2d..41d34d54da54 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -18,6 +18,8 @@ use std::sync::Arc; use crate::arrow_wrappers::WrappedSchema; +use crate::physical_expr::sort::FFI_PhysicalSortExpr; +use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; use abi_stable::{ std_types::{RString, RVec}, StableAbi, @@ -28,18 +30,7 @@ use datafusion::{ error::DataFusionError, logical_expr::function::AccumulatorArgs, physical_expr::{PhysicalExpr, PhysicalSortExpr}, - prelude::SessionContext, }; -use datafusion_common::exec_datafusion_err; -use datafusion_proto::{ - physical_plan::{ - from_proto::{parse_physical_exprs, parse_physical_sort_exprs}, - to_proto::{serialize_physical_exprs, serialize_physical_sort_exprs}, - DefaultPhysicalExtensionCodec, - }, - protobuf::PhysicalAggregateExprNode, -}; -use prost::Message; /// A stable struct for sharing [`AccumulatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding field @@ -50,9 +41,12 @@ use prost::Message; pub struct FFI_AccumulatorArgs { return_field: WrappedSchema, schema: WrappedSchema, + ignore_nulls: bool, + is_distinct: bool, is_reversed: bool, name: RString, - physical_expr_def: RVec, + order_bys: RVec, + exprs: RVec, } impl TryFrom> for FFI_AccumulatorArgs { @@ -63,29 +57,28 @@ impl TryFrom> for FFI_AccumulatorArgs { WrappedSchema(FFI_ArrowSchema::try_from(args.return_field.as_ref())?); let schema = WrappedSchema(FFI_ArrowSchema::try_from(args.schema)?); - let codec = DefaultPhysicalExtensionCodec {}; - let ordering_req = - serialize_physical_sort_exprs(args.order_bys.to_owned(), &codec)?; - - let expr = serialize_physical_exprs(args.exprs, &codec)?; + let exprs = args + .exprs + .iter() + .map(Arc::clone) + .map(FFI_PhysicalExpr::from) + .collect(); - let physical_expr_def = PhysicalAggregateExprNode { - expr, - ordering_req, - distinct: args.is_distinct, - ignore_nulls: args.ignore_nulls, - fun_definition: None, - aggregate_function: None, - human_display: args.name.to_string(), - }; - let physical_expr_def = physical_expr_def.encode_to_vec().into(); + let order_bys = args + .order_bys + .iter() + .map(FFI_PhysicalSortExpr::from) + .collect(); Ok(Self { return_field, schema, is_reversed: args.is_reversed, + ignore_nulls: args.ignore_nulls, + is_distinct: args.is_distinct, name: args.name.into(), - physical_expr_def, + order_bys, + exprs, }) } } @@ -110,28 +103,18 @@ impl TryFrom for ForeignAccumulatorArgs { type Error = DataFusionError; fn try_from(value: FFI_AccumulatorArgs) -> Result { - let proto_def = PhysicalAggregateExprNode::decode( - value.physical_expr_def.as_ref(), - ) - .map_err(|e| { - exec_datafusion_err!("Failed to decode PhysicalAggregateExprNode: {e}") - })?; + let exprs = value + .exprs + .into_iter() + .map(|expr| { + Arc::new(ForeignPhysicalExpr::from(expr)) as Arc + }) + .collect(); let return_field = Arc::new((&value.return_field.0).try_into()?); let schema = Schema::try_from(&value.schema.0)?; - let default_ctx = SessionContext::new(); - let task_ctx = default_ctx.task_ctx(); - let codex = DefaultPhysicalExtensionCodec {}; - - let order_bys = parse_physical_sort_exprs( - &proto_def.ordering_req, - &task_ctx, - &schema, - &codex, - )?; - - let exprs = parse_physical_exprs(&proto_def.expr, &task_ctx, &schema, &codex)?; + let order_bys = value.order_bys.iter().map(PhysicalSortExpr::from).collect(); let expr_fields = exprs .iter() @@ -142,11 +125,11 @@ impl TryFrom for ForeignAccumulatorArgs { return_field, schema, expr_fields, - ignore_nulls: proto_def.ignore_nulls, + ignore_nulls: value.ignore_nulls, order_bys, is_reversed: value.is_reversed, name: value.name.to_string(), - is_distinct: proto_def.distinct, + is_distinct: value.is_distinct, exprs, }) } From f536238644c3764e347c9e23ed6ec2a098c56ea3 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 21 Oct 2025 08:41:33 -0400 Subject: [PATCH 10/69] Switch partition evaluator to use ffi physical expr --- .../ffi/src/udwf/partition_evaluator_args.rs | 35 ++++++------------- 1 file changed, 10 insertions(+), 25 deletions(-) diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index cd2641256437..a42c828665e0 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -18,6 +18,7 @@ use std::{collections::HashMap, sync::Arc}; use crate::arrow_wrappers::WrappedSchema; +use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; use abi_stable::{std_types::RVec, StableAbi}; use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef}, @@ -29,17 +30,7 @@ use datafusion::{ error::{DataFusionError, Result}, logical_expr::function::PartitionEvaluatorArgs, physical_plan::{expressions::Column, PhysicalExpr}, - prelude::SessionContext, }; -use datafusion_common::exec_datafusion_err; -use datafusion_proto::{ - physical_plan::{ - from_proto::parse_physical_expr, to_proto::serialize_physical_exprs, - DefaultPhysicalExtensionCodec, - }, - protobuf::PhysicalExprNode, -}; -use prost::Message; /// A stable struct for sharing [`PartitionEvaluatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding function @@ -48,7 +39,7 @@ use prost::Message; #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] pub struct FFI_PartitionEvaluatorArgs { - input_exprs: RVec>, + input_exprs: RVec, input_fields: RVec, is_reversed: bool, ignore_nulls: bool, @@ -96,10 +87,11 @@ impl TryFrom> for FFI_PartitionEvaluatorArgs { let schema = Arc::new(Schema::new(fields)); - let codec = DefaultPhysicalExtensionCodec {}; - let input_exprs = serialize_physical_exprs(args.input_exprs(), &codec)? - .into_iter() - .map(|expr_node| expr_node.encode_to_vec().into()) + let input_exprs = args + .input_exprs() + .iter() + .map(Arc::clone) + .map(FFI_PhysicalExpr::from) .collect(); let input_fields = args @@ -136,22 +128,15 @@ impl TryFrom for ForeignPartitionEvaluatorArgs { type Error = DataFusionError; fn try_from(value: FFI_PartitionEvaluatorArgs) -> Result { - let default_ctx = SessionContext::new(); - let codec = DefaultPhysicalExtensionCodec {}; - let schema: SchemaRef = value.schema.into(); let input_exprs = value .input_exprs .into_iter() - .map(|input_expr_bytes| PhysicalExprNode::decode(input_expr_bytes.as_ref())) - .collect::, prost::DecodeError>>() - .map_err(|e| exec_datafusion_err!("Failed to decode PhysicalExprNode: {e}"))? - .iter() - .map(|expr_node| { - parse_physical_expr(expr_node, &default_ctx.task_ctx(), &schema, &codec) + .map(|expr| { + Arc::new(ForeignPhysicalExpr::from(expr)) as Arc }) - .collect::>>()?; + .collect::>(); let input_fields = input_exprs .iter() From 99d3cd9aad81319b1e5e68434c06ab7f81227037 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 21 Oct 2025 08:46:27 -0400 Subject: [PATCH 11/69] add license text --- datafusion/ffi/src/expr/columnar_value.rs | 17 +++++++++++++++++ datafusion/ffi/src/expr/distribution.rs | 17 +++++++++++++++++ datafusion/ffi/src/expr/expr_properties.rs | 17 +++++++++++++++++ datafusion/ffi/src/expr/interval.rs | 17 +++++++++++++++++ datafusion/ffi/src/expr/mod.rs | 17 +++++++++++++++++ datafusion/ffi/src/expr/util.rs | 17 +++++++++++++++++ datafusion/ffi/src/physical_expr/sort.rs | 17 +++++++++++++++++ 7 files changed, 119 insertions(+) diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs index 626dd37dd836..c20f599dc356 100644 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -1,3 +1,20 @@ +// 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::arrow_wrappers::WrappedArray; use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs index d5a924b6b399..1a773e414092 100644 --- a/datafusion/ffi/src/expr/distribution.rs +++ b/datafusion/ffi/src/expr/distribution.rs @@ -1,3 +1,20 @@ +// 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::expr::interval::FFI_Interval; use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs index c30dc0762265..4f873a61bcc6 100644 --- a/datafusion/ffi/src/expr/expr_properties.rs +++ b/datafusion/ffi/src/expr/expr_properties.rs @@ -1,3 +1,20 @@ +// 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::expr::interval::FFI_Interval; use abi_stable::StableAbi; use arrow_schema::SortOptions; diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs index 04f653269ec8..3eb65df73b84 100644 --- a/datafusion/ffi/src/expr/interval.rs +++ b/datafusion/ffi/src/expr/interval.rs @@ -1,3 +1,20 @@ +// 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::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; diff --git a/datafusion/ffi/src/expr/mod.rs b/datafusion/ffi/src/expr/mod.rs index 9bfdc58c40c0..717b7b4a2a16 100644 --- a/datafusion/ffi/src/expr/mod.rs +++ b/datafusion/ffi/src/expr/mod.rs @@ -1,3 +1,20 @@ +// 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. + pub mod columnar_value; pub mod distribution; pub mod expr_properties; diff --git a/datafusion/ffi/src/expr/util.rs b/datafusion/ffi/src/expr/util.rs index d383f257d618..f8ac973261a2 100644 --- a/datafusion/ffi/src/expr/util.rs +++ b/datafusion/ffi/src/expr/util.rs @@ -1,3 +1,20 @@ +// 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 abi_stable::std_types::RVec; use datafusion_common::{exec_datafusion_err, Result, ScalarValue}; use prost::Message; diff --git a/datafusion/ffi/src/physical_expr/sort.rs b/datafusion/ffi/src/physical_expr/sort.rs index f357f77f609a..75ebcb6b4337 100644 --- a/datafusion/ffi/src/physical_expr/sort.rs +++ b/datafusion/ffi/src/physical_expr/sort.rs @@ -1,3 +1,20 @@ +// 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::expr::expr_properties::FFI_SortOptions; use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; use abi_stable::StableAbi; From d8c6c9d993ca9e2be6d88b8774322ca252ee2d15 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Thu, 16 Oct 2025 15:57:15 -0400 Subject: [PATCH 12/69] Initial commit for FFI_FunctionRegistry --- Cargo.lock | 1 + datafusion/ffi/Cargo.toml | 1 + datafusion/ffi/src/function_registry.rs | 461 ++++++++++++++++++++++++ datafusion/ffi/src/lib.rs | 1 + 4 files changed, 464 insertions(+) create mode 100644 datafusion/ffi/src/function_registry.rs diff --git a/Cargo.lock b/Cargo.lock index d712eecfcc72..0f84a5c24783 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2293,6 +2293,7 @@ dependencies = [ "async-trait", "datafusion", "datafusion-common", + "datafusion-expr", "datafusion-functions-aggregate-common", "datafusion-proto", "datafusion-proto-common", diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index 3ac08180fb68..90551eaad374 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -47,6 +47,7 @@ arrow-schema = { workspace = true } async-ffi = { version = "0.5.0", features = ["abi_stable"] } async-trait = { workspace = true } datafusion = { workspace = true, default-features = false } +datafusion-expr = { workspace = true } datafusion-common = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } datafusion-proto = { workspace = true } diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs new file mode 100644 index 000000000000..187b66b0d4f3 --- /dev/null +++ b/datafusion/ffi/src/function_registry.rs @@ -0,0 +1,461 @@ +// 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::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; +use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; +use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; +use crate::{df_result, rresult_return}; +use abi_stable::{ + std_types::{ROption, RResult, RString, RVec}, + StableAbi, +}; +use datafusion_common::not_impl_err; +use datafusion_expr::expr_rewriter::FunctionRewrite; +use datafusion_expr::planner::ExprPlanner; +use datafusion_expr::registry::FunctionRegistry; +use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use std::collections::HashSet; +use std::sync::Mutex; +use std::{ffi::c_void, sync::Arc}; + +/// A stable struct for sharing [`FunctionRegistry`] across FFI boundaries. +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_FunctionRegistry { + pub udfs: unsafe extern "C" fn(&Self) -> RVec, + pub udafs: unsafe extern "C" fn(&Self) -> RVec, + pub udwfs: unsafe extern "C" fn(&Self) -> RVec, + + pub udf: + unsafe extern "C" fn(&Self, name: RString) -> RResult, + pub udaf: + unsafe extern "C" fn(&Self, name: RString) -> RResult, + pub udwf: + unsafe extern "C" fn(&Self, name: RString) -> RResult, + + pub register_udf: unsafe extern "C" fn( + &mut Self, + udf: FFI_ScalarUDF, + ) + -> RResult, RString>, + pub register_udaf: + unsafe extern "C" fn( + &mut Self, + udf: FFI_AggregateUDF, + ) -> RResult, RString>, + pub register_udwf: unsafe extern "C" fn( + &mut Self, + udf: FFI_WindowUDF, + ) + -> RResult, RString>, + + pub deregister_udf: unsafe extern "C" fn( + &mut Self, + name: RString, + ) + -> RResult, RString>, + pub deregister_udaf: + unsafe extern "C" fn( + &mut Self, + name: RString, + ) -> RResult, RString>, + pub deregister_udwf: unsafe extern "C" fn( + &mut Self, + name: RString, + ) + -> RResult, RString>, + + /// Used to create a clone on the provider of the registry. This should + /// only need to be called by the receiver of the plan. + pub clone: unsafe extern "C" fn(plan: &Self) -> Self, + + /// Release the memory of the private data when it is no longer being used. + pub release: unsafe extern "C" fn(arg: &mut Self), + + /// Return the major DataFusion version number of this registry. + pub version: unsafe extern "C" fn() -> u64, + + /// Internal data. This is only to be accessed by the provider of the plan. + /// A [`ForeignFunctionRegistry`] should never attempt to access this data. + pub private_data: *mut c_void, +} + +unsafe impl Send for FFI_FunctionRegistry {} +unsafe impl Sync for FFI_FunctionRegistry {} + +struct RegistryPrivateData { + registry: Arc>, +} + +impl FFI_FunctionRegistry { + unsafe fn inner(&self) -> &Arc> { + let private_data = self.private_data as *const RegistryPrivateData; + &(*private_data).registry + } +} + +unsafe extern "C" fn udfs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { + let Ok(registry) = registry.inner().lock() else { + log::error!("udwfs_fn_wrapper is unable to get a mutex lock"); + return RVec::new(); + }; + let udfs = registry.udfs(); + udfs.into_iter().map(|s| s.into()).collect() +} +unsafe extern "C" fn udafs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { + let Ok(registry) = registry.inner().lock() else { + log::error!("udwfs_fn_wrapper is unable to get a mutex lock"); + return RVec::new(); + }; + let udafs = registry.udafs(); + udafs.into_iter().map(|s| s.into()).collect() +} +unsafe extern "C" fn udwfs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { + let Ok(registry) = registry.inner().lock() else { + log::error!("udwfs_fn_wrapper is unable to get a mutex lock"); + return RVec::new(); + }; + let udwfs = registry.udwfs(); + udwfs.into_iter().map(|s| s.into()).collect() +} + +unsafe extern "C" fn udf_fn_wrapper( + registry: &FFI_FunctionRegistry, + name: RString, +) -> RResult { + let registry = rresult_return!(registry.inner().lock()); + let udf = rresult_return!(registry.udf(name.as_str())); + RResult::ROk(FFI_ScalarUDF::from(udf)) +} +unsafe extern "C" fn udaf_fn_wrapper( + registry: &FFI_FunctionRegistry, + name: RString, +) -> RResult { + let registry = rresult_return!(registry.inner().lock()); + let udaf = rresult_return!(registry.udaf(name.as_str())); + RResult::ROk(FFI_AggregateUDF::from(udaf)) +} +unsafe extern "C" fn udwf_fn_wrapper( + registry: &FFI_FunctionRegistry, + name: RString, +) -> RResult { + let registry = rresult_return!(registry.inner().lock()); + let udwf = rresult_return!(registry.udwf(name.as_str())); + RResult::ROk(FFI_WindowUDF::from(udwf)) +} + +unsafe extern "C" fn register_udf_fn_wrapper( + registry: &mut FFI_FunctionRegistry, + udf: FFI_ScalarUDF, +) -> RResult, RString> { + let udf: ForeignScalarUDF = rresult_return!((&udf).try_into()); + let mut registry = rresult_return!(registry.inner().lock()); + let udf = rresult_return!(registry.register_udf(Arc::new(udf.into()))) + .map(FFI_ScalarUDF::from); + RResult::ROk(udf.into()) +} +unsafe extern "C" fn register_udaf_fn_wrapper( + registry: &mut FFI_FunctionRegistry, + udaf: FFI_AggregateUDF, +) -> RResult, RString> { + let udaf: ForeignAggregateUDF = rresult_return!((&udaf).try_into()); + let mut registry = rresult_return!(registry.inner().lock()); + let udaf = rresult_return!(registry.register_udaf(Arc::new(udaf.into()))) + .map(FFI_AggregateUDF::from); + RResult::ROk(udaf.into()) +} +unsafe extern "C" fn register_udwf_fn_wrapper( + registry: &mut FFI_FunctionRegistry, + udwf: FFI_WindowUDF, +) -> RResult, RString> { + let udwf: ForeignWindowUDF = rresult_return!((&udwf).try_into()); + let mut registry = rresult_return!(registry.inner().lock()); + let udwf = rresult_return!(registry.register_udwf(Arc::new(udwf.into()))) + .map(FFI_WindowUDF::from); + RResult::ROk(udwf.into()) +} + +unsafe extern "C" fn deregister_udf_fn_wrapper( + registry: &mut FFI_FunctionRegistry, + name: RString, +) -> RResult, RString> { + let mut registry = rresult_return!(registry.inner().lock()); + let udf = + rresult_return!(registry.deregister_udf(name.as_str())).map(FFI_ScalarUDF::from); + RResult::ROk(udf.into()) +} +unsafe extern "C" fn deregister_udaf_fn_wrapper( + registry: &mut FFI_FunctionRegistry, + name: RString, +) -> RResult, RString> { + let mut registry = rresult_return!(registry.inner().lock()); + let udaf = rresult_return!(registry.deregister_udaf(name.as_str())) + .map(FFI_AggregateUDF::from); + RResult::ROk(udaf.into()) +} +unsafe extern "C" fn deregister_udwf_fn_wrapper( + registry: &mut FFI_FunctionRegistry, + name: RString, +) -> RResult, RString> { + let mut registry = rresult_return!(registry.inner().lock()); + let udwf = + rresult_return!(registry.deregister_udwf(name.as_str())).map(FFI_WindowUDF::from); + RResult::ROk(udwf.into()) +} + +unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_FunctionRegistry) { + let private_data = Box::from_raw(provider.private_data as *mut RegistryPrivateData); + drop(private_data); +} + +unsafe extern "C" fn clone_fn_wrapper( + provider: &FFI_FunctionRegistry, +) -> FFI_FunctionRegistry { + let old_private_data = provider.private_data as *const RegistryPrivateData; + + let private_data = Box::into_raw(Box::new(RegistryPrivateData { + registry: Arc::clone(&(*old_private_data).registry), + })) as *mut c_void; + + FFI_FunctionRegistry { + udfs: udfs_fn_wrapper, + udafs: udafs_fn_wrapper, + udwfs: udwfs_fn_wrapper, + + udf: udf_fn_wrapper, + udaf: udaf_fn_wrapper, + udwf: udwf_fn_wrapper, + + register_udf: register_udf_fn_wrapper, + register_udaf: register_udaf_fn_wrapper, + register_udwf: register_udwf_fn_wrapper, + + deregister_udf: deregister_udf_fn_wrapper, + deregister_udaf: deregister_udaf_fn_wrapper, + deregister_udwf: deregister_udwf_fn_wrapper, + + clone: clone_fn_wrapper, + release: release_fn_wrapper, + version: super::version, + private_data, + } +} + +impl Drop for FFI_FunctionRegistry { + fn drop(&mut self) { + unsafe { (self.release)(self) } + } +} + +impl FFI_FunctionRegistry { + /// Creates a new [`FFI_FunctionRegistry`]. + pub fn new(registry: Arc>) -> Self { + let private_data = Box::new(RegistryPrivateData { registry }); + + Self { + udfs: udfs_fn_wrapper, + udafs: udafs_fn_wrapper, + udwfs: udwfs_fn_wrapper, + + udf: udf_fn_wrapper, + udaf: udaf_fn_wrapper, + udwf: udwf_fn_wrapper, + + register_udf: register_udf_fn_wrapper, + register_udaf: register_udaf_fn_wrapper, + register_udwf: register_udwf_fn_wrapper, + + deregister_udf: deregister_udf_fn_wrapper, + deregister_udaf: deregister_udaf_fn_wrapper, + deregister_udwf: deregister_udwf_fn_wrapper, + + clone: clone_fn_wrapper, + release: release_fn_wrapper, + version: super::version, + private_data: Box::into_raw(private_data) as *mut c_void, + } + } +} + +/// This wrapper struct exists on the receiver side of the FFI interface, so it has +/// no guarantees about being able to access the data in `private_data`. Any functions +/// defined on this struct must only use the stable functions provided in +/// FFI_FunctionRegistry to interact with the foreign table provider. +#[derive(Debug)] +pub struct ForeignFunctionRegistry(FFI_FunctionRegistry); + +unsafe impl Send for ForeignFunctionRegistry {} +unsafe impl Sync for ForeignFunctionRegistry {} + +impl From<&FFI_FunctionRegistry> for ForeignFunctionRegistry { + fn from(provider: &FFI_FunctionRegistry) -> Self { + Self(provider.clone()) + } +} + +impl Clone for FFI_FunctionRegistry { + fn clone(&self) -> Self { + unsafe { (self.clone)(self) } + } +} + +impl FunctionRegistry for ForeignFunctionRegistry { + fn udfs(&self) -> HashSet { + let udfs = unsafe { (self.0.udfs)(&self.0) }; + + udfs.into_iter().map(String::from).collect() + } + + fn udafs(&self) -> HashSet { + let udafs = unsafe { (self.0.udafs)(&self.0) }; + + udafs.into_iter().map(String::from).collect() + } + + fn udwfs(&self) -> HashSet { + let udwfs = unsafe { (self.0.udwfs)(&self.0) }; + + udwfs.into_iter().map(String::from).collect() + } + + fn udf(&self, name: &str) -> datafusion_common::Result> { + let udf = df_result!(unsafe { (self.0.udf)(&self.0, name.into()) })?; + + let udf = ForeignScalarUDF::try_from(&udf)?; + Ok(Arc::new(udf.into())) + } + + fn udaf(&self, name: &str) -> datafusion_common::Result> { + let udaf = df_result!(unsafe { (self.0.udaf)(&self.0, name.into()) })?; + + let udaf = ForeignAggregateUDF::try_from(&udaf)?; + Ok(Arc::new(udaf.into())) + } + + fn udwf(&self, name: &str) -> datafusion_common::Result> { + let udwf = df_result!(unsafe { (self.0.udwf)(&self.0, name.into()) })?; + + let udwf = ForeignWindowUDF::try_from(&udwf)?; + Ok(Arc::new(udwf.into())) + } + + fn register_udf( + &mut self, + udf: Arc, + ) -> datafusion_common::Result>> { + let udf = FFI_ScalarUDF::from(udf); + let ROption::RSome(udf) = + df_result!(unsafe { (self.0.register_udf)(&mut self.0, udf) })? + else { + return Ok(None); + }; + let udf = ForeignScalarUDF::try_from(&udf)?; + + Ok(Some(Arc::new(ScalarUDF::from(udf)))) + } + + fn register_udaf( + &mut self, + udaf: Arc, + ) -> datafusion_common::Result>> { + let udaf = FFI_AggregateUDF::from(udaf); + let ROption::RSome(udaf) = + df_result!(unsafe { (self.0.register_udaf)(&mut self.0, udaf) })? + else { + return Ok(None); + }; + let udaf = ForeignAggregateUDF::try_from(&udaf)?; + + Ok(Some(Arc::new(AggregateUDF::from(udaf)))) + } + + fn register_udwf( + &mut self, + udwf: Arc, + ) -> datafusion_common::Result>> { + let udwf = FFI_WindowUDF::from(udwf); + let ROption::RSome(udwf) = + df_result!(unsafe { (self.0.register_udwf)(&mut self.0, udwf) })? + else { + return Ok(None); + }; + let udwf = ForeignWindowUDF::try_from(&udwf)?; + + Ok(Some(Arc::new(WindowUDF::from(udwf)))) + } + + fn deregister_udf( + &mut self, + name: &str, + ) -> datafusion_common::Result>> { + let ROption::RSome(udf) = + df_result!(unsafe { (self.0.deregister_udf)(&mut self.0, name.into()) })? + else { + return Ok(None); + }; + let udf = ForeignScalarUDF::try_from(&udf)?; + + Ok(Some(Arc::new(ScalarUDF::from(udf)))) + } + + fn deregister_udaf( + &mut self, + name: &str, + ) -> datafusion_common::Result>> { + let ROption::RSome(udaf) = + df_result!(unsafe { (self.0.deregister_udaf)(&mut self.0, name.into()) })? + else { + return Ok(None); + }; + let udwf = ForeignAggregateUDF::try_from(&udaf)?; + + Ok(Some(Arc::new(AggregateUDF::from(udwf)))) + } + + fn deregister_udwf( + &mut self, + name: &str, + ) -> datafusion_common::Result>> { + let ROption::RSome(udwf) = + df_result!(unsafe { (self.0.deregister_udwf)(&mut self.0, name.into()) })? + else { + return Ok(None); + }; + let udwf = ForeignWindowUDF::try_from(&udwf)?; + + Ok(Some(Arc::new(WindowUDF::from(udwf)))) + } + + fn register_function_rewrite( + &mut self, + _rewrite: Arc, + ) -> datafusion_common::Result<()> { + not_impl_err!("register_function_rewrite not implemented in FFI") + } + + fn expr_planners(&self) -> Vec> { + vec![] + } + + fn register_expr_planner( + &mut self, + _expr_planner: Arc, + ) -> datafusion_common::Result<()> { + not_impl_err!("register_function_rewrite not implemented in FFI") + } +} diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index 82afc4d2e456..f08185c4a34f 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -28,6 +28,7 @@ pub mod arrow_wrappers; pub mod catalog_provider; pub mod execution_plan; pub mod expr; +pub mod function_registry; pub mod insert_op; pub mod physical_expr; pub mod plan_properties; From a147c6ecca2b1f1c67ab1b8329c710c69fc1f7e4 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sat, 1 Nov 2025 08:41:20 -0400 Subject: [PATCH 13/69] Do not allow mutation of function registry via ffi --- datafusion/ffi/src/function_registry.rs | 216 +++--------------------- 1 file changed, 24 insertions(+), 192 deletions(-) diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 187b66b0d4f3..d0461c53f9eb 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -20,7 +20,7 @@ use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use crate::{df_result, rresult_return}; use abi_stable::{ - std_types::{ROption, RResult, RString, RVec}, + std_types::{RResult, RString, RVec}, StableAbi, }; use datafusion_common::not_impl_err; @@ -29,7 +29,6 @@ use datafusion_expr::planner::ExprPlanner; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use std::collections::HashSet; -use std::sync::Mutex; use std::{ffi::c_void, sync::Arc}; /// A stable struct for sharing [`FunctionRegistry`] across FFI boundaries. @@ -48,38 +47,6 @@ pub struct FFI_FunctionRegistry { pub udwf: unsafe extern "C" fn(&Self, name: RString) -> RResult, - pub register_udf: unsafe extern "C" fn( - &mut Self, - udf: FFI_ScalarUDF, - ) - -> RResult, RString>, - pub register_udaf: - unsafe extern "C" fn( - &mut Self, - udf: FFI_AggregateUDF, - ) -> RResult, RString>, - pub register_udwf: unsafe extern "C" fn( - &mut Self, - udf: FFI_WindowUDF, - ) - -> RResult, RString>, - - pub deregister_udf: unsafe extern "C" fn( - &mut Self, - name: RString, - ) - -> RResult, RString>, - pub deregister_udaf: - unsafe extern "C" fn( - &mut Self, - name: RString, - ) -> RResult, RString>, - pub deregister_udwf: unsafe extern "C" fn( - &mut Self, - name: RString, - ) - -> RResult, RString>, - /// Used to create a clone on the provider of the registry. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -99,38 +66,26 @@ unsafe impl Send for FFI_FunctionRegistry {} unsafe impl Sync for FFI_FunctionRegistry {} struct RegistryPrivateData { - registry: Arc>, + registry: Arc, } impl FFI_FunctionRegistry { - unsafe fn inner(&self) -> &Arc> { + unsafe fn inner(&self) -> &Arc { let private_data = self.private_data as *const RegistryPrivateData; &(*private_data).registry } } unsafe extern "C" fn udfs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { - let Ok(registry) = registry.inner().lock() else { - log::error!("udwfs_fn_wrapper is unable to get a mutex lock"); - return RVec::new(); - }; - let udfs = registry.udfs(); + let udfs = registry.inner().udfs(); udfs.into_iter().map(|s| s.into()).collect() } unsafe extern "C" fn udafs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { - let Ok(registry) = registry.inner().lock() else { - log::error!("udwfs_fn_wrapper is unable to get a mutex lock"); - return RVec::new(); - }; - let udafs = registry.udafs(); + let udafs = registry.inner().udafs(); udafs.into_iter().map(|s| s.into()).collect() } unsafe extern "C" fn udwfs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { - let Ok(registry) = registry.inner().lock() else { - log::error!("udwfs_fn_wrapper is unable to get a mutex lock"); - return RVec::new(); - }; - let udwfs = registry.udwfs(); + let udwfs = registry.inner().udwfs(); udwfs.into_iter().map(|s| s.into()).collect() } @@ -138,86 +93,24 @@ unsafe extern "C" fn udf_fn_wrapper( registry: &FFI_FunctionRegistry, name: RString, ) -> RResult { - let registry = rresult_return!(registry.inner().lock()); - let udf = rresult_return!(registry.udf(name.as_str())); + let udf = rresult_return!(registry.inner().udf(name.as_str())); RResult::ROk(FFI_ScalarUDF::from(udf)) } unsafe extern "C" fn udaf_fn_wrapper( registry: &FFI_FunctionRegistry, name: RString, ) -> RResult { - let registry = rresult_return!(registry.inner().lock()); - let udaf = rresult_return!(registry.udaf(name.as_str())); + let udaf = rresult_return!(registry.inner().udaf(name.as_str())); RResult::ROk(FFI_AggregateUDF::from(udaf)) } unsafe extern "C" fn udwf_fn_wrapper( registry: &FFI_FunctionRegistry, name: RString, ) -> RResult { - let registry = rresult_return!(registry.inner().lock()); - let udwf = rresult_return!(registry.udwf(name.as_str())); + let udwf = rresult_return!(registry.inner().udwf(name.as_str())); RResult::ROk(FFI_WindowUDF::from(udwf)) } -unsafe extern "C" fn register_udf_fn_wrapper( - registry: &mut FFI_FunctionRegistry, - udf: FFI_ScalarUDF, -) -> RResult, RString> { - let udf: ForeignScalarUDF = rresult_return!((&udf).try_into()); - let mut registry = rresult_return!(registry.inner().lock()); - let udf = rresult_return!(registry.register_udf(Arc::new(udf.into()))) - .map(FFI_ScalarUDF::from); - RResult::ROk(udf.into()) -} -unsafe extern "C" fn register_udaf_fn_wrapper( - registry: &mut FFI_FunctionRegistry, - udaf: FFI_AggregateUDF, -) -> RResult, RString> { - let udaf: ForeignAggregateUDF = rresult_return!((&udaf).try_into()); - let mut registry = rresult_return!(registry.inner().lock()); - let udaf = rresult_return!(registry.register_udaf(Arc::new(udaf.into()))) - .map(FFI_AggregateUDF::from); - RResult::ROk(udaf.into()) -} -unsafe extern "C" fn register_udwf_fn_wrapper( - registry: &mut FFI_FunctionRegistry, - udwf: FFI_WindowUDF, -) -> RResult, RString> { - let udwf: ForeignWindowUDF = rresult_return!((&udwf).try_into()); - let mut registry = rresult_return!(registry.inner().lock()); - let udwf = rresult_return!(registry.register_udwf(Arc::new(udwf.into()))) - .map(FFI_WindowUDF::from); - RResult::ROk(udwf.into()) -} - -unsafe extern "C" fn deregister_udf_fn_wrapper( - registry: &mut FFI_FunctionRegistry, - name: RString, -) -> RResult, RString> { - let mut registry = rresult_return!(registry.inner().lock()); - let udf = - rresult_return!(registry.deregister_udf(name.as_str())).map(FFI_ScalarUDF::from); - RResult::ROk(udf.into()) -} -unsafe extern "C" fn deregister_udaf_fn_wrapper( - registry: &mut FFI_FunctionRegistry, - name: RString, -) -> RResult, RString> { - let mut registry = rresult_return!(registry.inner().lock()); - let udaf = rresult_return!(registry.deregister_udaf(name.as_str())) - .map(FFI_AggregateUDF::from); - RResult::ROk(udaf.into()) -} -unsafe extern "C" fn deregister_udwf_fn_wrapper( - registry: &mut FFI_FunctionRegistry, - name: RString, -) -> RResult, RString> { - let mut registry = rresult_return!(registry.inner().lock()); - let udwf = - rresult_return!(registry.deregister_udwf(name.as_str())).map(FFI_WindowUDF::from); - RResult::ROk(udwf.into()) -} - unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_FunctionRegistry) { let private_data = Box::from_raw(provider.private_data as *mut RegistryPrivateData); drop(private_data); @@ -241,14 +134,6 @@ unsafe extern "C" fn clone_fn_wrapper( udaf: udaf_fn_wrapper, udwf: udwf_fn_wrapper, - register_udf: register_udf_fn_wrapper, - register_udaf: register_udaf_fn_wrapper, - register_udwf: register_udwf_fn_wrapper, - - deregister_udf: deregister_udf_fn_wrapper, - deregister_udaf: deregister_udaf_fn_wrapper, - deregister_udwf: deregister_udwf_fn_wrapper, - clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -264,7 +149,7 @@ impl Drop for FFI_FunctionRegistry { impl FFI_FunctionRegistry { /// Creates a new [`FFI_FunctionRegistry`]. - pub fn new(registry: Arc>) -> Self { + pub fn new(registry: Arc) -> Self { let private_data = Box::new(RegistryPrivateData { registry }); Self { @@ -276,14 +161,6 @@ impl FFI_FunctionRegistry { udaf: udaf_fn_wrapper, udwf: udwf_fn_wrapper, - register_udf: register_udf_fn_wrapper, - register_udaf: register_udaf_fn_wrapper, - register_udwf: register_udwf_fn_wrapper, - - deregister_udf: deregister_udf_fn_wrapper, - deregister_udaf: deregister_udaf_fn_wrapper, - deregister_udwf: deregister_udwf_fn_wrapper, - clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -356,96 +233,51 @@ impl FunctionRegistry for ForeignFunctionRegistry { fn register_udf( &mut self, - udf: Arc, + _udf: Arc, ) -> datafusion_common::Result>> { - let udf = FFI_ScalarUDF::from(udf); - let ROption::RSome(udf) = - df_result!(unsafe { (self.0.register_udf)(&mut self.0, udf) })? - else { - return Ok(None); - }; - let udf = ForeignScalarUDF::try_from(&udf)?; - - Ok(Some(Arc::new(ScalarUDF::from(udf)))) + not_impl_err!("Function Registry does not allow mutation via FFI") } fn register_udaf( &mut self, - udaf: Arc, + _udaf: Arc, ) -> datafusion_common::Result>> { - let udaf = FFI_AggregateUDF::from(udaf); - let ROption::RSome(udaf) = - df_result!(unsafe { (self.0.register_udaf)(&mut self.0, udaf) })? - else { - return Ok(None); - }; - let udaf = ForeignAggregateUDF::try_from(&udaf)?; - - Ok(Some(Arc::new(AggregateUDF::from(udaf)))) + not_impl_err!("Function Registry does not allow mutation via FFI") } fn register_udwf( &mut self, - udwf: Arc, + _udwf: Arc, ) -> datafusion_common::Result>> { - let udwf = FFI_WindowUDF::from(udwf); - let ROption::RSome(udwf) = - df_result!(unsafe { (self.0.register_udwf)(&mut self.0, udwf) })? - else { - return Ok(None); - }; - let udwf = ForeignWindowUDF::try_from(&udwf)?; - - Ok(Some(Arc::new(WindowUDF::from(udwf)))) + not_impl_err!("Function Registry does not allow mutation via FFI") } fn deregister_udf( &mut self, - name: &str, + _name: &str, ) -> datafusion_common::Result>> { - let ROption::RSome(udf) = - df_result!(unsafe { (self.0.deregister_udf)(&mut self.0, name.into()) })? - else { - return Ok(None); - }; - let udf = ForeignScalarUDF::try_from(&udf)?; - - Ok(Some(Arc::new(ScalarUDF::from(udf)))) + not_impl_err!("Function Registry does not allow mutation via FFI") } fn deregister_udaf( &mut self, - name: &str, + _name: &str, ) -> datafusion_common::Result>> { - let ROption::RSome(udaf) = - df_result!(unsafe { (self.0.deregister_udaf)(&mut self.0, name.into()) })? - else { - return Ok(None); - }; - let udwf = ForeignAggregateUDF::try_from(&udaf)?; - - Ok(Some(Arc::new(AggregateUDF::from(udwf)))) + not_impl_err!("Function Registry does not allow mutation via FFI") } fn deregister_udwf( &mut self, - name: &str, + _name: &str, ) -> datafusion_common::Result>> { - let ROption::RSome(udwf) = - df_result!(unsafe { (self.0.deregister_udwf)(&mut self.0, name.into()) })? - else { - return Ok(None); - }; - let udwf = ForeignWindowUDF::try_from(&udwf)?; - - Ok(Some(Arc::new(WindowUDF::from(udwf)))) + not_impl_err!("Function Registry does not allow mutation via FFI") } fn register_function_rewrite( &mut self, _rewrite: Arc, ) -> datafusion_common::Result<()> { - not_impl_err!("register_function_rewrite not implemented in FFI") + not_impl_err!("Function Registry does not allow mutation via FFI") } fn expr_planners(&self) -> Vec> { @@ -456,6 +288,6 @@ impl FunctionRegistry for ForeignFunctionRegistry { &mut self, _expr_planner: Arc, ) -> datafusion_common::Result<()> { - not_impl_err!("register_function_rewrite not implemented in FFI") + not_impl_err!("Function Registry does not allow mutation via FFI") } } From 2bae44372e111f5c9440ca9e6e87ad05c4c8a480 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 2 Nov 2025 07:19:38 -0500 Subject: [PATCH 14/69] Add paritioning to plan properties to remove proto usage --- datafusion/ffi/src/function_registry.rs | 124 +++++++++++------- datafusion/ffi/src/physical_expr/mod.rs | 3 +- .../ffi/src/physical_expr/partitioning.rs | 51 +++++++ datafusion/ffi/src/physical_expr/sort.rs | 26 +++- datafusion/ffi/src/plan_properties.rs | 111 ++++------------ 5 files changed, 183 insertions(+), 132 deletions(-) create mode 100644 datafusion/ffi/src/physical_expr/partitioning.rs diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index d0461c53f9eb..19fa4a36665d 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -28,17 +28,19 @@ use datafusion_expr::expr_rewriter::FunctionRewrite; use datafusion_expr::planner::ExprPlanner; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use log::warn; use std::collections::HashSet; +use std::sync::Weak; use std::{ffi::c_void, sync::Arc}; /// A stable struct for sharing [`FunctionRegistry`] across FFI boundaries. #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] -pub struct FFI_FunctionRegistry { - pub udfs: unsafe extern "C" fn(&Self) -> RVec, - pub udafs: unsafe extern "C" fn(&Self) -> RVec, - pub udwfs: unsafe extern "C" fn(&Self) -> RVec, +pub struct FFI_WeakFunctionRegistry { + pub udfs: unsafe extern "C" fn(&Self) -> RResult, RString>, + pub udafs: unsafe extern "C" fn(&Self) -> RResult, RString>, + pub udwfs: unsafe extern "C" fn(&Self) -> RResult, RString>, pub udf: unsafe extern "C" fn(&Self, name: RString) -> RResult, @@ -58,74 +60,86 @@ pub struct FFI_FunctionRegistry { pub version: unsafe extern "C" fn() -> u64, /// Internal data. This is only to be accessed by the provider of the plan. - /// A [`ForeignFunctionRegistry`] should never attempt to access this data. + /// A [`ForeignWeakFunctionRegistry`] should never attempt to access this data. pub private_data: *mut c_void, } -unsafe impl Send for FFI_FunctionRegistry {} -unsafe impl Sync for FFI_FunctionRegistry {} +unsafe impl Send for FFI_WeakFunctionRegistry {} +unsafe impl Sync for FFI_WeakFunctionRegistry {} struct RegistryPrivateData { - registry: Arc, + registry: Weak, } -impl FFI_FunctionRegistry { - unsafe fn inner(&self) -> &Arc { +impl FFI_WeakFunctionRegistry { + unsafe fn inner(&self) -> Result, RString> { let private_data = self.private_data as *const RegistryPrivateData; - &(*private_data).registry + (*private_data).registry.upgrade().ok_or_else(|| "Unable to access FunctionRegistry via FFI. Ensure owning object has not gone out of scope.".into()) } } -unsafe extern "C" fn udfs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { - let udfs = registry.inner().udfs(); - udfs.into_iter().map(|s| s.into()).collect() +unsafe extern "C" fn udfs_fn_wrapper( + registry: &FFI_WeakFunctionRegistry, +) -> RResult, RString> { + let inner = rresult_return!(registry.inner()); + let udfs = inner.udfs().into_iter().map(|s| s.into()).collect(); + RResult::ROk(udfs) } -unsafe extern "C" fn udafs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { - let udafs = registry.inner().udafs(); - udafs.into_iter().map(|s| s.into()).collect() +unsafe extern "C" fn udafs_fn_wrapper( + registry: &FFI_WeakFunctionRegistry, +) -> RResult, RString> { + let inner = rresult_return!(registry.inner()); + let udafs = inner.udafs().into_iter().map(|s| s.into()).collect(); + RResult::ROk(udafs) } -unsafe extern "C" fn udwfs_fn_wrapper(registry: &FFI_FunctionRegistry) -> RVec { - let udwfs = registry.inner().udwfs(); - udwfs.into_iter().map(|s| s.into()).collect() +unsafe extern "C" fn udwfs_fn_wrapper( + registry: &FFI_WeakFunctionRegistry, +) -> RResult, RString> { + let inner = rresult_return!(registry.inner()); + let udwfs = inner.udwfs().into_iter().map(|s| s.into()).collect(); + RResult::ROk(udwfs) } unsafe extern "C" fn udf_fn_wrapper( - registry: &FFI_FunctionRegistry, + registry: &FFI_WeakFunctionRegistry, name: RString, ) -> RResult { - let udf = rresult_return!(registry.inner().udf(name.as_str())); + let inner = rresult_return!(registry.inner()); + let udf = rresult_return!(inner.udf(name.as_str())); RResult::ROk(FFI_ScalarUDF::from(udf)) } unsafe extern "C" fn udaf_fn_wrapper( - registry: &FFI_FunctionRegistry, + registry: &FFI_WeakFunctionRegistry, name: RString, ) -> RResult { - let udaf = rresult_return!(registry.inner().udaf(name.as_str())); + let inner = rresult_return!(registry.inner()); + let udaf = rresult_return!(inner.udaf(name.as_str())); RResult::ROk(FFI_AggregateUDF::from(udaf)) } unsafe extern "C" fn udwf_fn_wrapper( - registry: &FFI_FunctionRegistry, + registry: &FFI_WeakFunctionRegistry, name: RString, ) -> RResult { - let udwf = rresult_return!(registry.inner().udwf(name.as_str())); + let inner = rresult_return!(registry.inner()); + let udwf = rresult_return!(inner.udwf(name.as_str())); RResult::ROk(FFI_WindowUDF::from(udwf)) } -unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_FunctionRegistry) { +unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_WeakFunctionRegistry) { let private_data = Box::from_raw(provider.private_data as *mut RegistryPrivateData); drop(private_data); } unsafe extern "C" fn clone_fn_wrapper( - provider: &FFI_FunctionRegistry, -) -> FFI_FunctionRegistry { + provider: &FFI_WeakFunctionRegistry, +) -> FFI_WeakFunctionRegistry { let old_private_data = provider.private_data as *const RegistryPrivateData; let private_data = Box::into_raw(Box::new(RegistryPrivateData { - registry: Arc::clone(&(*old_private_data).registry), + registry: Weak::clone(&(*old_private_data).registry), })) as *mut c_void; - FFI_FunctionRegistry { + FFI_WeakFunctionRegistry { udfs: udfs_fn_wrapper, udafs: udafs_fn_wrapper, udwfs: udwfs_fn_wrapper, @@ -141,15 +155,16 @@ unsafe extern "C" fn clone_fn_wrapper( } } -impl Drop for FFI_FunctionRegistry { +impl Drop for FFI_WeakFunctionRegistry { fn drop(&mut self) { unsafe { (self.release)(self) } } } -impl FFI_FunctionRegistry { - /// Creates a new [`FFI_FunctionRegistry`]. +impl FFI_WeakFunctionRegistry { + /// Creates a new [`FFI_WeakFunctionRegistry`]. pub fn new(registry: Arc) -> Self { + let registry = Arc::downgrade(®istry); let private_data = Box::new(RegistryPrivateData { registry }); Self { @@ -174,40 +189,55 @@ impl FFI_FunctionRegistry { /// defined on this struct must only use the stable functions provided in /// FFI_FunctionRegistry to interact with the foreign table provider. #[derive(Debug)] -pub struct ForeignFunctionRegistry(FFI_FunctionRegistry); +pub struct ForeignWeakFunctionRegistry(FFI_WeakFunctionRegistry); -unsafe impl Send for ForeignFunctionRegistry {} -unsafe impl Sync for ForeignFunctionRegistry {} +unsafe impl Send for ForeignWeakFunctionRegistry {} +unsafe impl Sync for ForeignWeakFunctionRegistry {} -impl From<&FFI_FunctionRegistry> for ForeignFunctionRegistry { - fn from(provider: &FFI_FunctionRegistry) -> Self { +impl From<&FFI_WeakFunctionRegistry> for ForeignWeakFunctionRegistry { + fn from(provider: &FFI_WeakFunctionRegistry) -> Self { Self(provider.clone()) } } -impl Clone for FFI_FunctionRegistry { +impl Clone for FFI_WeakFunctionRegistry { fn clone(&self) -> Self { unsafe { (self.clone)(self) } } } -impl FunctionRegistry for ForeignFunctionRegistry { +impl FunctionRegistry for ForeignWeakFunctionRegistry { fn udfs(&self) -> HashSet { let udfs = unsafe { (self.0.udfs)(&self.0) }; - - udfs.into_iter().map(String::from).collect() + match udfs { + RResult::ROk(udfs) => udfs.into_iter().map(String::from).collect(), + RResult::RErr(err) => { + warn!("{err}"); + HashSet::with_capacity(0) + } + } } fn udafs(&self) -> HashSet { let udafs = unsafe { (self.0.udafs)(&self.0) }; - - udafs.into_iter().map(String::from).collect() + match udafs { + RResult::ROk(udafs) => udafs.into_iter().map(String::from).collect(), + RResult::RErr(err) => { + warn!("{err}"); + HashSet::with_capacity(0) + } + } } fn udwfs(&self) -> HashSet { let udwfs = unsafe { (self.0.udwfs)(&self.0) }; - - udwfs.into_iter().map(String::from).collect() + match udwfs { + RResult::ROk(udwfs) => udwfs.into_iter().map(String::from).collect(), + RResult::RErr(err) => { + warn!("{err}"); + HashSet::with_capacity(0) + } + } } fn udf(&self, name: &str) -> datafusion_common::Result> { diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index df08f2f8c5ab..6d67f45c5a7a 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -pub(crate) mod sort; +pub mod partitioning; +pub mod sort; use crate::arrow_wrappers::WrappedArray; use crate::expr::columnar_value::FFI_ColumnarValue; diff --git a/datafusion/ffi/src/physical_expr/partitioning.rs b/datafusion/ffi/src/physical_expr/partitioning.rs new file mode 100644 index 000000000000..760c5960aa87 --- /dev/null +++ b/datafusion/ffi/src/physical_expr/partitioning.rs @@ -0,0 +1,51 @@ +use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; +use abi_stable::std_types::RVec; +use abi_stable::StableAbi; +use datafusion::physical_expr::{Partitioning, PhysicalExpr}; +use std::sync::Arc; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub enum FFI_Partitioning { + /// Allocate batches using a round-robin algorithm and the specified number of partitions + RoundRobinBatch(usize), + /// Allocate rows based on a hash of one of more expressions and the specified number of + /// partitions + Hash(RVec, usize), + /// Unknown partitioning scheme with a known number of partitions + UnknownPartitioning(usize), +} + +impl From<&Partitioning> for FFI_Partitioning { + fn from(partitioning: &Partitioning) -> Self { + match partitioning { + Partitioning::UnknownPartitioning(size) => Self::UnknownPartitioning(*size), + Partitioning::RoundRobinBatch(size) => Self::RoundRobinBatch(*size), + Partitioning::Hash(exprs, size) => { + let exprs = exprs.iter().cloned().map(Into::into).collect(); + Self::Hash(exprs, *size) + } + } + } +} + +impl From for Partitioning { + fn from(partitioning: FFI_Partitioning) -> Self { + match partitioning { + FFI_Partitioning::UnknownPartitioning(size) => { + Self::UnknownPartitioning(size) + } + FFI_Partitioning::RoundRobinBatch(size) => Self::RoundRobinBatch(size), + FFI_Partitioning::Hash(exprs, size) => { + let exprs = exprs + .into_iter() + .map(|expr| { + Arc::new(ForeignPhysicalExpr::from(expr)) as Arc + }) + .collect(); + Self::Hash(exprs, size) + } + } + } +} diff --git a/datafusion/ffi/src/physical_expr/sort.rs b/datafusion/ffi/src/physical_expr/sort.rs index 75ebcb6b4337..a957c911c778 100644 --- a/datafusion/ffi/src/physical_expr/sort.rs +++ b/datafusion/ffi/src/physical_expr/sort.rs @@ -17,9 +17,11 @@ use crate::expr::expr_properties::FFI_SortOptions; use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; +use abi_stable::std_types::RVec; use abi_stable::StableAbi; use arrow_schema::SortOptions; -use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_common::{exec_datafusion_err, DataFusionError}; use std::sync::Arc; #[repr(C)] @@ -47,3 +49,25 @@ impl From<&FFI_PhysicalSortExpr> for PhysicalSortExpr { Self { expr, options } } } + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_LexOrdering { + pub expr: RVec, +} + +impl From<&LexOrdering> for FFI_LexOrdering { + fn from(value: &LexOrdering) -> Self { + let expr = value.iter().map(FFI_PhysicalSortExpr::from).collect(); + FFI_LexOrdering { expr } + } +} + +impl TryFrom<&FFI_LexOrdering> for LexOrdering { + type Error = DataFusionError; + fn try_from(value: &FFI_LexOrdering) -> Result { + LexOrdering::new(value.expr.iter().map(PhysicalSortExpr::from)) + .ok_or(exec_datafusion_err!("FFI_LexOrdering was empty")) + } +} diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index 48c2698a58c7..4846e39ee613 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -17,14 +17,13 @@ use std::{ffi::c_void, sync::Arc}; -use abi_stable::{ - std_types::{ - RResult::{self, ROk}, - RString, RVec, - }, - StableAbi, -}; +use crate::arrow_wrappers::WrappedSchema; +use crate::physical_expr::partitioning::FFI_Partitioning; +use crate::physical_expr::sort::FFI_LexOrdering; +use abi_stable::std_types::ROption; +use abi_stable::StableAbi; use arrow::datatypes::SchemaRef; +use datafusion::physical_expr::LexOrdering; use datafusion::{ error::{DataFusionError, Result}, physical_expr::EquivalenceProperties, @@ -32,19 +31,7 @@ use datafusion::{ execution_plan::{Boundedness, EmissionType}, PlanProperties, }, - prelude::SessionContext, -}; -use datafusion_proto::{ - physical_plan::{ - from_proto::{parse_physical_sort_exprs, parse_protobuf_partitioning}, - to_proto::{serialize_partitioning, serialize_physical_sort_exprs}, - DefaultPhysicalExtensionCodec, - }, - protobuf::{Partitioning, PhysicalSortExprNodeCollection}, }; -use prost::Message; - -use crate::{arrow_wrappers::WrappedSchema, df_result, rresult_return}; /// A stable struct for sharing [`PlanProperties`] across FFI boundaries. #[repr(C)] @@ -53,8 +40,7 @@ use crate::{arrow_wrappers::WrappedSchema, df_result, rresult_return}; pub struct FFI_PlanProperties { /// The output partitioning is a [`Partitioning`] protobuf message serialized /// into bytes to pass across the FFI boundary. - pub output_partitioning: - unsafe extern "C" fn(plan: &Self) -> RResult, RString>, + pub output_partitioning: unsafe extern "C" fn(plan: &Self) -> FFI_Partitioning, /// Return the emission type of the plan. pub emission_type: unsafe extern "C" fn(plan: &Self) -> FFI_EmissionType, @@ -62,9 +48,8 @@ pub struct FFI_PlanProperties { /// Indicate boundedness of the plan and its memory requirements. pub boundedness: unsafe extern "C" fn(plan: &Self) -> FFI_Boundedness, - /// The output ordering is a [`PhysicalSortExprNodeCollection`] protobuf message - /// serialized into bytes to pass across the FFI boundary. - pub output_ordering: unsafe extern "C" fn(plan: &Self) -> RResult, RString>, + /// The output ordering of the plan. + pub output_ordering: unsafe extern "C" fn(plan: &Self) -> ROption, /// Return the schema of the plan. pub schema: unsafe extern "C" fn(plan: &Self) -> WrappedSchema, @@ -83,16 +68,17 @@ struct PlanPropertiesPrivateData { unsafe extern "C" fn output_partitioning_fn_wrapper( properties: &FFI_PlanProperties, -) -> RResult, RString> { +) -> FFI_Partitioning { let private_data = properties.private_data as *const PlanPropertiesPrivateData; let props = &(*private_data).props; - - let codec = DefaultPhysicalExtensionCodec {}; - let partitioning_data = - rresult_return!(serialize_partitioning(props.output_partitioning(), &codec)); - let output_partitioning = partitioning_data.encode_to_vec(); - - ROk(output_partitioning.into()) + // + // let codec = DefaultPhysicalExtensionCodec {}; + // let partitioning_data = + // rresult_return!(serialize_partitioning(props.output_partitioning(), &codec)); + // let output_partitioning = partitioning_data.encode_to_vec(); + + // ROk(output_partitioning.into()) + (&props.partitioning).into() } unsafe extern "C" fn emission_type_fn_wrapper( @@ -113,25 +99,11 @@ unsafe extern "C" fn boundedness_fn_wrapper( unsafe extern "C" fn output_ordering_fn_wrapper( properties: &FFI_PlanProperties, -) -> RResult, RString> { +) -> ROption { let private_data = properties.private_data as *const PlanPropertiesPrivateData; let props = &(*private_data).props; - let codec = DefaultPhysicalExtensionCodec {}; - let output_ordering = match props.output_ordering() { - Some(ordering) => { - let physical_sort_expr_nodes = rresult_return!( - serialize_physical_sort_exprs(ordering.to_owned(), &codec) - ); - let ordering_data = PhysicalSortExprNodeCollection { - physical_sort_expr_nodes, - }; - - ordering_data.encode_to_vec() - } - None => Vec::default(), - }; - ROk(output_ordering.into()) + props.output_ordering().map(FFI_LexOrdering::from).into() } unsafe extern "C" fn schema_fn_wrapper(properties: &FFI_PlanProperties) -> WrappedSchema { @@ -179,43 +151,16 @@ impl TryFrom for PlanProperties { let ffi_schema = unsafe { (ffi_props.schema)(&ffi_props) }; let schema = (&ffi_schema.0).try_into()?; - // TODO Extend FFI to get the registry and codex - let default_ctx = SessionContext::new(); - let task_context = default_ctx.task_ctx(); - let codex = DefaultPhysicalExtensionCodec {}; - let ffi_orderings = unsafe { (ffi_props.output_ordering)(&ffi_props) }; - let proto_output_ordering = - PhysicalSortExprNodeCollection::decode(df_result!(ffi_orderings)?.as_ref()) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - let sort_exprs = parse_physical_sort_exprs( - &proto_output_ordering.physical_sort_expr_nodes, - &task_context, - &schema, - &codex, - )?; - - let partitioning_vec = - unsafe { df_result!((ffi_props.output_partitioning)(&ffi_props))? }; - let proto_output_partitioning = - Partitioning::decode(partitioning_vec.as_ref()) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - let partitioning = parse_protobuf_partitioning( - Some(&proto_output_partitioning), - &task_context, - &schema, - &codex, - )? - .ok_or(DataFusionError::Plan( - "Unable to deserialize partitioning protobuf in FFI_PlanProperties" - .to_string(), - ))?; - - let eq_properties = if sort_exprs.is_empty() { - EquivalenceProperties::new(Arc::new(schema)) - } else { - EquivalenceProperties::new_with_orderings(Arc::new(schema), [sort_exprs]) + let partitioning = unsafe { (ffi_props.output_partitioning)(&ffi_props) }.into(); + + let eq_properties = match ffi_orderings { + ROption::RSome(lex_ordering) => { + let ordering = LexOrdering::try_from(&lex_ordering)?; + EquivalenceProperties::new_with_orderings(Arc::new(schema), [ordering]) + } + ROption::RNone => EquivalenceProperties::new(Arc::new(schema)), }; let emission_type: EmissionType = From ed95eebfc5f51480fd48ec026d1bc76f60deac15 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 2 Nov 2025 07:52:56 -0500 Subject: [PATCH 15/69] Intermediate work on removing datafusion core --- Cargo.lock | 6 ++ datafusion/ffi/Cargo.toml | 8 ++- datafusion/ffi/src/catalog_provider.rs | 4 +- datafusion/ffi/src/execution_plan.rs | 10 ++-- datafusion/ffi/src/expr/columnar_value.rs | 2 +- datafusion/ffi/src/expr/distribution.rs | 2 +- datafusion/ffi/src/expr/expr_properties.rs | 2 +- datafusion/ffi/src/expr/interval.rs | 2 +- datafusion/ffi/src/insert_op.rs | 2 +- datafusion/ffi/src/physical_expr/mod.rs | 11 ++-- .../ffi/src/physical_expr/partitioning.rs | 2 +- datafusion/ffi/src/physical_expr/sort.rs | 2 +- datafusion/ffi/src/plan_properties.rs | 14 ++--- datafusion/ffi/src/record_batch_stream.rs | 7 +-- datafusion/ffi/src/schema_provider.rs | 8 +-- datafusion/ffi/src/session_config.rs | 4 +- datafusion/ffi/src/table_provider.rs | 44 +++++++------- datafusion/ffi/src/table_source.rs | 2 +- datafusion/ffi/src/udaf/accumulator.rs | 4 +- datafusion/ffi/src/udaf/accumulator_args.rs | 6 +- datafusion/ffi/src/udaf/groups_accumulator.rs | 4 +- datafusion/ffi/src/udaf/mod.rs | 21 +++---- datafusion/ffi/src/udf/mod.rs | 16 ++--- datafusion/ffi/src/udf/return_type_args.rs | 6 +- datafusion/ffi/src/udtf.rs | 60 ++++++++++--------- datafusion/ffi/src/udwf/mod.rs | 22 +++---- .../ffi/src/udwf/partition_evaluator.rs | 4 +- .../ffi/src/udwf/partition_evaluator_args.rs | 6 +- datafusion/ffi/src/volatility.rs | 2 +- 29 files changed, 139 insertions(+), 144 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0f84a5c24783..0c4754bd63ce 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2292,9 +2292,15 @@ dependencies = [ "async-ffi", "async-trait", "datafusion", + "datafusion-catalog", "datafusion-common", + "datafusion-execution", "datafusion-expr", + "datafusion-expr-common", "datafusion-functions-aggregate-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", "datafusion-proto", "datafusion-proto-common", "doc-comment", diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index 90551eaad374..3ad5f4af7a62 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -46,10 +46,15 @@ arrow = { workspace = true, features = ["ffi"] } arrow-schema = { workspace = true } async-ffi = { version = "0.5.0", features = ["abi_stable"] } async-trait = { workspace = true } -datafusion = { workspace = true, default-features = false } +datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-expr-common = { workspace = true } +datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-plan = { workspace = true } datafusion-proto = { workspace = true } datafusion-proto-common = { workspace = true } futures = { workspace = true } @@ -59,6 +64,7 @@ semver = "1.0.27" tokio = { workspace = true } [dev-dependencies] +datafusion = { workspace = true, default-features = false } doc-comment = { workspace = true } [features] diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index 65dcab34f17d..64c557999ad0 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -21,7 +21,7 @@ use abi_stable::{ std_types::{ROption, RResult, RString, RVec}, StableAbi, }; -use datafusion::catalog::{CatalogProvider, SchemaProvider}; +use datafusion_catalog::{CatalogProvider, SchemaProvider}; use tokio::runtime::Handle; use crate::{ @@ -29,7 +29,7 @@ use crate::{ schema_provider::{FFI_SchemaProvider, ForeignSchemaProvider}, }; -use datafusion::error::Result; +use datafusion_common::error::Result; /// A stable struct for sharing [`CatalogProvider`] across FFI boundaries. #[repr(C)] diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index 70c957d8c373..1d72e66355d6 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -21,12 +21,12 @@ use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, }; -use datafusion::{ - error::DataFusionError, - execution::{SendableRecordBatchStream, TaskContext}, - physical_plan::{DisplayAs, ExecutionPlan, PlanProperties}, +use datafusion_common::{ + error::{DataFusionError, Result}, }; -use datafusion::{error::Result, physical_plan::DisplayFormatType}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_plan::{DisplayAs, ExecutionPlan, PlanProperties}; +use datafusion_physical_plan::DisplayFormatType; use tokio::runtime::Handle; use crate::{ diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs index c20f599dc356..bef88a7fbac5 100644 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -19,7 +19,7 @@ use crate::arrow_wrappers::WrappedArray; use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion::logical_expr::ColumnarValue; +use datafusion_expr::ColumnarValue; use datafusion_common::DataFusionError; #[repr(C)] diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs index 1a773e414092..476819634c95 100644 --- a/datafusion/ffi/src/expr/distribution.rs +++ b/datafusion/ffi/src/expr/distribution.rs @@ -19,7 +19,7 @@ use crate::expr::interval::FFI_Interval; use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion::logical_expr::statistics::{ +use datafusion_expr::statistics::{ BernoulliDistribution, Distribution, ExponentialDistribution, GaussianDistribution, GenericDistribution, UniformDistribution, }; diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs index 4f873a61bcc6..a175e6f70c5d 100644 --- a/datafusion/ffi/src/expr/expr_properties.rs +++ b/datafusion/ffi/src/expr/expr_properties.rs @@ -18,7 +18,7 @@ use crate::expr::interval::FFI_Interval; use abi_stable::StableAbi; use arrow_schema::SortOptions; -use datafusion::logical_expr::sort_properties::{ExprProperties, SortProperties}; +use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_common::DataFusionError; #[repr(C)] diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs index 3eb65df73b84..637982f5bdaa 100644 --- a/datafusion/ffi/src/expr/interval.rs +++ b/datafusion/ffi/src/expr/interval.rs @@ -18,7 +18,7 @@ use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion::logical_expr::interval_arithmetic::Interval; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_common::DataFusionError; #[repr(C)] diff --git a/datafusion/ffi/src/insert_op.rs b/datafusion/ffi/src/insert_op.rs index 8e8693076cc0..acba1aa2ae08 100644 --- a/datafusion/ffi/src/insert_op.rs +++ b/datafusion/ffi/src/insert_op.rs @@ -16,7 +16,7 @@ // under the License. use abi_stable::StableAbi; -use datafusion::logical_expr::logical_plan::dml::InsertOp; +use datafusion_expr::logical_plan::dml::InsertOp; /// FFI safe version of [`InsertOp`]. #[repr(C)] diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index 6d67f45c5a7a..9c323f7281fc 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -37,14 +37,13 @@ use arrow::array::{ArrayRef, BooleanArray, RecordBatch}; use arrow::datatypes::SchemaRef; use arrow_schema::ffi::FFI_ArrowSchema; use arrow_schema::{DataType, Field, FieldRef, Schema}; -use datafusion::logical_expr::ColumnarValue; -use datafusion::logical_expr_common::interval_arithmetic::Interval; -use datafusion::logical_expr_common::sort_properties::ExprProperties; -use datafusion::logical_expr_common::statistics::Distribution; -use datafusion::physical_expr::PhysicalExpr; -use datafusion::physical_expr_common::physical_expr::fmt_sql; use datafusion_common::exec_datafusion_err; use datafusion_common::Result; +use datafusion_expr::ColumnarValue; +use datafusion_expr_common::interval_arithmetic::Interval; +use datafusion_expr_common::sort_properties::ExprProperties; +use datafusion_expr_common::statistics::Distribution; +use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExpr}; use std::fmt::{Display, Formatter}; use std::hash::{DefaultHasher, Hash, Hasher}; use std::{any::Any, ffi::c_void, sync::Arc}; diff --git a/datafusion/ffi/src/physical_expr/partitioning.rs b/datafusion/ffi/src/physical_expr/partitioning.rs index 760c5960aa87..617be82267d2 100644 --- a/datafusion/ffi/src/physical_expr/partitioning.rs +++ b/datafusion/ffi/src/physical_expr/partitioning.rs @@ -1,7 +1,7 @@ use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion::physical_expr::{Partitioning, PhysicalExpr}; +use datafusion_physical_expr::{Partitioning, PhysicalExpr}; use std::sync::Arc; #[repr(C)] diff --git a/datafusion/ffi/src/physical_expr/sort.rs b/datafusion/ffi/src/physical_expr/sort.rs index a957c911c778..c6ff559b9805 100644 --- a/datafusion/ffi/src/physical_expr/sort.rs +++ b/datafusion/ffi/src/physical_expr/sort.rs @@ -20,7 +20,7 @@ use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; use arrow_schema::SortOptions; -use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_common::{exec_datafusion_err, DataFusionError}; use std::sync::Arc; diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index 4846e39ee613..ffb00f8d61f1 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -23,14 +23,12 @@ use crate::physical_expr::sort::FFI_LexOrdering; use abi_stable::std_types::ROption; use abi_stable::StableAbi; use arrow::datatypes::SchemaRef; -use datafusion::physical_expr::LexOrdering; -use datafusion::{ - error::{DataFusionError, Result}, - physical_expr::EquivalenceProperties, - physical_plan::{ - execution_plan::{Boundedness, EmissionType}, - PlanProperties, - }, +use datafusion_common::error::{DataFusionError, Result}; +use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::LexOrdering; +use datafusion_physical_plan::{ + execution_plan::{Boundedness, EmissionType}, + PlanProperties, }; /// A stable struct for sharing [`PlanProperties`] across FFI boundaries. diff --git a/datafusion/ffi/src/record_batch_stream.rs b/datafusion/ffi/src/record_batch_stream.rs index 89fe6df617cf..bb76dec80f02 100644 --- a/datafusion/ffi/src/record_batch_stream.rs +++ b/datafusion/ffi/src/record_batch_stream.rs @@ -27,11 +27,8 @@ use arrow::{ ffi::{from_ffi, to_ffi}, }; use async_ffi::{ContextExt, FfiContext, FfiPoll}; -use datafusion::error::Result; -use datafusion::{ - error::DataFusionError, - execution::{RecordBatchStream, SendableRecordBatchStream}, -}; +use datafusion_common::error::{DataFusionError, Result}; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; use datafusion_common::{exec_datafusion_err, exec_err}; use futures::{Stream, TryStreamExt}; use tokio::runtime::Handle; diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index b5970d5881d6..5b63d88d08ab 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -23,10 +23,8 @@ use abi_stable::{ }; use async_ffi::{FfiFuture, FutureExt}; use async_trait::async_trait; -use datafusion::{ - catalog::{SchemaProvider, TableProvider}, - error::DataFusionError, -}; +use datafusion_catalog::{SchemaProvider, TableProvider}; +use datafusion_common::error::{DataFusionError, Result}; use tokio::runtime::Handle; use crate::{ @@ -34,8 +32,6 @@ use crate::{ table_provider::{FFI_TableProvider, ForeignTableProvider}, }; -use datafusion::error::Result; - /// A stable struct for sharing [`SchemaProvider`] across FFI boundaries. #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/session_config.rs b/datafusion/ffi/src/session_config.rs index a07b66c60196..78675eebb35c 100644 --- a/datafusion/ffi/src/session_config.rs +++ b/datafusion/ffi/src/session_config.rs @@ -19,13 +19,13 @@ use abi_stable::{ std_types::{RHashMap, RString}, StableAbi, }; -use datafusion::{config::ConfigOptions, error::Result}; -use datafusion::{error::DataFusionError, prelude::SessionConfig}; +use datafusion_common::{config::ConfigOptions, error::{DataFusionError, Result}}; use std::sync::Arc; use std::{ collections::HashMap, ffi::{c_char, c_void, CString}, }; +use datafusion_execution::config::SessionConfig; /// A stable struct for sharing [`SessionConfig`] across FFI boundaries. /// Instead of attempting to expose the entire SessionConfig interface, we diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 890511997a70..a37c1f416f2e 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -24,15 +24,8 @@ use abi_stable::{ use arrow::datatypes::SchemaRef; use async_ffi::{FfiFuture, FutureExt}; use async_trait::async_trait; -use datafusion::{ - catalog::{Session, TableProvider}, - datasource::TableType, - error::DataFusionError, - execution::{session_state::SessionStateBuilder, TaskContext}, - logical_expr::{logical_plan::dml::InsertOp, TableProviderFilterPushDown}, - physical_plan::ExecutionPlan, - prelude::{Expr, SessionContext}, -}; +use datafusion_catalog::{Session, TableProvider}; + use datafusion_proto::{ logical_plan::{ from_proto::parse_exprs, to_proto::serialize_exprs, DefaultLogicalExtensionCodec, @@ -55,6 +48,12 @@ use super::{ session_config::FFI_SessionConfig, }; use datafusion::error::Result; +use datafusion_common::DataFusionError; +use datafusion_execution::TaskContext; +use datafusion_expr::dml::InsertOp; +use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType}; +use datafusion_physical_plan::ExecutionPlan; +use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; /// A stable struct for sharing [`TableProvider`] across FFI boundaries. /// @@ -143,6 +142,8 @@ pub struct FFI_TableProvider { insert_op: FFI_InsertOp, ) -> FfiFuture>, + pub function_registry: FFI_WeakFunctionRegistry, + /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -185,8 +186,8 @@ unsafe extern "C" fn table_type_fn_wrapper( fn supports_filters_pushdown_internal( provider: &Arc, filters_serialized: &[u8], + function_registry: &ForeignWeakFunctionRegistry, ) -> Result> { - let default_ctx = SessionContext::new(); let codec = DefaultLogicalExtensionCodec {}; let filters = match filters_serialized.is_empty() { @@ -195,7 +196,7 @@ fn supports_filters_pushdown_internal( let proto_filters = LogicalExprList::decode(filters_serialized) .map_err(|e| DataFusionError::Plan(e.to_string()))?; - parse_exprs(proto_filters.expr.iter(), &default_ctx, &codec)? + parse_exprs(proto_filters.expr.iter(), function_registry, &codec)? } }; let filters_borrowed: Vec<&Expr> = filters.iter().collect(); @@ -213,38 +214,41 @@ unsafe extern "C" fn supports_filters_pushdown_fn_wrapper( provider: &FFI_TableProvider, filters_serialized: RVec, ) -> RResult, RString> { + let function_registry = rresult_return!(ForeignWeakFunctionRegistry::try_from(&provider.function_registry)); let private_data = provider.private_data as *const ProviderPrivateData; let provider = &(*private_data).provider; - supports_filters_pushdown_internal(provider, &filters_serialized) + supports_filters_pushdown_internal(provider, &filters_serialized, &function_registry) .map_err(|e| e.to_string().into()) .into() } unsafe extern "C" fn scan_fn_wrapper( provider: &FFI_TableProvider, - session_config: &FFI_SessionConfig, + session: &FFI_Session, projections: RVec, filters_serialized: RVec, limit: ROption, ) -> FfiFuture> { + let function_registry = ForeignWeakFunctionRegistry::try_from(&provider.function_registry); let private_data = provider.private_data as *mut ProviderPrivateData; let internal_provider = &(*private_data).provider; let session_config = session_config.clone(); let runtime = &(*private_data).runtime; async move { + let function_registry = rresult_return!(function_registry); let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); - let session = SessionStateBuilder::new() - .with_default_features() - .with_config(config.0) - .build(); - let ctx = SessionContext::new_with_state(session); + // let session = SessionStateBuilder::new() + // .with_default_features() + // .with_config(config.0) + // .build(); + // let ctx = SessionContext::new_with_state(session); let filters = match filters_serialized.is_empty() { true => vec![], false => { - let default_ctx = SessionContext::new(); + // let default_ctx = SessionContext::new(); let codec = DefaultLogicalExtensionCodec {}; let proto_filters = @@ -252,7 +256,7 @@ unsafe extern "C" fn scan_fn_wrapper( rresult_return!(parse_exprs( proto_filters.expr.iter(), - &default_ctx, + &function_registry, &codec )) } diff --git a/datafusion/ffi/src/table_source.rs b/datafusion/ffi/src/table_source.rs index 418fdf16a564..dfdf8c1c64ae 100644 --- a/datafusion/ffi/src/table_source.rs +++ b/datafusion/ffi/src/table_source.rs @@ -16,7 +16,7 @@ // under the License. use abi_stable::StableAbi; -use datafusion::{datasource::TableType, logical_expr::TableProviderFilterPushDown}; +use datafusion_expr::{TableProviderFilterPushDown, TableType}; /// FFI safe version of [`TableProviderFilterPushDown`]. #[repr(C)] diff --git a/datafusion/ffi/src/udaf/accumulator.rs b/datafusion/ffi/src/udaf/accumulator.rs index 80b872159f48..f7efa88dc90d 100644 --- a/datafusion/ffi/src/udaf/accumulator.rs +++ b/datafusion/ffi/src/udaf/accumulator.rs @@ -22,11 +22,11 @@ use abi_stable::{ StableAbi, }; use arrow::{array::ArrayRef, error::ArrowError}; -use datafusion::{ +use datafusion_common::{ error::{DataFusionError, Result}, - logical_expr::Accumulator, scalar::ScalarValue, }; +use datafusion_expr::Accumulator; use prost::Message; use crate::{arrow_wrappers::WrappedArray, df_result, rresult, rresult_return}; diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 41d34d54da54..1cead1ea1da7 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -26,11 +26,11 @@ use abi_stable::{ }; use arrow::{datatypes::Schema, ffi::FFI_ArrowSchema}; use arrow_schema::FieldRef; -use datafusion::{ +use datafusion_common::{ error::DataFusionError, - logical_expr::function::AccumulatorArgs, - physical_expr::{PhysicalExpr, PhysicalSortExpr}, }; +use datafusion_expr::function::AccumulatorArgs; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; /// A stable struct for sharing [`AccumulatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding field diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 58a18c69db7c..914ae9858c1b 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -30,10 +30,10 @@ use arrow::{ error::ArrowError, ffi::to_ffi, }; -use datafusion::{ +use datafusion_common::{ error::{DataFusionError, Result}, - logical_expr::{EmitTo, GroupsAccumulator}, }; +use datafusion_expr::{EmitTo, GroupsAccumulator}; /// A stable struct for sharing [`GroupsAccumulator`] across FFI boundaries. /// For an explanation of each field, see the corresponding function diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index ce5611590b67..f77dcf3a69ce 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -24,20 +24,17 @@ use accumulator_args::{FFI_AccumulatorArgs, ForeignAccumulatorArgs}; use arrow::datatypes::{DataType, Field}; use arrow::ffi::FFI_ArrowSchema; use arrow_schema::FieldRef; -use datafusion::{ - error::DataFusionError, - logical_expr::{ - function::{AccumulatorArgs, AggregateFunctionSimplification, StateFieldsArgs}, - type_coercion::functions::fields_with_aggregate_udf, - utils::AggregateOrderSensitivity, - Accumulator, GroupsAccumulator, - }, -}; -use datafusion::{ - error::Result, - logical_expr::{AggregateUDF, AggregateUDFImpl, Signature}, +use datafusion_common::error::DataFusionError; +use datafusion_expr::{ + function::{AccumulatorArgs, AggregateFunctionSimplification, StateFieldsArgs}, + type_coercion::functions::fields_with_aggregate_udf, + utils::AggregateOrderSensitivity, + Accumulator, GroupsAccumulator, }; + +use datafusion_common::error::Result; use datafusion_common::exec_datafusion_err; +use datafusion_expr::{AggregateUDF, AggregateUDFImpl, Signature}; use datafusion_proto_common::from_proto::parse_proto_fields_to_fields; use groups_accumulator::{FFI_GroupsAccumulator, ForeignGroupsAccumulator}; use std::hash::{Hash, Hasher}; diff --git a/datafusion/ffi/src/udf/mod.rs b/datafusion/ffi/src/udf/mod.rs index 5e59cfc5ecb0..1a31efecb22f 100644 --- a/datafusion/ffi/src/udf/mod.rs +++ b/datafusion/ffi/src/udf/mod.rs @@ -32,17 +32,11 @@ use arrow::{ ffi::{from_ffi, to_ffi, FFI_ArrowSchema}, }; use arrow_schema::FieldRef; -use datafusion::config::ConfigOptions; -use datafusion::logical_expr::ReturnFieldArgs; -use datafusion::{ - error::DataFusionError, - logical_expr::type_coercion::functions::data_types_with_scalar_udf, -}; -use datafusion::{ - error::Result, - logical_expr::{ - ColumnarValue, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, - }, +use datafusion_common::config::ConfigOptions; +use datafusion_common::error::{DataFusionError, Result}; +use datafusion_expr::{ + type_coercion::functions::data_types_with_scalar_udf, ColumnarValue, ReturnFieldArgs, + ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, }; use return_type_args::{ FFI_ReturnFieldArgs, ForeignReturnFieldArgs, ForeignReturnFieldArgsOwned, diff --git a/datafusion/ffi/src/udf/return_type_args.rs b/datafusion/ffi/src/udf/return_type_args.rs index c437c9537be6..463b3d28a9f3 100644 --- a/datafusion/ffi/src/udf/return_type_args.rs +++ b/datafusion/ffi/src/udf/return_type_args.rs @@ -20,13 +20,13 @@ use abi_stable::{ StableAbi, }; use arrow_schema::FieldRef; -use datafusion::{ - common::exec_datafusion_err, error::DataFusionError, logical_expr::ReturnFieldArgs, - scalar::ScalarValue, +use datafusion_common::{ + error::DataFusionError, exec_datafusion_err, scalar::ScalarValue, }; use crate::arrow_wrappers::WrappedSchema; use crate::util::{rvec_wrapped_to_vec_fieldref, vec_fieldref_to_rvec_wrapped}; +use datafusion_expr::ReturnFieldArgs; use prost::Message; /// A stable struct for sharing a [`ReturnFieldArgs`] across FFI boundaries. diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index edd5273c70a8..b81bda4dda08 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -22,11 +22,9 @@ use abi_stable::{ StableAbi, }; -use datafusion::error::Result; -use datafusion::{ - catalog::{TableFunctionImpl, TableProvider}, - prelude::{Expr, SessionContext}, -}; +use datafusion_common::error::Result; +use datafusion_catalog::{TableFunctionImpl, TableProvider}; +use datafusion_expr::Expr; use datafusion_proto::{ logical_plan::{ from_proto::parse_exprs, to_proto::serialize_exprs, DefaultLogicalExtensionCodec, @@ -35,11 +33,11 @@ use datafusion_proto::{ }; use prost::Message; use tokio::runtime::Handle; - use crate::{ df_result, rresult_return, table_provider::{FFI_TableProvider, ForeignTableProvider}, }; +use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; /// A stable struct for sharing a [`TableFunctionImpl`] across FFI boundaries. #[repr(C)] @@ -53,6 +51,8 @@ pub struct FFI_TableFunction { args: RVec, ) -> RResult, + pub function_registry: FFI_WeakFunctionRegistry, + /// Used to create a clone on the provider of the udtf. This should /// only need to be called by the receiver of the udtf. pub clone: unsafe extern "C" fn(udtf: &Self) -> Self, @@ -89,16 +89,17 @@ unsafe extern "C" fn call_fn_wrapper( udtf: &FFI_TableFunction, args: RVec, ) -> RResult { + let function_registry = ForeignWeakFunctionRegistry::try_from(&udtf.function_registry)?; + let runtime = udtf.runtime(); let udtf = udtf.inner(); - let default_ctx = SessionContext::new(); let codec = DefaultLogicalExtensionCodec {}; let proto_filters = rresult_return!(LogicalExprList::decode(args.as_ref())); let args = - rresult_return!(parse_exprs(proto_filters.expr.iter(), &default_ctx, &codec)); + rresult_return!(parse_exprs(proto_filters.expr.iter(), &function_registry, &codec)); let table_provider = rresult_return!(udtf.call(&args)); RResult::ROk(FFI_TableProvider::new(table_provider, false, runtime)) @@ -110,10 +111,11 @@ unsafe extern "C" fn release_fn_wrapper(udtf: &mut FFI_TableFunction) { } unsafe extern "C" fn clone_fn_wrapper(udtf: &FFI_TableFunction) -> FFI_TableFunction { + let function_registry = udtf.function_registry.clone(); let runtime = udtf.runtime(); let udtf = udtf.inner(); - FFI_TableFunction::new(Arc::clone(udtf), runtime) + FFI_TableFunction::new(Arc::clone(udtf), runtime, function_registry) } impl Clone for FFI_TableFunction { @@ -123,11 +125,13 @@ impl Clone for FFI_TableFunction { } impl FFI_TableFunction { - pub fn new(udtf: Arc, runtime: Option) -> Self { + pub fn new(udtf: Arc, runtime: Option, function_registry: impl Into) -> Self { + let function_registry = function_registry.into(); let private_data = Box::new(TableFunctionPrivateData { udtf, runtime }); Self { call: call_fn_wrapper, + function_registry: function_registry.into(), clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, @@ -135,21 +139,21 @@ impl FFI_TableFunction { } } -impl From> for FFI_TableFunction { - fn from(udtf: Arc) -> Self { - let private_data = Box::new(TableFunctionPrivateData { - udtf, - runtime: None, - }); - - Self { - call: call_fn_wrapper, - clone: clone_fn_wrapper, - release: release_fn_wrapper, - private_data: Box::into_raw(private_data) as *mut c_void, - } - } -} +// impl From> for FFI_TableFunction { +// fn from(udtf: Arc) -> Self { +// let private_data = Box::new(TableFunctionPrivateData { +// udtf, +// runtime: None, +// }); +// +// Self { +// call: call_fn_wrapper, +// clone: clone_fn_wrapper, +// release: release_fn_wrapper, +// private_data: Box::into_raw(private_data) as *mut c_void, +// } +// } +// } impl Drop for FFI_TableFunction { fn drop(&mut self) { @@ -203,7 +207,7 @@ mod tests { use datafusion::{ catalog::MemTable, common::exec_err, prelude::lit, scalar::ScalarValue, }; - + use datafusion::prelude::SessionContext; use super::*; #[derive(Debug)] @@ -288,14 +292,14 @@ mod tests { async fn test_round_trip_udtf() -> Result<()> { let original_udtf = Arc::new(TestUDTF {}) as Arc; + let ctx = Arc::new(SessionContext::default()); let local_udtf: FFI_TableFunction = - FFI_TableFunction::new(Arc::clone(&original_udtf), None); + FFI_TableFunction::new(Arc::clone(&original_udtf), None, Arc::clone(&ctx))?; let foreign_udf: ForeignTableFunction = local_udtf.into(); let table = foreign_udf.call(&[lit(6_u64), lit("one"), lit(2.0), lit(3_u64)])?; - let ctx = SessionContext::default(); let _ = ctx.register_table("test-table", table)?; let returned_batches = ctx.table("test-table").await?.collect().await?; diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index 9f56e2d4788b..d9459f89be04 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -25,20 +25,14 @@ use arrow::{ datatypes::{DataType, SchemaRef}, }; use arrow_schema::{Field, FieldRef}; -use datafusion::logical_expr::LimitEffect; -use datafusion::physical_expr::PhysicalExpr; -use datafusion::{ - error::DataFusionError, - logical_expr::{ - function::WindowUDFFieldArgs, type_coercion::functions::fields_with_window_udf, - PartitionEvaluator, - }, -}; -use datafusion::{ - error::Result, - logical_expr::{Signature, WindowUDF, WindowUDFImpl}, -}; +use datafusion_common::error::{DataFusionError, Result}; use datafusion_common::exec_err; +use datafusion_expr::function::WindowUDFFieldArgs; +use datafusion_expr::type_coercion::functions::fields_with_window_udf; +use datafusion_expr::{ + LimitEffect, PartitionEvaluator, Signature, WindowUDF, WindowUDFImpl, +}; +use datafusion_physical_expr::PhysicalExpr; use partition_evaluator::{FFI_PartitionEvaluator, ForeignPartitionEvaluator}; use partition_evaluator_args::{ FFI_PartitionEvaluatorArgs, ForeignPartitionEvaluatorArgs, @@ -315,7 +309,7 @@ impl WindowUDFImpl for ForeignWindowUDF { fn partition_evaluator( &self, - args: datafusion::logical_expr::function::PartitionEvaluatorArgs, + args: datafusion_expr::function::PartitionEvaluatorArgs, ) -> Result> { let evaluator = unsafe { let args = FFI_PartitionEvaluatorArgs::try_from(args)?; diff --git a/datafusion/ffi/src/udwf/partition_evaluator.rs b/datafusion/ffi/src/udwf/partition_evaluator.rs index 14cf23b919aa..b8cdb61e8ea6 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator.rs @@ -23,11 +23,11 @@ use abi_stable::{ StableAbi, }; use arrow::{array::ArrayRef, error::ArrowError}; -use datafusion::{ +use datafusion_common::{ error::{DataFusionError, Result}, - logical_expr::{window_state::WindowAggState, PartitionEvaluator}, scalar::ScalarValue, }; +use datafusion_expr::{window_state::WindowAggState, PartitionEvaluator}; use prost::Message; use super::range::FFI_Range; diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index a42c828665e0..16ae5c955e9a 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -26,11 +26,11 @@ use arrow::{ ffi::FFI_ArrowSchema, }; use arrow_schema::FieldRef; -use datafusion::{ +use datafusion_common::{ error::{DataFusionError, Result}, - logical_expr::function::PartitionEvaluatorArgs, - physical_plan::{expressions::Column, PhysicalExpr}, }; +use datafusion_expr::function::PartitionEvaluatorArgs; +use datafusion_physical_plan::{expressions::Column, PhysicalExpr}; /// A stable struct for sharing [`PartitionEvaluatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding function diff --git a/datafusion/ffi/src/volatility.rs b/datafusion/ffi/src/volatility.rs index f1705da294a3..a3faf8098b7f 100644 --- a/datafusion/ffi/src/volatility.rs +++ b/datafusion/ffi/src/volatility.rs @@ -16,7 +16,7 @@ // under the License. use abi_stable::StableAbi; -use datafusion::logical_expr::Volatility; +use datafusion_expr::Volatility; #[repr(C)] #[derive(Debug, StableAbi, Clone)] From 77cf9a9c63fb9174c3ca3e2e9b837e07779358de Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 17 Oct 2025 15:37:10 -0400 Subject: [PATCH 16/69] Intermediate work on FFI_Session --- datafusion/ffi/src/lib.rs | 2 +- .../{session_config.rs => session/config.rs} | 0 datafusion/ffi/src/session/mod.rs | 347 ++++++++++++++++++ datafusion/ffi/src/session/task.rs | 221 +++++++++++ 4 files changed, 569 insertions(+), 1 deletion(-) rename datafusion/ffi/src/{session_config.rs => session/config.rs} (100%) create mode 100644 datafusion/ffi/src/session/mod.rs create mode 100644 datafusion/ffi/src/session/task.rs diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index f08185c4a34f..adfdf5edc3b3 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -34,7 +34,7 @@ pub mod physical_expr; pub mod plan_properties; pub mod record_batch_stream; pub mod schema_provider; -pub mod session_config; +pub mod session; pub mod table_provider; pub mod table_source; pub mod udaf; diff --git a/datafusion/ffi/src/session_config.rs b/datafusion/ffi/src/session/config.rs similarity index 100% rename from datafusion/ffi/src/session_config.rs rename to datafusion/ffi/src/session/config.rs diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs new file mode 100644 index 000000000000..81d57dc0ffc3 --- /dev/null +++ b/datafusion/ffi/src/session/mod.rs @@ -0,0 +1,347 @@ +// 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::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; +use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; +use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; +use crate::{df_result, rresult_return}; +use abi_stable::{ + std_types::{ROption, RResult, RString, RVec}, + StableAbi, +}; +use datafusion_common::{not_impl_err, DFSchema}; +use datafusion_expr::expr_rewriter::FunctionRewrite; +use datafusion_expr::planner::ExprPlanner; +use datafusion_expr::{AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF}; +use std::collections::{HashMap, HashSet}; +use std::sync::Mutex; +use std::{ffi::c_void, sync::Arc}; +use std::any::Any; +use abi_stable::std_types::{RHashMap, RStr}; +use async_ffi::FfiFuture; +use datafusion::catalog::Session; +use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::execution::TaskContext; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionConfig; +use datafusion_common::config::{ConfigOptions, TableOptions}; +use datafusion_expr::execution_props::ExecutionProps; +use crate::execution_plan::FFI_ExecutionPlan; +use crate::plan_properties::{FFI_EmissionType, FFI_PlanProperties}; +use crate::session::config::FFI_SessionConfig; +use crate::session::task::FFI_TaskContext; +use crate::tests::create_test_schema; + +pub mod config; +mod task; + +/// A stable struct for sharing [`Session`] across FFI boundaries. +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_Session { + + pub session_id: unsafe extern "C" fn(&Self) -> RStr, + + pub config: unsafe extern "C" fn(&Self) -> FFI_SessionConfig, + + pub config_options: unsafe extern "C" fn(&Self) -> RHashMap, + + pub create_physical_plan: unsafe extern "C" fn( + &Self, + logical_plan_serialized: &RVec, + ) -> FfiFuture>, + + pub create_physical_expr: unsafe extern "C" fn(&Self, expr: Expr, df_schema: &DFSchema) -> RResult, RString>, + + pub scalar_functions: unsafe extern "C" fn(&Self) -> RHashMap, + + pub aggregate_functions: unsafe extern "C" fn(&Self) -> RHashMap, + + pub window_functions: unsafe extern "C" fn(&Self) -> RHashMap, + + // TODO: Expand scope of FFI to include runtime environment + // pub runtime_env: unsafe extern "C" fn(&Self) -> FFI_RuntimeEnv, + + // pub execution_props: unsafe extern "C" fn(&Self) -> FFI_ExecutionProps, + + pub table_options: unsafe extern "C" fn(&Self) -> RHashMap, + + pub default_table_options: unsafe extern "C" fn(&Self) -> RHashMap, + + pub task_ctx: unsafe extern "C" fn(&Self) -> FFI_TaskContext, + + /// Used to create a clone on the provider of the registry. This should + /// only need to be called by the receiver of the plan. + pub clone: unsafe extern "C" fn(plan: &Self) -> Self, + + /// Release the memory of the private data when it is no longer being used. + pub release: unsafe extern "C" fn(arg: &mut Self), + + /// Return the major DataFusion version number of this registry. + pub version: unsafe extern "C" fn() -> u64, + + /// Internal data. This is only to be accessed by the provider of the plan. + /// A [`ForeignSession`] should never attempt to access this data. + pub private_data: *mut c_void, +} + +unsafe impl Send for FFI_Session {} +unsafe impl Sync for FFI_Session {} + +struct SessionPrivateData { + registry: Arc, +} + +impl FFI_Session { + unsafe fn inner(&self) -> &Arc { + let private_data = self.private_data as *const SessionPrivateData; + &(*private_data).registry + } +} + +unsafe extern "C" fn session_id_fn_wrapper( + session: &FFI_Session, +) -> RStr { + let session = session.inner(); + session.session_id().into() +} + +unsafe extern "C" fn config_fn_wrapper( + session: &FFI_Session, +) -> FFI_SessionConfig { + let session = session.inner(); + session.config().into() +} + + +unsafe extern "C" fn config_options_fn_wrapper( + session: &FFI_Session, +) -> RHashMap { + let session = session.inner(); + session.config_options().into() +} + +unsafe extern "C" fn create_physical_plan_fn_wrapper( + session: &FFI_Session, + logical_plan_serialized: &RVec, +) -> FfiFuture> { + let session = session.inner(); + session.create_physical_plan().into() +} + +unsafe extern "C" fn create_physical_expr_fn_wrapper( + session: &FFI_Session, + expr: Expr, df_schema: &DFSchema, +) -> RResult, RString> { + let session = session.inner(); + session.create_physical_expr().into() +} + +unsafe extern "C" fn scalar_functions_fn_wrapper( + session: &FFI_Session, +) -> RHashMap { + let session = session.inner(); + session.scalar_functions().into() +} + +unsafe extern "C" fn aggregate_functions_fn_wrapper( + session: &FFI_Session, +) -> RHashMap { + let session = session.inner(); + session.aggregate_functions().into() +} + +unsafe extern "C" fn window_functions_fn_wrapper( + session: &FFI_Session, +) -> RHashMap { + let session = session.inner(); + session.window_functions().into() +} + +unsafe extern "C" fn table_options_fn_wrapper( + session: &FFI_Session, +) -> RHashMap { + let session = session.inner(); + session.table_options().into() +} + +unsafe extern "C" fn default_table_options_fn_wrapper( + session: &FFI_Session, +) -> RHashMap { + let session = session.inner(); + session.default_table_options().into() +} + + +unsafe extern "C" fn task_ctx_fn_wrapper( + session: &FFI_Session, +) -> FFI_TaskContext { + let session = session.inner(); + session.task_ctx().into() +} + +unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_Session) { + let private_data = Box::from_raw(provider.private_data as *mut SessionPrivateData); + drop(private_data); +} + +unsafe extern "C" fn clone_fn_wrapper( + provider: &FFI_Session, +) -> FFI_Session { + let old_private_data = provider.private_data as *const SessionPrivateData; + + let private_data = Box::into_raw(Box::new(SessionPrivateData { + registry: Arc::clone(&(*old_private_data).registry), + })) as *mut c_void; + + FFI_Session { + session_id: session_id_fn_wrapper, + config: config_fn_wrapper, + config_options: config_options_fn_wrapper, + create_physical_plan: create_physical_plan_fn_wrapper, + create_physical_expr: create_physical_expr_fn_wrapper, + scalar_functions: scalar_functions_fn_wrapper, + aggregate_functions: aggregate_functions_fn_wrapper, + window_functions: window_functions_fn_wrapper, + table_options: table_options_fn_wrapper, + default_table_options: default_table_options_fn_wrapper, + task_ctx: task_ctx_fn_wrapper, + + clone: clone_fn_wrapper, + release: release_fn_wrapper, + version: super::version, + private_data, + } +} + +impl Drop for FFI_Session { + fn drop(&mut self) { + unsafe { (self.release)(self) } + } +} + +impl FFI_Session { + /// Creates a new [`FFI_Session`]. + pub fn new(session: Arc) -> Self { + let private_data = Box::new(SessionPrivateData { registry }); + + Self { + session_id: session_id_fn_wrapper, + config: config_fn_wrapper, + config_options: config_options_fn_wrapper, + create_physical_plan: create_physical_plan_fn_wrapper, + create_physical_expr: create_physical_expr_fn_wrapper, + scalar_functions: scalar_functions_fn_wrapper, + aggregate_functions: aggregate_functions_fn_wrapper, + window_functions: window_functions_fn_wrapper, + table_options: table_options_fn_wrapper, + default_table_options: default_table_options_fn_wrapper, + task_ctx: task_ctx_fn_wrapper, + + clone: clone_fn_wrapper, + release: release_fn_wrapper, + version: super::version, + private_data: Box::into_raw(private_data) as *mut c_void, + } + } +} + +/// This wrapper struct exists on the receiver side of the FFI interface, so it has +/// no guarantees about being able to access the data in `private_data`. Any functions +/// defined on this struct must only use the stable functions provided in +/// FFI_Session to interact with the foreign table provider. +#[derive(Debug)] +pub struct ForeignSession(FFI_Session); + +unsafe impl Send for ForeignSession {} +unsafe impl Sync for ForeignSession {} + +impl From<&FFI_Session> for ForeignSession { + fn from(provider: &FFI_Session) -> Self { + Self(provider.clone()) + } +} + +impl Clone for FFI_Session { + fn clone(&self) -> Self { + unsafe { (self.clone)(self) } + } +} + +impl Session for ForeignSession { + fn session_id(&self) -> &str { + todo!() + } + + fn config(&self) -> &SessionConfig { + todo!() + } + + fn config_options(&self) -> &ConfigOptions { + todo!() + } + + async fn create_physical_plan(&self, logical_plan: &LogicalPlan) -> datafusion_common::Result> { + todo!() + } + + fn create_physical_expr(&self, expr: Expr, df_schema: &DFSchema) -> datafusion_common::Result> { + todo!() + } + + fn scalar_functions(&self) -> &HashMap> { + todo!() + } + + fn aggregate_functions(&self) -> &HashMap> { + todo!() + } + + fn window_functions(&self) -> &HashMap> { + todo!() + } + + fn runtime_env(&self) -> &Arc { + todo!() + } + + fn execution_props(&self) -> &ExecutionProps { + todo!() + } + + fn as_any(&self) -> &dyn Any { + todo!() + } + + fn table_options(&self) -> &TableOptions { + todo!() + } + + fn default_table_options(&self) -> TableOptions { + todo!() + } + + fn table_options_mut(&mut self) -> &mut TableOptions { + todo!() + } + + fn task_ctx(&self) -> Arc { + todo!() + } +} diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs new file mode 100644 index 000000000000..a668b6b4de37 --- /dev/null +++ b/datafusion/ffi/src/session/task.rs @@ -0,0 +1,221 @@ +// 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::session::config::FFI_SessionConfig; +use crate::udaf::FFI_AggregateUDF; +use crate::udf::FFI_ScalarUDF; +use crate::udwf::FFI_WindowUDF; +use crate::{arrow_wrappers::WrappedSchema, df_result, rresult_return}; +use abi_stable::pmr::ROption; +use abi_stable::std_types::{RHashMap, RStr}; +use abi_stable::{ + std_types::{ + RResult::{self, ROk}, + RString, RVec, + }, + StableAbi, +}; +use arrow::datatypes::SchemaRef; +use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::execution::TaskContext; +use datafusion::prelude::SessionConfig; +use datafusion::{ + error::{DataFusionError, Result}, + physical_expr::EquivalenceProperties, + physical_plan::execution_plan::{Boundedness, EmissionType}, + prelude::SessionContext, +}; +use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use datafusion_proto::{ + physical_plan::{ + from_proto::{parse_physical_sort_exprs, parse_protobuf_partitioning}, + to_proto::{serialize_partitioning, serialize_physical_sort_exprs}, + DefaultPhysicalExtensionCodec, + }, + protobuf::{Partitioning, PhysicalSortExprNodeCollection}, +}; +use prost::Message; +use std::collections::HashMap; +use std::{ffi::c_void, sync::Arc}; +use datafusion::catalog::SchemaProvider; + +/// A stable struct for sharing [`TaskContext`] across FFI boundaries. +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_TaskContext { + pub session_id: unsafe extern "C" fn(&Self) -> RStr, + + pub task_id: unsafe extern "C" fn(&Self) -> ROption, + + pub session_config: unsafe extern "C" fn(&Self) -> FFI_SessionConfig, + + pub scalar_functions: + unsafe extern "C" fn(&Self) -> RHashMap, + + pub aggregate_functions: + unsafe extern "C" fn(&Self) -> RHashMap, + + pub window_functions: + unsafe extern "C" fn(&Self) -> RHashMap, + + /// Release the memory of the private data when it is no longer being used. + pub release: unsafe extern "C" fn(arg: &mut Self), + + /// Internal data. This is only to be accessed by the provider of the plan. + /// The foreign library should never attempt to access this data. + pub private_data: *mut c_void, +} + +struct TaskContextPrivateData { + ctx: TaskContext, +} + +impl FFI_TaskContext { + unsafe fn inner(&self) -> &TaskContext { + let private_data = self.private_data as *const TaskContextPrivateData; + &(*private_data).ctx + } +} + +unsafe extern "C" fn session_id_fn_wrapper(ctx: &FFI_TaskContext) -> RStr { + let ctx = ctx.inner(); + ctx.session_id().as_str().into() +} + +unsafe extern "C" fn task_id_fn_wrapper(ctx: &FFI_TaskContext) -> ROption { + let ctx = ctx.inner(); + ctx.session_id().map(|s| s.as_str().into()).into() +} + +unsafe extern "C" fn session_config_fn_wrapper(ctx: &FFI_TaskContext) -> FFI_SessionConfig { + let ctx = ctx.inner(); + ctx.session_config().into() +} + +unsafe extern "C" fn scalar_functions_fn_wrapper(ctx: &FFI_TaskContext) -> RHashMap { + let ctx = ctx.inner(); + ctx.scalar_functions() + .into_iter() + .map(|(name, udf)| (name.into(), udf.into())) + .collect() +} + +unsafe extern "C" fn aggregate_functions_fn_wrapper(ctx: &FFI_TaskContext) -> RHashMap { + let ctx = ctx.inner(); + ctx.aggregate_functions() + .into_iter() + .map(|(name, udf)| (name.into(), udf.into())) + .collect() +} + +unsafe extern "C" fn window_functions_fn_wrapper(ctx: &FFI_TaskContext) -> RHashMap { + let ctx = ctx.inner(); + ctx.window_functions() + .into_iter() + .map(|(name, udf)| (name.into(), udf.into())) + .collect() +} + +unsafe extern "C" fn release_fn_wrapper(ctx: &mut FFI_TaskContext) { + let private_data = Box::from_raw(ctx.private_data as *mut TaskContextPrivateData); + drop(private_data); +} + +impl Drop for FFI_TaskContext { + fn drop(&mut self) { + unsafe { (self.release)(self) } + } +} + +impl From<&TaskContext> for FFI_TaskContext { + fn from(ctx: TaskContext) -> Self { + let private_data = Box::new(TaskContextPrivateData { ctx }); + + FFI_TaskContext { + session_id: session_id_fn_wrapper, + task_id: task_id_fn_wrapper, + session_config: session_config_fn_wrapper, + scalar_functions: scalar_functions_fn_wrapper, + aggregate_functions: aggregate_functions_fn_wrapper, + window_functions: window_functions_fn_wrapper, + release: release_fn_wrapper, + private_data: Box::into_raw(private_data) as *mut c_void, + } + } +} + +impl TryFrom for TaskContext { + type Error = DataFusionError; + + fn try_from(ffi_ctx: FFI_TaskContext) -> Result { + unsafe { + let task_id = (ffi_ctx.task_id)(&ffi_ctx).map(|s| s.to_string()).into(); + let sesion_id = (ffi_ctx.session_id)(&ffi_ctx).into(); + let session_config = (ffi_ctx.session_config)(&ffi_ctx).into(); + + let scalar_functions = (ffi_ctx.scalar_functions)(&ffi_ctx).into(); + let aggregate_functions = (ffi_ctx.aggregate_functions)(&ffi_ctx).into(); + let window_functions = (ffi_ctx.window_functions)(&ffi_ctx).into(); + + let runtime = Arc::new(RuntimeEnv::default()); + + Ok(TaskContext::new( + task_id, + sesion_id, + session_config, + scalar_functions, + aggregate_functions, + window_functions, + runtime, + )) + } + } +} +// +// #[cfg(test)] +// mod tests { +// use datafusion::{physical_expr::PhysicalSortExpr, physical_plan::Partitioning}; +// +// use super::*; +// +// #[test] +// fn test_round_trip_ffi_plan_properties() -> Result<()> { +// use arrow::datatypes::{DataType, Field, Schema}; +// let schema = +// Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, false)])); +// +// let mut eqp = EquivalenceProperties::new(Arc::clone(&schema)); +// let _ = eqp.reorder([PhysicalSortExpr::new_default( +// datafusion::physical_plan::expressions::col("a", &schema)?, +// )]); +// let original_ctx = TaskContext::new( +// eqp, +// Partitioning::RoundRobinBatch(3), +// EmissionType::Incremental, +// Boundedness::Bounded, +// ); +// +// let local_ctx_ptr = FFI_TaskContext::from(&original_ctx); +// +// let foreign_ctx: TaskContext = local_ctx_ptr.try_into()?; +// +// assert_eq!(format!("{foreign_props:?}"), format!("{original_props:?}")); +// +// Ok(()) +// } +// } From 211c4d0dec7d657a6d736b875106b0344103588c Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sat, 18 Oct 2025 07:15:48 -0400 Subject: [PATCH 17/69] Implement FFI_Session --- datafusion/ffi/src/session/mod.rs | 369 ++++++++++++++++++++------- datafusion/ffi/src/session/task.rs | 161 +++++++----- datafusion/ffi/src/table_provider.rs | 11 +- datafusion/ffi/src/udaf/mod.rs | 6 + datafusion/ffi/src/udf/mod.rs | 6 + datafusion/ffi/src/udwf/mod.rs | 6 + 6 files changed, 400 insertions(+), 159 deletions(-) diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 81d57dc0ffc3..ac198f78bd4b 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -15,24 +15,23 @@ // specific language governing permissions and limitations // under the License. +use crate::arrow_wrappers::WrappedSchema; +use crate::execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}; +use crate::session::config::{FFI_SessionConfig, ForeignSessionConfig}; +use crate::session::task::FFI_TaskContext; use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; -use crate::{df_result, rresult_return}; +use crate::{df_result, rresult, rresult_return}; +use abi_stable::std_types::{RHashMap, RStr}; use abi_stable::{ - std_types::{ROption, RResult, RString, RVec}, + std_types::{RResult, RString, RVec}, StableAbi, }; -use datafusion_common::{not_impl_err, DFSchema}; -use datafusion_expr::expr_rewriter::FunctionRewrite; -use datafusion_expr::planner::ExprPlanner; -use datafusion_expr::{AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF}; -use std::collections::{HashMap, HashSet}; -use std::sync::Mutex; -use std::{ffi::c_void, sync::Arc}; -use std::any::Any; -use abi_stable::std_types::{RHashMap, RStr}; -use async_ffi::FfiFuture; +use arrow_schema::ffi::FFI_ArrowSchema; +use arrow_schema::SchemaRef; +use async_ffi::{FfiFuture, FutureExt}; +use async_trait::async_trait; use datafusion::catalog::Session; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::TaskContext; @@ -40,12 +39,23 @@ use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionConfig; use datafusion_common::config::{ConfigOptions, TableOptions}; +use datafusion_common::{DFSchema, DataFusionError}; use datafusion_expr::execution_props::ExecutionProps; -use crate::execution_plan::FFI_ExecutionPlan; -use crate::plan_properties::{FFI_EmissionType, FFI_PlanProperties}; -use crate::session::config::FFI_SessionConfig; -use crate::session::task::FFI_TaskContext; -use crate::tests::create_test_schema; +use datafusion_expr::registry::FunctionRegistry; +use datafusion_expr::{AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF}; +use datafusion_proto::bytes::{logical_plan_from_bytes, logical_plan_to_bytes}; +use datafusion_proto::logical_plan::from_proto::parse_expr; +use datafusion_proto::logical_plan::to_proto::serialize_expr; +use datafusion_proto::logical_plan::DefaultLogicalExtensionCodec; +use datafusion_proto::physical_plan::from_proto::parse_physical_expr; +use datafusion_proto::physical_plan::to_proto::serialize_physical_expr; +use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; +use datafusion_proto::protobuf::{LogicalExprNode, PhysicalExprNode}; +use prost::Message; +use std::any::Any; +use std::collections::HashMap; +use std::{ffi::c_void, sync::Arc}; +use tokio::runtime::Handle; pub mod config; mod task; @@ -55,23 +65,26 @@ mod task; #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] pub struct FFI_Session { - pub session_id: unsafe extern "C" fn(&Self) -> RStr, pub config: unsafe extern "C" fn(&Self) -> FFI_SessionConfig, - pub config_options: unsafe extern "C" fn(&Self) -> RHashMap, + pub create_physical_plan: + unsafe extern "C" fn( + &Self, + logical_plan_serialized: RVec, + ) -> FfiFuture>, - pub create_physical_plan: unsafe extern "C" fn( + pub create_physical_expr: unsafe extern "C" fn( &Self, - logical_plan_serialized: &RVec, - ) -> FfiFuture>, - - pub create_physical_expr: unsafe extern "C" fn(&Self, expr: Expr, df_schema: &DFSchema) -> RResult, RString>, + expr_serialized: RVec, + schema: WrappedSchema, + ) -> RResult, RString>, pub scalar_functions: unsafe extern "C" fn(&Self) -> RHashMap, - pub aggregate_functions: unsafe extern "C" fn(&Self) -> RHashMap, + pub aggregate_functions: + unsafe extern "C" fn(&Self) -> RHashMap, pub window_functions: unsafe extern "C" fn(&Self) -> RHashMap, @@ -79,7 +92,6 @@ pub struct FFI_Session { // pub runtime_env: unsafe extern "C" fn(&Self) -> FFI_RuntimeEnv, // pub execution_props: unsafe extern "C" fn(&Self) -> FFI_ExecutionProps, - pub table_options: unsafe extern "C" fn(&Self) -> RHashMap, pub default_table_options: unsafe extern "C" fn(&Self) -> RHashMap, @@ -105,95 +117,153 @@ unsafe impl Send for FFI_Session {} unsafe impl Sync for FFI_Session {} struct SessionPrivateData { - registry: Arc, + session: Arc, + function_registry: Arc, + runtime: Handle, } impl FFI_Session { unsafe fn inner(&self) -> &Arc { let private_data = self.private_data as *const SessionPrivateData; - &(*private_data).registry + &(*private_data).session + } + + unsafe fn function_registry(&self) -> &Arc { + let private_data = self.private_data as *const SessionPrivateData; + &(*private_data).function_registry + } + + unsafe fn runtime(&self) -> &Handle { + let private_data = self.private_data as *const SessionPrivateData; + &(*private_data).runtime } } -unsafe extern "C" fn session_id_fn_wrapper( - session: &FFI_Session, -) -> RStr { +unsafe extern "C" fn session_id_fn_wrapper(session: &FFI_Session) -> RStr<'_> { let session = session.inner(); session.session_id().into() } -unsafe extern "C" fn config_fn_wrapper( - session: &FFI_Session, -) -> FFI_SessionConfig { +unsafe extern "C" fn config_fn_wrapper(session: &FFI_Session) -> FFI_SessionConfig { let session = session.inner(); session.config().into() } - -unsafe extern "C" fn config_options_fn_wrapper( - session: &FFI_Session, -) -> RHashMap { - let session = session.inner(); - session.config_options().into() -} - unsafe extern "C" fn create_physical_plan_fn_wrapper( session: &FFI_Session, - logical_plan_serialized: &RVec, + logical_plan_serialized: RVec, ) -> FfiFuture> { - let session = session.inner(); - session.create_physical_plan().into() + let runtime = session.runtime().clone(); + let session = Arc::clone(session.inner()); + async move { + let task_ctx = session.task_ctx(); + + let logical_plan = rresult_return!(logical_plan_from_bytes( + logical_plan_serialized.as_slice(), + &task_ctx + )); + + let physical_plan = session.create_physical_plan(&logical_plan).await; + + rresult!(physical_plan.map(|plan| FFI_ExecutionPlan::new( + plan, + task_ctx, + Some(runtime) + ))) + } + .into_ffi() } unsafe extern "C" fn create_physical_expr_fn_wrapper( session: &FFI_Session, - expr: Expr, df_schema: &DFSchema, + expr_serialized: RVec, + schema: WrappedSchema, ) -> RResult, RString> { + let function_registry = session.function_registry(); let session = session.inner(); - session.create_physical_expr().into() + + let codec = DefaultLogicalExtensionCodec {}; + let logical_expr = LogicalExprNode::decode(expr_serialized.as_slice()).unwrap(); + let logical_expr = + parse_expr(&logical_expr, function_registry.as_ref(), &codec).unwrap(); + let schema: SchemaRef = schema.into(); + let schema: DFSchema = rresult_return!((schema).try_into()); + + let physical_expr = + rresult_return!(session.create_physical_expr(logical_expr, &schema)); + let codec = DefaultPhysicalExtensionCodec {}; + let physical_expr = + rresult_return!(serialize_physical_expr(&physical_expr, &codec)).encode_to_vec(); + + RResult::ROk(physical_expr.into()) } unsafe extern "C" fn scalar_functions_fn_wrapper( session: &FFI_Session, ) -> RHashMap { let session = session.inner(); - session.scalar_functions().into() + session + .scalar_functions() + .iter() + .map(|(name, udf)| (name.clone().into(), FFI_ScalarUDF::from(Arc::clone(udf)))) + .collect() } unsafe extern "C" fn aggregate_functions_fn_wrapper( session: &FFI_Session, ) -> RHashMap { let session = session.inner(); - session.aggregate_functions().into() + session + .aggregate_functions() + .iter() + .map(|(name, udaf)| { + ( + name.clone().into(), + FFI_AggregateUDF::from(Arc::clone(udaf)), + ) + }) + .collect() } unsafe extern "C" fn window_functions_fn_wrapper( session: &FFI_Session, ) -> RHashMap { let session = session.inner(); - session.window_functions().into() + session + .window_functions() + .iter() + .map(|(name, udwf)| (name.clone().into(), FFI_WindowUDF::from(Arc::clone(udwf)))) + .collect() +} + +fn table_options_to_rhash(options: &TableOptions) -> RHashMap { + options + .entries() + .into_iter() + .filter_map(|entry| entry.value.map(|v| (entry.key.into(), v.into()))) + .collect() } unsafe extern "C" fn table_options_fn_wrapper( session: &FFI_Session, ) -> RHashMap { let session = session.inner(); - session.table_options().into() + let table_options = session.table_options(); + table_options_to_rhash(table_options) } unsafe extern "C" fn default_table_options_fn_wrapper( session: &FFI_Session, ) -> RHashMap { let session = session.inner(); - session.default_table_options().into() -} + let table_options = session.default_table_options(); + table_options_to_rhash(&table_options) +} -unsafe extern "C" fn task_ctx_fn_wrapper( - session: &FFI_Session, -) -> FFI_TaskContext { +unsafe extern "C" fn task_ctx_fn_wrapper(session: &FFI_Session) -> FFI_TaskContext { let session = session.inner(); - session.task_ctx().into() + FFI_TaskContext::from(session.task_ctx()) } unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_Session) { @@ -201,19 +271,18 @@ unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_Session) { drop(private_data); } -unsafe extern "C" fn clone_fn_wrapper( - provider: &FFI_Session, -) -> FFI_Session { +unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_Session) -> FFI_Session { let old_private_data = provider.private_data as *const SessionPrivateData; let private_data = Box::into_raw(Box::new(SessionPrivateData { - registry: Arc::clone(&(*old_private_data).registry), + session: Arc::clone(&(*old_private_data).session), + function_registry: Arc::clone(&(*old_private_data).function_registry), + runtime: (*old_private_data).runtime.clone(), })) as *mut c_void; FFI_Session { session_id: session_id_fn_wrapper, config: config_fn_wrapper, - config_options: config_options_fn_wrapper, create_physical_plan: create_physical_plan_fn_wrapper, create_physical_expr: create_physical_expr_fn_wrapper, scalar_functions: scalar_functions_fn_wrapper, @@ -238,13 +307,20 @@ impl Drop for FFI_Session { impl FFI_Session { /// Creates a new [`FFI_Session`]. - pub fn new(session: Arc) -> Self { - let private_data = Box::new(SessionPrivateData { registry }); + pub fn new( + session: Arc, + function_registry: Arc, + runtime: Handle, + ) -> Self { + let private_data = Box::new(SessionPrivateData { + session, + function_registry, + runtime, + }); Self { session_id: session_id_fn_wrapper, config: config_fn_wrapper, - config_options: config_options_fn_wrapper, create_physical_plan: create_physical_plan_fn_wrapper, create_physical_expr: create_physical_expr_fn_wrapper, scalar_functions: scalar_functions_fn_wrapper, @@ -267,14 +343,75 @@ impl FFI_Session { /// defined on this struct must only use the stable functions provided in /// FFI_Session to interact with the foreign table provider. #[derive(Debug)] -pub struct ForeignSession(FFI_Session); +pub struct ForeignSession { + session: FFI_Session, + config: SessionConfig, + scalar_functions: HashMap>, + aggregate_functions: HashMap>, + window_functions: HashMap>, + table_options: TableOptions, + runtime_env: Arc, + props: ExecutionProps, +} unsafe impl Send for ForeignSession {} unsafe impl Sync for ForeignSession {} -impl From<&FFI_Session> for ForeignSession { - fn from(provider: &FFI_Session) -> Self { - Self(provider.clone()) +impl TryFrom<&FFI_Session> for ForeignSession { + type Error = DataFusionError; + fn try_from(session: &FFI_Session) -> Result { + unsafe { + let table_options = + table_options_from_rhashmap((session.table_options)(session)); + + let config = (session.config)(session); + let config = ForeignSessionConfig::try_from(&config)?.0; + + let scalar_functions = (session.scalar_functions)(session) + .into_iter() + .map(|kv_pair| { + let udf = ForeignScalarUDF::try_from(&kv_pair.1)?; + + Ok(( + kv_pair.0.into_string(), + Arc::new(ScalarUDF::new_from_impl(udf)), + )) + }) + .collect::>()?; + let aggregate_functions = (session.aggregate_functions)(session) + .into_iter() + .map(|kv_pair| { + let udaf = ForeignAggregateUDF::try_from(&kv_pair.1)?; + + Ok(( + kv_pair.0.into_string(), + Arc::new(AggregateUDF::new_from_impl(udaf)), + )) + }) + .collect::>()?; + let window_functions = (session.window_functions)(session) + .into_iter() + .map(|kv_pair| { + let udwf = ForeignWindowUDF::try_from(&kv_pair.1)?; + + Ok(( + kv_pair.0.into_string(), + Arc::new(WindowUDF::new_from_impl(udwf)), + )) + }) + .collect::>()?; + + Ok(Self { + session: session.clone(), + config, + table_options, + scalar_functions, + aggregate_functions, + window_functions, + runtime_env: Default::default(), + props: Default::default(), + }) + } } } @@ -284,64 +421,124 @@ impl Clone for FFI_Session { } } +fn table_options_from_rhashmap(options: RHashMap) -> TableOptions { + let options = options + .into_iter() + .map(|kv_pair| (kv_pair.0.into_string(), kv_pair.1.into_string())) + .collect(); + + TableOptions::from_string_hash_map(&options).unwrap_or_else(|err| { + log::warn!("Error parsing default table options: {err}"); + TableOptions::default() + }) +} + +#[async_trait] impl Session for ForeignSession { fn session_id(&self) -> &str { - todo!() + unsafe { (self.session.session_id)(&self.session).as_str() } } fn config(&self) -> &SessionConfig { - todo!() + &self.config } fn config_options(&self) -> &ConfigOptions { - todo!() + self.config.options() } - async fn create_physical_plan(&self, logical_plan: &LogicalPlan) -> datafusion_common::Result> { - todo!() + async fn create_physical_plan( + &self, + logical_plan: &LogicalPlan, + ) -> datafusion_common::Result> { + unsafe { + let logical_plan = logical_plan_to_bytes(logical_plan)?; + let physical_plan = df_result!( + (self.session.create_physical_plan)( + &self.session, + logical_plan.as_ref().into() + ) + .await + )?; + let physical_plan = ForeignExecutionPlan::try_from(&physical_plan)?; + + Ok(Arc::new(physical_plan)) + } } - fn create_physical_expr(&self, expr: Expr, df_schema: &DFSchema) -> datafusion_common::Result> { - todo!() + fn create_physical_expr( + &self, + expr: Expr, + df_schema: &DFSchema, + ) -> datafusion_common::Result> { + unsafe { + let codec = DefaultLogicalExtensionCodec {}; + let logical_expr = serialize_expr(&expr, &codec)?.encode_to_vec(); + let schema = WrappedSchema(FFI_ArrowSchema::try_from(df_schema.as_arrow())?); + + let physical_expr = df_result!((self.session.create_physical_expr)( + &self.session, + logical_expr.into(), + schema + ))?; + + let physical_expr = PhysicalExprNode::decode(physical_expr.as_slice()) + .map_err(|err| DataFusionError::External(Box::new(err)))?; + + let codec = DefaultPhysicalExtensionCodec {}; + let physical_expr = parse_physical_expr( + &physical_expr, + self.task_ctx().as_ref(), + df_schema.as_arrow(), + &codec, + )?; + + Ok(physical_expr) + } } fn scalar_functions(&self) -> &HashMap> { - todo!() + &self.scalar_functions } fn aggregate_functions(&self) -> &HashMap> { - todo!() + &self.aggregate_functions } fn window_functions(&self) -> &HashMap> { - todo!() + &self.window_functions } fn runtime_env(&self) -> &Arc { - todo!() + &self.runtime_env } fn execution_props(&self) -> &ExecutionProps { - todo!() + &self.props } fn as_any(&self) -> &dyn Any { - todo!() + self } fn table_options(&self) -> &TableOptions { - todo!() + &self.table_options } fn default_table_options(&self) -> TableOptions { - todo!() + unsafe { + table_options_from_rhashmap((self.session.default_table_options)( + &self.session, + )) + } } fn table_options_mut(&mut self) -> &mut TableOptions { - todo!() + log::warn!("Mutating table options is not supported via FFI. Changes will not have an effect."); + &mut self.table_options } fn task_ctx(&self) -> Arc { - todo!() + unsafe { Arc::new((self.session.task_ctx)(&self.session).into()) } } } diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index a668b6b4de37..521514510f15 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -15,63 +15,35 @@ // specific language governing permissions and limitations // under the License. -use crate::session::config::FFI_SessionConfig; -use crate::udaf::FFI_AggregateUDF; -use crate::udf::FFI_ScalarUDF; -use crate::udwf::FFI_WindowUDF; -use crate::{arrow_wrappers::WrappedSchema, df_result, rresult_return}; +use crate::session::config::{FFI_SessionConfig, ForeignSessionConfig}; +use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; +use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; +use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use abi_stable::pmr::ROption; -use abi_stable::std_types::{RHashMap, RStr}; -use abi_stable::{ - std_types::{ - RResult::{self, ROk}, - RString, RVec, - }, - StableAbi, -}; -use arrow::datatypes::SchemaRef; +use abi_stable::std_types::RHashMap; +use abi_stable::{std_types::RString, StableAbi}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::TaskContext; -use datafusion::prelude::SessionConfig; -use datafusion::{ - error::{DataFusionError, Result}, - physical_expr::EquivalenceProperties, - physical_plan::execution_plan::{Boundedness, EmissionType}, - prelude::SessionContext, -}; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; -use datafusion_proto::{ - physical_plan::{ - from_proto::{parse_physical_sort_exprs, parse_protobuf_partitioning}, - to_proto::{serialize_partitioning, serialize_physical_sort_exprs}, - DefaultPhysicalExtensionCodec, - }, - protobuf::{Partitioning, PhysicalSortExprNodeCollection}, -}; -use prost::Message; -use std::collections::HashMap; use std::{ffi::c_void, sync::Arc}; -use datafusion::catalog::SchemaProvider; /// A stable struct for sharing [`TaskContext`] across FFI boundaries. #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] pub struct FFI_TaskContext { - pub session_id: unsafe extern "C" fn(&Self) -> RStr, + pub session_id: unsafe extern "C" fn(&Self) -> RString, - pub task_id: unsafe extern "C" fn(&Self) -> ROption, + pub task_id: unsafe extern "C" fn(&Self) -> ROption, pub session_config: unsafe extern "C" fn(&Self) -> FFI_SessionConfig, - pub scalar_functions: - unsafe extern "C" fn(&Self) -> RHashMap, + pub scalar_functions: unsafe extern "C" fn(&Self) -> RHashMap, pub aggregate_functions: unsafe extern "C" fn(&Self) -> RHashMap, - pub window_functions: - unsafe extern "C" fn(&Self) -> RHashMap, + pub window_functions: unsafe extern "C" fn(&Self) -> RHashMap, /// Release the memory of the private data when it is no longer being used. pub release: unsafe extern "C" fn(arg: &mut Self), @@ -82,7 +54,7 @@ pub struct FFI_TaskContext { } struct TaskContextPrivateData { - ctx: TaskContext, + ctx: Arc, } impl FFI_TaskContext { @@ -92,42 +64,50 @@ impl FFI_TaskContext { } } -unsafe extern "C" fn session_id_fn_wrapper(ctx: &FFI_TaskContext) -> RStr { +unsafe extern "C" fn session_id_fn_wrapper(ctx: &FFI_TaskContext) -> RString { let ctx = ctx.inner(); - ctx.session_id().as_str().into() + ctx.session_id().into() } -unsafe extern "C" fn task_id_fn_wrapper(ctx: &FFI_TaskContext) -> ROption { +unsafe extern "C" fn task_id_fn_wrapper(ctx: &FFI_TaskContext) -> ROption { let ctx = ctx.inner(); - ctx.session_id().map(|s| s.as_str().into()).into() + ctx.task_id().map(|s| s.as_str().into()).into() } -unsafe extern "C" fn session_config_fn_wrapper(ctx: &FFI_TaskContext) -> FFI_SessionConfig { +unsafe extern "C" fn session_config_fn_wrapper( + ctx: &FFI_TaskContext, +) -> FFI_SessionConfig { let ctx = ctx.inner(); ctx.session_config().into() } -unsafe extern "C" fn scalar_functions_fn_wrapper(ctx: &FFI_TaskContext) -> RHashMap { +unsafe extern "C" fn scalar_functions_fn_wrapper( + ctx: &FFI_TaskContext, +) -> RHashMap { let ctx = ctx.inner(); ctx.scalar_functions() - .into_iter() - .map(|(name, udf)| (name.into(), udf.into())) + .iter() + .map(|(name, udf)| (name.to_owned().into(), udf.into())) .collect() } -unsafe extern "C" fn aggregate_functions_fn_wrapper(ctx: &FFI_TaskContext) -> RHashMap { +unsafe extern "C" fn aggregate_functions_fn_wrapper( + ctx: &FFI_TaskContext, +) -> RHashMap { let ctx = ctx.inner(); ctx.aggregate_functions() - .into_iter() - .map(|(name, udf)| (name.into(), udf.into())) + .iter() + .map(|(name, udf)| (name.to_owned().into(), udf.into())) .collect() } -unsafe extern "C" fn window_functions_fn_wrapper(ctx: &FFI_TaskContext) -> RHashMap { +unsafe extern "C" fn window_functions_fn_wrapper( + ctx: &FFI_TaskContext, +) -> RHashMap { let ctx = ctx.inner(); ctx.window_functions() - .into_iter() - .map(|(name, udf)| (name.into(), udf.into())) + .iter() + .map(|(name, udf)| (name.to_owned().into(), udf.into())) .collect() } @@ -142,8 +122,8 @@ impl Drop for FFI_TaskContext { } } -impl From<&TaskContext> for FFI_TaskContext { - fn from(ctx: TaskContext) -> Self { +impl From> for FFI_TaskContext { + fn from(ctx: Arc) -> Self { let private_data = Box::new(TaskContextPrivateData { ctx }); FFI_TaskContext { @@ -159,22 +139,71 @@ impl From<&TaskContext> for FFI_TaskContext { } } -impl TryFrom for TaskContext { - type Error = DataFusionError; - - fn try_from(ffi_ctx: FFI_TaskContext) -> Result { +impl From for TaskContext { + fn from(ffi_ctx: FFI_TaskContext) -> Self { unsafe { let task_id = (ffi_ctx.task_id)(&ffi_ctx).map(|s| s.to_string()).into(); let sesion_id = (ffi_ctx.session_id)(&ffi_ctx).into(); - let session_config = (ffi_ctx.session_config)(&ffi_ctx).into(); - - let scalar_functions = (ffi_ctx.scalar_functions)(&ffi_ctx).into(); - let aggregate_functions = (ffi_ctx.aggregate_functions)(&ffi_ctx).into(); - let window_functions = (ffi_ctx.window_functions)(&ffi_ctx).into(); + let session_config = (ffi_ctx.session_config)(&ffi_ctx); + let session_config = ForeignSessionConfig::try_from(&session_config) + .map(|v| v.0) + .unwrap_or_default(); + + let scalar_functions = (ffi_ctx.scalar_functions)(&ffi_ctx) + .into_iter() + .filter_map(|kv_pair| { + let udf = ForeignScalarUDF::try_from(&kv_pair.1); + + if let Err(err) = &udf { + log::error!("Unable to create WindowUDF in FFI: {err}") + } + + udf.ok().map(|udf| { + ( + kv_pair.0.into_string(), + Arc::new(ScalarUDF::new_from_impl(udf)), + ) + }) + }) + .collect(); + let aggregate_functions = (ffi_ctx.aggregate_functions)(&ffi_ctx) + .into_iter() + .filter_map(|kv_pair| { + let udaf = ForeignAggregateUDF::try_from(&kv_pair.1); + + if let Err(err) = &udaf { + log::error!("Unable to create AggregateUDF in FFI: {err}") + } + + udaf.ok().map(|udaf| { + ( + kv_pair.0.into_string(), + Arc::new(AggregateUDF::new_from_impl(udaf)), + ) + }) + }) + .collect(); + let window_functions = (ffi_ctx.window_functions)(&ffi_ctx) + .into_iter() + .filter_map(|kv_pair| { + let udwf = ForeignWindowUDF::try_from(&kv_pair.1); + + if let Err(err) = &udwf { + log::error!("Unable to create WindowUDF in FFI: {err}") + } + + udwf.ok().map(|udwf| { + ( + kv_pair.0.into_string(), + Arc::new(WindowUDF::new_from_impl(udwf)), + ) + }) + }) + .collect(); let runtime = Arc::new(RuntimeEnv::default()); - Ok(TaskContext::new( + TaskContext::new( task_id, sesion_id, session_config, @@ -182,7 +211,7 @@ impl TryFrom for TaskContext { aggregate_functions, window_functions, runtime, - )) + ) } } } diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index a37c1f416f2e..878b67c977a2 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -37,15 +37,12 @@ use tokio::runtime::Handle; use crate::{ arrow_wrappers::WrappedSchema, - df_result, rresult_return, - session_config::ForeignSessionConfig, - table_source::{FFI_TableProviderFilterPushDown, FFI_TableType}, -}; - -use super::{ + df_result, execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}, insert_op::FFI_InsertOp, - session_config::FFI_SessionConfig, + rresult_return, + session::config::{FFI_SessionConfig, ForeignSessionConfig}, + table_source::{FFI_TableProviderFilterPushDown, FFI_TableType}, }; use datafusion::error::Result; use datafusion_common::DataFusionError; diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index f77dcf3a69ce..ff9104981989 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -332,6 +332,12 @@ impl Clone for FFI_AggregateUDF { } } +impl From<&Arc> for FFI_AggregateUDF { + fn from(udf: &Arc) -> Self { + Arc::clone(udf).into() + } +} + impl From> for FFI_AggregateUDF { fn from(udaf: Arc) -> Self { let name = udaf.name().into(); diff --git a/datafusion/ffi/src/udf/mod.rs b/datafusion/ffi/src/udf/mod.rs index 1a31efecb22f..a834066041f9 100644 --- a/datafusion/ffi/src/udf/mod.rs +++ b/datafusion/ffi/src/udf/mod.rs @@ -236,6 +236,12 @@ impl Clone for FFI_ScalarUDF { } } +impl From<&Arc> for FFI_ScalarUDF { + fn from(udf: &Arc) -> Self { + Arc::clone(udf).into() + } +} + impl From> for FFI_ScalarUDF { fn from(udf: Arc) -> Self { let name = udf.name().into(); diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index d9459f89be04..7e53a4e9cbe0 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -204,6 +204,12 @@ impl Clone for FFI_WindowUDF { } } +impl From<&Arc> for FFI_WindowUDF { + fn from(udwf: &Arc) -> Self { + Arc::clone(udwf).into() + } +} + impl From> for FFI_WindowUDF { fn from(udf: Arc) -> Self { let name = udf.name().into(); From d6debab9f0daf73e0176a68104ed9cc796e57735 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 4 Nov 2025 13:34:27 -0500 Subject: [PATCH 18/69] More intermediate work --- Cargo.lock | 4 + .../ffi/ffi_example_table_provider/src/lib.rs | 5 +- .../ffi/ffi_module_interface/src/lib.rs | 3 +- datafusion/ffi/Cargo.toml | 8 +- datafusion/ffi/src/catalog_provider.rs | 54 ++++++++--- datafusion/ffi/src/execution_plan.rs | 6 +- datafusion/ffi/src/expr/columnar_value.rs | 2 +- datafusion/ffi/src/expr/distribution.rs | 2 +- datafusion/ffi/src/expr/expr_properties.rs | 2 +- datafusion/ffi/src/expr/interval.rs | 2 +- datafusion/ffi/src/function_registry.rs | 4 +- datafusion/ffi/src/lib.rs | 6 ++ datafusion/ffi/src/physical_expr/mod.rs | 28 +++--- .../ffi/src/physical_expr/partitioning.rs | 2 +- datafusion/ffi/src/physical_expr/sort.rs | 5 +- datafusion/ffi/src/record_batch_stream.rs | 2 +- datafusion/ffi/src/schema_provider.rs | 32 +++++-- datafusion/ffi/src/session/config.rs | 7 +- datafusion/ffi/src/session/mod.rs | 58 ++++++------ datafusion/ffi/src/session/task.rs | 4 +- datafusion/ffi/src/table_provider.rs | 94 +++++++++++-------- datafusion/ffi/src/tests/async_provider.rs | 43 ++++----- datafusion/ffi/src/tests/catalog.rs | 20 ++-- datafusion/ffi/src/tests/mod.rs | 19 ++-- datafusion/ffi/src/tests/sync_provider.rs | 9 +- datafusion/ffi/src/tests/udf_udaf_udwf.rs | 24 ++--- datafusion/ffi/src/tests/utils.rs | 2 +- datafusion/ffi/src/udaf/accumulator_args.rs | 6 +- datafusion/ffi/src/udaf/groups_accumulator.rs | 4 +- datafusion/ffi/src/udtf.rs | 51 ++++++---- .../ffi/src/udwf/partition_evaluator_args.rs | 6 +- datafusion/ffi/tests/ffi_integration.rs | 13 ++- datafusion/ffi/tests/ffi_udtf.rs | 8 +- 33 files changed, 314 insertions(+), 221 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0c4754bd63ce..c0c64367fe0a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2297,7 +2297,11 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-expr-common", + "datafusion-functions", + "datafusion-functions-aggregate", "datafusion-functions-aggregate-common", + "datafusion-functions-table", + "datafusion-functions-window", "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", diff --git a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs index a83f15926f05..33c2d5b6993d 100644 --- a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs @@ -21,6 +21,7 @@ use abi_stable::{export_root_module, prefix_type::PrefixTypeTrait}; use arrow::array::RecordBatch; use arrow::datatypes::{DataType, Field, Schema}; use datafusion::{common::record_batch, datasource::MemTable}; +use datafusion_ffi::function_registry::FFI_WeakFunctionRegistry; use datafusion_ffi::table_provider::FFI_TableProvider; use ffi_module_interface::{TableProviderModule, TableProviderModuleRef}; @@ -34,7 +35,7 @@ fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { /// Here we only wish to create a simple table provider as an example. /// We create an in-memory table and convert it to it's FFI counterpart. -extern "C" fn construct_simple_table_provider() -> FFI_TableProvider { +extern "C" fn construct_simple_table_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Float64, true), @@ -50,7 +51,7 @@ extern "C" fn construct_simple_table_provider() -> FFI_TableProvider { let table_provider = MemTable::try_new(schema, vec![batches]).unwrap(); - FFI_TableProvider::new(Arc::new(table_provider), true, None) + FFI_TableProvider::new(Arc::new(table_provider), true, None, function_registry) } #[export_root_module] diff --git a/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs b/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs index 88690e929713..4b277eb0ed5c 100644 --- a/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs @@ -22,6 +22,7 @@ use abi_stable::{ sabi_types::VersionStrings, StableAbi, }; +use datafusion_ffi::function_registry::FFI_WeakFunctionRegistry; use datafusion_ffi::table_provider::FFI_TableProvider; #[repr(C)] @@ -34,7 +35,7 @@ use datafusion_ffi::table_provider::FFI_TableProvider; /// how a user may wish to separate these concerns. pub struct TableProviderModule { /// Constructs the table provider - pub create_table: extern "C" fn() -> FFI_TableProvider, + pub create_table: extern "C" fn(FFI_WeakFunctionRegistry) -> FFI_TableProvider, } impl RootModule for TableProviderModuleRef { diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index 3ad5f4af7a62..c202f372bca1 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -51,7 +51,11 @@ datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true } datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } +datafusion-functions = { workspace = true, optional = true } +datafusion-functions-aggregate = { workspace = true, optional = true } datafusion-functions-aggregate-common = { workspace = true } +datafusion-functions-table = { workspace = true, optional = true } +datafusion-functions-window = { workspace = true, optional = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } @@ -64,9 +68,9 @@ semver = "1.0.27" tokio = { workspace = true } [dev-dependencies] -datafusion = { workspace = true, default-features = false } +datafusion = { workspace = true, default-features = false, features = ["sql"] } doc-comment = { workspace = true } [features] -integration-tests = [] +integration-tests = ["dep:datafusion-functions"] tarpaulin_include = [] # Exists only to prevent warnings on stable and still have accurate coverage diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index 64c557999ad0..2c399283a09c 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -29,6 +29,7 @@ use crate::{ schema_provider::{FFI_SchemaProvider, ForeignSchemaProvider}, }; +use crate::function_registry::FFI_WeakFunctionRegistry; use datafusion_common::error::Result; /// A stable struct for sharing [`CatalogProvider`] across FFI boundaries. @@ -57,6 +58,8 @@ pub struct FFI_CatalogProvider { cascade: bool, ) -> RResult, RString>, + function_registry: FFI_WeakFunctionRegistry, + /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -105,7 +108,13 @@ unsafe extern "C" fn schema_fn_wrapper( ) -> ROption { let maybe_schema = provider.inner().schema(name.as_str()); maybe_schema - .map(|schema| FFI_SchemaProvider::new(schema, provider.runtime())) + .map(|schema| { + FFI_SchemaProvider::new( + schema, + provider.runtime(), + provider.function_registry.clone(), + ) + }) .into() } @@ -115,13 +124,15 @@ unsafe extern "C" fn register_schema_fn_wrapper( schema: &FFI_SchemaProvider, ) -> RResult, RString> { let runtime = provider.runtime(); - let provider = provider.inner(); let schema = Arc::new(ForeignSchemaProvider::from(schema)); - let returned_schema = - rresult_return!(provider.register_schema(name.as_str(), schema)) - .map(|schema| FFI_SchemaProvider::new(schema, runtime)) - .into(); + let returned_schema = rresult_return!(provider + .inner() + .register_schema(name.as_str(), schema)) + .map(|schema| { + FFI_SchemaProvider::new(schema, runtime, provider.function_registry.clone()) + }) + .into(); RResult::ROk(returned_schema) } @@ -132,14 +143,19 @@ unsafe extern "C" fn deregister_schema_fn_wrapper( cascade: bool, ) -> RResult, RString> { let runtime = provider.runtime(); - let provider = provider.inner(); let maybe_schema = - rresult_return!(provider.deregister_schema(name.as_str(), cascade)); + rresult_return!(provider.inner().deregister_schema(name.as_str(), cascade)); RResult::ROk( maybe_schema - .map(|schema| FFI_SchemaProvider::new(schema, runtime)) + .map(|schema| { + FFI_SchemaProvider::new( + schema, + runtime, + provider.function_registry.clone(), + ) + }) .into(), ) } @@ -165,6 +181,7 @@ unsafe extern "C" fn clone_fn_wrapper( schema: schema_fn_wrapper, register_schema: register_schema_fn_wrapper, deregister_schema: deregister_schema_fn_wrapper, + function_registry: provider.function_registry.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -183,6 +200,7 @@ impl FFI_CatalogProvider { pub fn new( provider: Arc, runtime: Option, + function_registry: FFI_WeakFunctionRegistry, ) -> Self { let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -191,6 +209,7 @@ impl FFI_CatalogProvider { schema: schema_fn_wrapper, register_schema: register_schema_fn_wrapper, deregister_schema: deregister_schema_fn_wrapper, + function_registry, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -254,7 +273,11 @@ impl CatalogProvider for ForeignCatalogProvider { unsafe { let schema = match schema.as_any().downcast_ref::() { Some(s) => &s.0, - None => &FFI_SchemaProvider::new(schema, None), + None => &FFI_SchemaProvider::new( + schema, + None, + self.0.function_registry.clone(), + ), }; let returned_schema: Option = df_result!((self.0.register_schema)(&self.0, name.into(), schema))? @@ -283,9 +306,10 @@ impl CatalogProvider for ForeignCatalogProvider { #[cfg(test)] mod tests { - use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; - use super::*; + use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; + use datafusion::prelude::SessionContext; + use datafusion_expr::registry::FunctionRegistry; #[test] fn test_round_trip_ffi_catalog_provider() { @@ -298,7 +322,11 @@ mod tests { .unwrap() .is_none()); - let ffi_catalog = FFI_CatalogProvider::new(catalog, None); + let ctx = Arc::new(SessionContext::new()); + let function_registry = Arc::clone(&ctx) as Arc; + + let ffi_catalog = + FFI_CatalogProvider::new(catalog, None, function_registry.into()); let foreign_catalog: ForeignCatalogProvider = (&ffi_catalog).into(); diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index 1d72e66355d6..a31e2ea7dbef 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -21,12 +21,10 @@ use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, }; -use datafusion_common::{ - error::{DataFusionError, Result}, -}; +use datafusion_common::error::{DataFusionError, Result}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_plan::{DisplayAs, ExecutionPlan, PlanProperties}; use datafusion_physical_plan::DisplayFormatType; +use datafusion_physical_plan::{DisplayAs, ExecutionPlan, PlanProperties}; use tokio::runtime::Handle; use crate::{ diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs index bef88a7fbac5..ac98d4d09a37 100644 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ b/datafusion/ffi/src/expr/columnar_value.rs @@ -19,8 +19,8 @@ use crate::arrow_wrappers::WrappedArray; use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion_expr::ColumnarValue; use datafusion_common::DataFusionError; +use datafusion_expr::ColumnarValue; #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs index 476819634c95..c17768c590a0 100644 --- a/datafusion/ffi/src/expr/distribution.rs +++ b/datafusion/ffi/src/expr/distribution.rs @@ -19,11 +19,11 @@ use crate::expr::interval::FFI_Interval; use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; +use datafusion_common::DataFusionError; use datafusion_expr::statistics::{ BernoulliDistribution, Distribution, ExponentialDistribution, GaussianDistribution, GenericDistribution, UniformDistribution, }; -use datafusion_common::DataFusionError; #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs index a175e6f70c5d..b7bfabc22c2f 100644 --- a/datafusion/ffi/src/expr/expr_properties.rs +++ b/datafusion/ffi/src/expr/expr_properties.rs @@ -18,8 +18,8 @@ use crate::expr::interval::FFI_Interval; use abi_stable::StableAbi; use arrow_schema::SortOptions; -use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_common::DataFusionError; +use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs index 637982f5bdaa..59f9a7ecc3ed 100644 --- a/datafusion/ffi/src/expr/interval.rs +++ b/datafusion/ffi/src/expr/interval.rs @@ -18,8 +18,8 @@ use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; -use datafusion_expr::interval_arithmetic::Interval; use datafusion_common::DataFusionError; +use datafusion_expr::interval_arithmetic::Interval; #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 19fa4a36665d..850f4e5ecc5e 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -161,9 +161,9 @@ impl Drop for FFI_WeakFunctionRegistry { } } -impl FFI_WeakFunctionRegistry { +impl From> for FFI_WeakFunctionRegistry { /// Creates a new [`FFI_WeakFunctionRegistry`]. - pub fn new(registry: Arc) -> Self { + fn from(registry: Arc) -> Self { let registry = Arc::downgrade(®istry); let private_data = Box::new(RegistryPrivateData { registry }); diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index adfdf5edc3b3..65126f116e8f 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -57,5 +57,11 @@ pub extern "C" fn version() -> u64 { version.major } +static LIBRARY_MARKER: u8 = 0; + +pub extern "C" fn get_library_marker_id() -> u64 { + &LIBRARY_MARKER as *const u8 as u64 +} + #[cfg(doctest)] doc_comment::doctest!("../README.md", readme_example_test); diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index 9c323f7281fc..bb3b6f4eb682 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -135,6 +135,8 @@ pub struct FFI_PhysicalExpr { /// Internal data. This is only to be accessed by the provider of the plan. /// A [`ForeignExecutionPlan`] should never attempt to access this data. pub private_data: *mut c_void, + + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_PhysicalExpr {} @@ -233,8 +235,7 @@ unsafe extern "C" fn new_with_children_fn_wrapper( let expr = Arc::clone(expr.inner()); let children = children .iter() - .map(|e| ForeignPhysicalExpr::from(e.clone())) - .map(|e| Arc::new(e) as Arc) + .map(|e| >::from(e.clone())) .collect::>(); rresult!(expr.with_new_children(children).map(FFI_PhysicalExpr::from)) } @@ -405,6 +406,7 @@ unsafe extern "C" fn clone_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFI_PhysicalEx release: release_fn_wrapper, version: super::version, private_data, + library_marker_id: crate::get_library_marker_id, } } @@ -442,6 +444,7 @@ impl From> for FFI_PhysicalExpr { release: release_fn_wrapper, version: super::version, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -459,18 +462,23 @@ pub struct ForeignPhysicalExpr { unsafe impl Send for ForeignPhysicalExpr {} unsafe impl Sync for ForeignPhysicalExpr {} -impl From for ForeignPhysicalExpr { +impl From for Arc { fn from(expr: FFI_PhysicalExpr) -> Self { + if (expr.library_marker_id)() == crate::get_library_marker_id() { + // We are built in the same library so safe to access inner member + return Arc::clone(expr.inner()); + } + let children = unsafe { (expr.children)(&expr) .into_iter() - .map(|child| { - Arc::new(ForeignPhysicalExpr::from(child)) as Arc - }) + .map(|child| >::from(child)) .collect() }; - Self { expr, children } + let expr = ForeignPhysicalExpr { expr, children }; + + Arc::new(expr) } } @@ -544,7 +552,7 @@ impl PhysicalExpr for ForeignPhysicalExpr { unsafe { let children = children.into_iter().map(FFI_PhysicalExpr::from).collect(); df_result!((self.expr.new_with_children)(&self.expr, &children) - .map(|expr| Arc::new(ForeignPhysicalExpr::from(expr)))) + .map(|expr| >::from(expr))) } } @@ -651,9 +659,7 @@ impl PhysicalExpr for ForeignPhysicalExpr { unsafe { let result = df_result!((self.expr.snapshot)(&self.expr))?; Ok(result - .map(|ffi_expr| { - Arc::new(ForeignPhysicalExpr::from(ffi_expr)) as Arc - }) + .map(|ffi_expr| >::from(ffi_expr)) .into()) } } diff --git a/datafusion/ffi/src/physical_expr/partitioning.rs b/datafusion/ffi/src/physical_expr/partitioning.rs index 617be82267d2..5bc5b1302094 100644 --- a/datafusion/ffi/src/physical_expr/partitioning.rs +++ b/datafusion/ffi/src/physical_expr/partitioning.rs @@ -41,7 +41,7 @@ impl From for Partitioning { let exprs = exprs .into_iter() .map(|expr| { - Arc::new(ForeignPhysicalExpr::from(expr)) as Arc + >::from(expr) }) .collect(); Self::Hash(exprs, size) diff --git a/datafusion/ffi/src/physical_expr/sort.rs b/datafusion/ffi/src/physical_expr/sort.rs index c6ff559b9805..a8e6b0d2109e 100644 --- a/datafusion/ffi/src/physical_expr/sort.rs +++ b/datafusion/ffi/src/physical_expr/sort.rs @@ -20,9 +20,10 @@ use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; use abi_stable::std_types::RVec; use abi_stable::StableAbi; use arrow_schema::SortOptions; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_common::{exec_datafusion_err, DataFusionError}; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; use std::sync::Arc; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; #[repr(C)] #[derive(Debug, StableAbi)] @@ -43,7 +44,7 @@ impl From<&PhysicalSortExpr> for FFI_PhysicalSortExpr { impl From<&FFI_PhysicalSortExpr> for PhysicalSortExpr { fn from(value: &FFI_PhysicalSortExpr) -> Self { - let expr = Arc::new(ForeignPhysicalExpr::from(value.expr.clone())); + let expr = >::from(value.expr.clone()); let options = SortOptions::from(&value.options); Self { expr, options } diff --git a/datafusion/ffi/src/record_batch_stream.rs b/datafusion/ffi/src/record_batch_stream.rs index bb76dec80f02..608e731f2115 100644 --- a/datafusion/ffi/src/record_batch_stream.rs +++ b/datafusion/ffi/src/record_batch_stream.rs @@ -28,8 +28,8 @@ use arrow::{ }; use async_ffi::{ContextExt, FfiContext, FfiPoll}; use datafusion_common::error::{DataFusionError, Result}; -use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; use datafusion_common::{exec_datafusion_err, exec_err}; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; use futures::{Stream, TryStreamExt}; use tokio::runtime::Handle; diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index 5b63d88d08ab..4ea26dd65ad5 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -27,6 +27,7 @@ use datafusion_catalog::{SchemaProvider, TableProvider}; use datafusion_common::error::{DataFusionError, Result}; use tokio::runtime::Handle; +use crate::function_registry::FFI_WeakFunctionRegistry; use crate::{ df_result, rresult_return, table_provider::{FFI_TableProvider, ForeignTableProvider}, @@ -63,6 +64,8 @@ pub struct FFI_SchemaProvider { pub table_exist: unsafe extern "C" fn(provider: &Self, name: RString) -> bool, + pub function_registry: FFI_WeakFunctionRegistry, + /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -111,12 +114,13 @@ unsafe extern "C" fn table_fn_wrapper( provider: &FFI_SchemaProvider, name: RString, ) -> FfiFuture, RString>> { + let function_registry = provider.function_registry.clone(); let runtime = provider.runtime(); let provider = Arc::clone(provider.inner()); async move { let table = rresult_return!(provider.table(name.as_str()).await) - .map(|t| FFI_TableProvider::new(t, true, runtime)) + .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry)) .into(); RResult::ROk(table) @@ -130,12 +134,13 @@ unsafe extern "C" fn register_table_fn_wrapper( table: FFI_TableProvider, ) -> RResult, RString> { let runtime = provider.runtime(); + let function_registry = provider.function_registry.clone(); let provider = provider.inner(); let table = Arc::new(ForeignTableProvider(table)); let returned_table = rresult_return!(provider.register_table(name.into(), table)) - .map(|t| FFI_TableProvider::new(t, true, runtime)); + .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry)); RResult::ROk(returned_table.into()) } @@ -144,11 +149,12 @@ unsafe extern "C" fn deregister_table_fn_wrapper( provider: &FFI_SchemaProvider, name: RString, ) -> RResult, RString> { + let function_registry = provider.function_registry.clone(); let runtime = provider.runtime(); let provider = provider.inner(); let returned_table = rresult_return!(provider.deregister_table(name.as_str())) - .map(|t| FFI_TableProvider::new(t, true, runtime)); + .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry)); RResult::ROk(returned_table.into()) } @@ -187,6 +193,7 @@ unsafe extern "C" fn clone_fn_wrapper( register_table: register_table_fn_wrapper, deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, + function_registry: provider.function_registry.clone(), } } @@ -201,6 +208,7 @@ impl FFI_SchemaProvider { pub fn new( provider: Arc, runtime: Option, + function_registry: FFI_WeakFunctionRegistry, ) -> Self { let owner_name = provider.owner_name().map(|s| s.into()).into(); let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -216,6 +224,7 @@ impl FFI_SchemaProvider { register_table: register_table_fn_wrapper, deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, + function_registry, } } } @@ -286,7 +295,12 @@ impl SchemaProvider for ForeignSchemaProvider { unsafe { let ffi_table = match table.as_any().downcast_ref::() { Some(t) => t.0.clone(), - None => FFI_TableProvider::new(table, true, None), + None => FFI_TableProvider::new( + table, + true, + None, + self.0.function_registry.clone(), + ), }; let returned_provider: Option = @@ -315,10 +329,11 @@ impl SchemaProvider for ForeignSchemaProvider { #[cfg(test)] mod tests { + use super::*; use arrow::datatypes::Schema; + use datafusion::prelude::SessionContext; use datafusion::{catalog::MemorySchemaProvider, datasource::empty::EmptyTable}; - - use super::*; + use datafusion_expr::registry::FunctionRegistry; fn empty_table() -> Arc { Arc::new(EmptyTable::new(Arc::new(Schema::empty()))) @@ -332,8 +347,11 @@ mod tests { .register_table("prior_table".to_string(), empty_table()) .unwrap() .is_none()); + let ctx = Arc::new(SessionContext::new()); + let function_registry = Arc::clone(&ctx) as Arc; - let ffi_schema_provider = FFI_SchemaProvider::new(schema_provider, None); + let ffi_schema_provider = + FFI_SchemaProvider::new(schema_provider, None, function_registry.into()); let foreign_schema_provider: ForeignSchemaProvider = (&ffi_schema_provider).into(); diff --git a/datafusion/ffi/src/session/config.rs b/datafusion/ffi/src/session/config.rs index 78675eebb35c..bf46a1e77a6c 100644 --- a/datafusion/ffi/src/session/config.rs +++ b/datafusion/ffi/src/session/config.rs @@ -19,13 +19,16 @@ use abi_stable::{ std_types::{RHashMap, RString}, StableAbi, }; -use datafusion_common::{config::ConfigOptions, error::{DataFusionError, Result}}; +use datafusion_common::{ + config::ConfigOptions, + error::{DataFusionError, Result}, +}; +use datafusion_execution::config::SessionConfig; use std::sync::Arc; use std::{ collections::HashMap, ffi::{c_char, c_void, CString}, }; -use datafusion_execution::config::SessionConfig; /// A stable struct for sharing [`SessionConfig`] across FFI boundaries. /// Instead of attempting to expose the entire SessionConfig interface, we diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index ac198f78bd4b..f58375c9a898 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -17,6 +17,7 @@ use crate::arrow_wrappers::WrappedSchema; use crate::execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}; +use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; use crate::session::config::{FFI_SessionConfig, ForeignSessionConfig}; use crate::session::task::FFI_TaskContext; use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; @@ -32,17 +33,17 @@ use arrow_schema::ffi::FFI_ArrowSchema; use arrow_schema::SchemaRef; use async_ffi::{FfiFuture, FutureExt}; use async_trait::async_trait; -use datafusion::catalog::Session; -use datafusion::execution::runtime_env::RuntimeEnv; -use datafusion::execution::TaskContext; -use datafusion::physical_expr::PhysicalExpr; -use datafusion::physical_plan::ExecutionPlan; -use datafusion::prelude::SessionConfig; +use datafusion_catalog::Session; use datafusion_common::config::{ConfigOptions, TableOptions}; use datafusion_common::{DFSchema, DataFusionError}; +use datafusion_execution::config::SessionConfig; +use datafusion_execution::runtime_env::RuntimeEnv; +use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::ExecutionPlan; use datafusion_proto::bytes::{logical_plan_from_bytes, logical_plan_to_bytes}; use datafusion_proto::logical_plan::from_proto::parse_expr; use datafusion_proto::logical_plan::to_proto::serialize_expr; @@ -98,6 +99,8 @@ pub struct FFI_Session { pub task_ctx: unsafe extern "C" fn(&Self) -> FFI_TaskContext, + pub function_registry: FFI_WeakFunctionRegistry, + /// Used to create a clone on the provider of the registry. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -116,24 +119,23 @@ pub struct FFI_Session { unsafe impl Send for FFI_Session {} unsafe impl Sync for FFI_Session {} -struct SessionPrivateData { - session: Arc, - function_registry: Arc, - runtime: Handle, +struct SessionPrivateData<'a> { + session: &'a (dyn Session + Send), + runtime: Option, } impl FFI_Session { - unsafe fn inner(&self) -> &Arc { + unsafe fn inner(&self) -> &(dyn Session + Send) { let private_data = self.private_data as *const SessionPrivateData; - &(*private_data).session + (*private_data).session } - unsafe fn function_registry(&self) -> &Arc { - let private_data = self.private_data as *const SessionPrivateData; - &(*private_data).function_registry + unsafe fn function_registry(&self) -> Arc { + let registry = ForeignWeakFunctionRegistry::from(&self.function_registry); + Arc::new(registry) as Arc } - unsafe fn runtime(&self) -> &Handle { + unsafe fn runtime(&self) -> &Option { let private_data = self.private_data as *const SessionPrivateData; &(*private_data).runtime } @@ -154,8 +156,9 @@ unsafe extern "C" fn create_physical_plan_fn_wrapper( logical_plan_serialized: RVec, ) -> FfiFuture> { let runtime = session.runtime().clone(); - let session = Arc::clone(session.inner()); + let session = session.clone(); async move { + let session = session.inner(); let task_ctx = session.task_ctx(); let logical_plan = rresult_return!(logical_plan_from_bytes( @@ -168,7 +171,7 @@ unsafe extern "C" fn create_physical_plan_fn_wrapper( rresult!(physical_plan.map(|plan| FFI_ExecutionPlan::new( plan, task_ctx, - Some(runtime) + runtime ))) } .into_ffi() @@ -187,7 +190,7 @@ unsafe extern "C" fn create_physical_expr_fn_wrapper( let logical_expr = parse_expr(&logical_expr, function_registry.as_ref(), &codec).unwrap(); let schema: SchemaRef = schema.into(); - let schema: DFSchema = rresult_return!((schema).try_into()); + let schema: DFSchema = rresult_return!(schema.try_into()); let physical_expr = rresult_return!(session.create_physical_expr(logical_expr, &schema)); @@ -275,8 +278,7 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_Session) -> FFI_Session { let old_private_data = provider.private_data as *const SessionPrivateData; let private_data = Box::into_raw(Box::new(SessionPrivateData { - session: Arc::clone(&(*old_private_data).session), - function_registry: Arc::clone(&(*old_private_data).function_registry), + session: (*old_private_data).session, runtime: (*old_private_data).runtime.clone(), })) as *mut c_void; @@ -291,6 +293,7 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_Session) -> FFI_Session { table_options: table_options_fn_wrapper, default_table_options: default_table_options_fn_wrapper, task_ctx: task_ctx_fn_wrapper, + function_registry: provider.function_registry.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, @@ -308,15 +311,11 @@ impl Drop for FFI_Session { impl FFI_Session { /// Creates a new [`FFI_Session`]. pub fn new( - session: Arc, - function_registry: Arc, - runtime: Handle, + session: &(dyn Session + Send), + function_registry: FFI_WeakFunctionRegistry, + runtime: Option, ) -> Self { - let private_data = Box::new(SessionPrivateData { - session, - function_registry, - runtime, - }); + let private_data = Box::new(SessionPrivateData { session, runtime }); Self { session_id: session_id_fn_wrapper, @@ -329,6 +328,7 @@ impl FFI_Session { table_options: table_options_fn_wrapper, default_table_options: default_table_options_fn_wrapper, task_ctx: task_ctx_fn_wrapper, + function_registry, clone: clone_fn_wrapper, release: release_fn_wrapper, diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index 521514510f15..c2d418677eb3 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -22,8 +22,8 @@ use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use abi_stable::pmr::ROption; use abi_stable::std_types::RHashMap; use abi_stable::{std_types::RString, StableAbi}; -use datafusion::execution::runtime_env::RuntimeEnv; -use datafusion::execution::TaskContext; +use datafusion_execution::runtime_env::RuntimeEnv; +use datafusion_execution::TaskContext; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use std::{ffi::c_void, sync::Arc}; diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 878b67c977a2..9027c1f18162 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -35,22 +35,20 @@ use datafusion_proto::{ use prost::Message; use tokio::runtime::Handle; +use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; +use crate::session::{FFI_Session, ForeignSession}; use crate::{ arrow_wrappers::WrappedSchema, df_result, execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}, insert_op::FFI_InsertOp, rresult_return, - session::config::{FFI_SessionConfig, ForeignSessionConfig}, table_source::{FFI_TableProviderFilterPushDown, FFI_TableType}, }; -use datafusion::error::Result; -use datafusion_common::DataFusionError; -use datafusion_execution::TaskContext; +use datafusion_common::{DataFusionError, Result}; use datafusion_expr::dml::InsertOp; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType}; use datafusion_physical_plan::ExecutionPlan; -use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; /// A stable struct for sharing [`TableProvider`] across FFI boundaries. /// @@ -111,7 +109,7 @@ pub struct FFI_TableProvider { /// * `limit` - if specified, limit the number of rows returned pub scan: unsafe extern "C" fn( provider: &Self, - session_config: &FFI_SessionConfig, + session_config: &FFI_Session, projections: RVec, filters_serialized: RVec, limit: ROption, @@ -134,7 +132,7 @@ pub struct FFI_TableProvider { pub insert_into: unsafe extern "C" fn( provider: &Self, - session_config: &FFI_SessionConfig, + session_config: &FFI_Session, input: &FFI_ExecutionPlan, insert_op: FFI_InsertOp, ) -> FfiFuture>, @@ -211,7 +209,8 @@ unsafe extern "C" fn supports_filters_pushdown_fn_wrapper( provider: &FFI_TableProvider, filters_serialized: RVec, ) -> RResult, RString> { - let function_registry = rresult_return!(ForeignWeakFunctionRegistry::try_from(&provider.function_registry)); + let function_registry = + ForeignWeakFunctionRegistry::from(&provider.function_registry); let private_data = provider.private_data as *const ProviderPrivateData; let provider = &(*private_data).provider; @@ -227,15 +226,16 @@ unsafe extern "C" fn scan_fn_wrapper( filters_serialized: RVec, limit: ROption, ) -> FfiFuture> { - let function_registry = ForeignWeakFunctionRegistry::try_from(&provider.function_registry); + let function_registry = + ForeignWeakFunctionRegistry::from(&provider.function_registry); + let session = ForeignSession::try_from(session); let private_data = provider.private_data as *mut ProviderPrivateData; let internal_provider = &(*private_data).provider; - let session_config = session_config.clone(); let runtime = &(*private_data).runtime; async move { - let function_registry = rresult_return!(function_registry); - let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); + let session = rresult_return!(session); + // let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); // let session = SessionStateBuilder::new() // .with_default_features() // .with_config(config.0) @@ -263,13 +263,13 @@ unsafe extern "C" fn scan_fn_wrapper( let plan = rresult_return!( internal_provider - .scan(&ctx.state(), Some(&projections), &filters, limit.into()) + .scan(&session, Some(&projections), &filters, limit.into()) .await ); RResult::ROk(FFI_ExecutionPlan::new( plan, - ctx.task_ctx(), + session.task_ctx(), runtime.clone(), )) } @@ -278,23 +278,25 @@ unsafe extern "C" fn scan_fn_wrapper( unsafe extern "C" fn insert_into_fn_wrapper( provider: &FFI_TableProvider, - session_config: &FFI_SessionConfig, + session: &FFI_Session, input: &FFI_ExecutionPlan, insert_op: FFI_InsertOp, ) -> FfiFuture> { let private_data = provider.private_data as *mut ProviderPrivateData; let internal_provider = &(*private_data).provider; - let session_config = session_config.clone(); + // let session_config = session_config.clone(); + let session = ForeignSession::try_from(session); let input = input.clone(); let runtime = &(*private_data).runtime; async move { - let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); - let session = SessionStateBuilder::new() - .with_default_features() - .with_config(config.0) - .build(); - let ctx = SessionContext::new_with_state(session); + let session = rresult_return!(session); + // let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); + // let session = SessionStateBuilder::new() + // .with_default_features() + // .with_config(config.0) + // .build(); + // let ctx = SessionContext::new_with_state(session); let input = rresult_return!(ForeignExecutionPlan::try_from(&input).map(Arc::new)); @@ -302,13 +304,13 @@ unsafe extern "C" fn insert_into_fn_wrapper( let plan = rresult_return!( internal_provider - .insert_into(&ctx.state(), input, insert_op) + .insert_into(&session, input, insert_op) .await ); RResult::ROk(FFI_ExecutionPlan::new( plan, - ctx.task_ctx(), + session.task_ctx(), runtime.clone(), )) } @@ -335,6 +337,7 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_TableProvider) -> FFI_Table table_type: table_type_fn_wrapper, supports_filters_pushdown: provider.supports_filters_pushdown, insert_into: provider.insert_into, + function_registry: provider.function_registry.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -354,6 +357,7 @@ impl FFI_TableProvider { provider: Arc, can_support_pushdown_filters: bool, runtime: Option, + function_registry: FFI_WeakFunctionRegistry, ) -> Self { let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -366,6 +370,7 @@ impl FFI_TableProvider { false => None, }, insert_into: insert_into_fn_wrapper, + function_registry, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -418,7 +423,8 @@ impl TableProvider for ForeignTableProvider { filters: &[Expr], limit: Option, ) -> Result> { - let session_config: FFI_SessionConfig = session.config().into(); + // let session_config: FFI_SessionConfig = session.config().into(); + let session = FFI_Session::new(session, self.0.function_registry.clone(), None); let projections: Option> = projection.map(|p| p.iter().map(|v| v.to_owned()).collect()); @@ -432,7 +438,7 @@ impl TableProvider for ForeignTableProvider { let plan = unsafe { let maybe_plan = (self.0.scan)( &self.0, - &session_config, + &session, projections.unwrap_or_default(), filters_serialized, limit.into(), @@ -481,16 +487,18 @@ impl TableProvider for ForeignTableProvider { input: Arc, insert_op: InsertOp, ) -> Result> { - let session_config: FFI_SessionConfig = session.config().into(); - + // let session_config: FFI_SessionConfig = session.config().into(); + let task_ctx = session.task_ctx(); let rc = Handle::try_current().ok(); - let input = - FFI_ExecutionPlan::new(input, Arc::new(TaskContext::from(session)), rc); + let session = + FFI_Session::new(session, self.0.function_registry.clone(), rc.clone()); + + let input = FFI_ExecutionPlan::new(input, task_ctx, rc); let insert_op: FFI_InsertOp = insert_op.into(); let plan = unsafe { let maybe_plan = - (self.0.insert_into)(&self.0, &session_config, &input, insert_op).await; + (self.0.insert_into)(&self.0, &session, &input, insert_op).await; ForeignExecutionPlan::try_from(&df_result!(maybe_plan)?)? }; @@ -501,10 +509,10 @@ impl TableProvider for ForeignTableProvider { #[cfg(test)] mod tests { - use arrow::datatypes::Schema; - use datafusion::prelude::{col, lit}; - use super::*; + use arrow::datatypes::Schema; + use datafusion::prelude::{col, lit, SessionContext}; + use datafusion_expr::registry::FunctionRegistry; #[tokio::test] async fn test_round_trip_ffi_table_provider_scan() -> Result<()> { @@ -527,12 +535,14 @@ mod tests { vec![Arc::new(Float32Array::from(vec![64.0]))], )?; - let ctx = SessionContext::new(); + let ctx = Arc::new(SessionContext::new()); + let function_registry = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); - let ffi_provider = FFI_TableProvider::new(provider, true, None); + let ffi_provider = + FFI_TableProvider::new(provider, true, None, function_registry.into()); let foreign_table_provider: ForeignTableProvider = (&ffi_provider).into(); @@ -569,12 +579,14 @@ mod tests { vec![Arc::new(Float32Array::from(vec![64.0]))], )?; - let ctx = SessionContext::new(); + let ctx = Arc::new(SessionContext::new()); + let function_registry = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); - let ffi_provider = FFI_TableProvider::new(provider, true, None); + let ffi_provider = + FFI_TableProvider::new(provider, true, None, function_registry.into()); let foreign_table_provider: ForeignTableProvider = (&ffi_provider).into(); @@ -616,11 +628,13 @@ mod tests { vec![Arc::new(Float32Array::from(vec![2.0, 4.0, 8.0]))], )?; - let ctx = SessionContext::new(); + let ctx = Arc::new(SessionContext::new()); + let function_registry = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1]])?); - let ffi_provider = FFI_TableProvider::new(provider, true, None); + let ffi_provider = + FFI_TableProvider::new(provider, true, None, function_registry.into()); let foreign_table_provider: ForeignTableProvider = (&ffi_provider).into(); diff --git a/datafusion/ffi/src/tests/async_provider.rs b/datafusion/ffi/src/tests/async_provider.rs index cef4161d8c1f..fe5551f23999 100644 --- a/datafusion/ffi/src/tests/async_provider.rs +++ b/datafusion/ffi/src/tests/async_provider.rs @@ -31,21 +31,18 @@ use crate::table_provider::FFI_TableProvider; use arrow::array::RecordBatch; use arrow::datatypes::Schema; use async_trait::async_trait; -use datafusion::{ - catalog::{Session, TableProvider}, - error::Result, - execution::RecordBatchStream, - physical_expr::EquivalenceProperties, - physical_plan::{ExecutionPlan, Partitioning}, - prelude::Expr, -}; -use datafusion_common::exec_err; +use datafusion_common::{exec_err, error::Result}; use futures::Stream; use tokio::{ runtime::Handle, sync::{broadcast, mpsc}, }; - +use datafusion_catalog::{Session, TableProvider}; +use datafusion_execution::RecordBatchStream; +use datafusion_expr::Expr; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_plan::ExecutionPlan; +use crate::function_registry::FFI_WeakFunctionRegistry; use super::create_record_batch; #[derive(Debug)] @@ -135,8 +132,8 @@ impl TableProvider for AsyncTableProvider { super::create_test_schema() } - fn table_type(&self) -> datafusion::logical_expr::TableType { - datafusion::logical_expr::TableType::Base + fn table_type(&self) -> datafusion_expr::TableType { + datafusion_expr::TableType::Base } async fn scan( @@ -163,7 +160,7 @@ impl Drop for AsyncTableProvider { #[derive(Debug)] struct AsyncTestExecutionPlan { - properties: datafusion::physical_plan::PlanProperties, + properties: datafusion_physical_plan::PlanProperties, batch_request: mpsc::Sender, batch_receiver: broadcast::Receiver>, } @@ -174,11 +171,11 @@ impl AsyncTestExecutionPlan { batch_receiver: broadcast::Receiver>, ) -> Self { Self { - properties: datafusion::physical_plan::PlanProperties::new( + properties: datafusion_physical_plan::PlanProperties::new( EquivalenceProperties::new(super::create_test_schema()), Partitioning::UnknownPartitioning(3), - datafusion::physical_plan::execution_plan::EmissionType::Incremental, - datafusion::physical_plan::execution_plan::Boundedness::Bounded, + datafusion_physical_plan::execution_plan::EmissionType::Incremental, + datafusion_physical_plan::execution_plan::Boundedness::Bounded, ), batch_request, batch_receiver, @@ -195,7 +192,7 @@ impl ExecutionPlan for AsyncTestExecutionPlan { self } - fn properties(&self) -> &datafusion::physical_plan::PlanProperties { + fn properties(&self) -> &datafusion_physical_plan::PlanProperties { &self.properties } @@ -213,8 +210,8 @@ impl ExecutionPlan for AsyncTestExecutionPlan { fn execute( &self, _partition: usize, - _context: Arc, - ) -> Result { + _context: Arc, + ) -> Result { Ok(Box::pin(AsyncTestRecordBatchStream { batch_request: self.batch_request.clone(), batch_receiver: self.batch_receiver.resubscribe(), @@ -222,10 +219,10 @@ impl ExecutionPlan for AsyncTestExecutionPlan { } } -impl datafusion::physical_plan::DisplayAs for AsyncTestExecutionPlan { +impl datafusion_physical_plan::DisplayAs for AsyncTestExecutionPlan { fn fmt_as( &self, - _t: datafusion::physical_plan::DisplayFormatType, + _t: datafusion_physical_plan::DisplayFormatType, _f: &mut std::fmt::Formatter, ) -> std::fmt::Result { // Do nothing, just a test @@ -277,7 +274,7 @@ impl Stream for AsyncTestRecordBatchStream { } } -pub(crate) fn create_async_table_provider() -> FFI_TableProvider { +pub(crate) fn create_async_table_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { let (table_provider, tokio_rt) = start_async_provider(); - FFI_TableProvider::new(Arc::new(table_provider), true, Some(tokio_rt)) + FFI_TableProvider::new(Arc::new(table_provider), true, Some(tokio_rt), function_registry) } diff --git a/datafusion/ffi/src/tests/catalog.rs b/datafusion/ffi/src/tests/catalog.rs index f4293adb41b9..69774807dd9d 100644 --- a/datafusion/ffi/src/tests/catalog.rs +++ b/datafusion/ffi/src/tests/catalog.rs @@ -30,15 +30,11 @@ use std::{any::Any, fmt::Debug, sync::Arc}; use crate::catalog_provider::FFI_CatalogProvider; use arrow::datatypes::Schema; use async_trait::async_trait; -use datafusion::{ - catalog::{ - CatalogProvider, MemoryCatalogProvider, MemorySchemaProvider, SchemaProvider, - TableProvider, - }, - common::exec_err, - datasource::MemTable, - error::{DataFusionError, Result}, -}; +use datafusion_catalog::{CatalogProvider, MemTable, MemoryCatalogProvider, MemorySchemaProvider, SchemaProvider, TableProvider}; +use datafusion_common::error::{DataFusionError, Result} +; +use datafusion_common::exec_err; +use crate::function_registry::FFI_WeakFunctionRegistry; /// This schema provider is intended only for unit tests. It prepopulates with one /// table and only allows for tables named sales and purchases. @@ -49,7 +45,7 @@ pub struct FixedSchemaProvider { pub fn fruit_table() -> Arc { use arrow::datatypes::{DataType, Field}; - use datafusion::common::record_batch; + use datafusion_common::record_batch; let schema = Arc::new(Schema::new(vec![ Field::new("units", DataType::Int32, true), @@ -177,7 +173,7 @@ impl CatalogProvider for FixedCatalogProvider { } } -pub(crate) extern "C" fn create_catalog_provider() -> FFI_CatalogProvider { +pub(crate) extern "C" fn create_catalog_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_CatalogProvider { let catalog_provider = Arc::new(FixedCatalogProvider::default()); - FFI_CatalogProvider::new(catalog_provider, None) + FFI_CatalogProvider::new(catalog_provider, None, function_registry) } diff --git a/datafusion/ffi/src/tests/mod.rs b/datafusion/ffi/src/tests/mod.rs index 816086c32041..9ed98387317e 100644 --- a/datafusion/ffi/src/tests/mod.rs +++ b/datafusion/ffi/src/tests/mod.rs @@ -36,15 +36,14 @@ use crate::udwf::FFI_WindowUDF; use super::{table_provider::FFI_TableProvider, udf::FFI_ScalarUDF}; use arrow::array::RecordBatch; use async_provider::create_async_table_provider; -use datafusion::{ - arrow::datatypes::{DataType, Field, Schema}, - common::record_batch, -}; +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion_common::record_batch; use sync_provider::create_sync_table_provider; use udf_udaf_udwf::{ create_ffi_abs_func, create_ffi_random_func, create_ffi_rank_func, create_ffi_stddev_func, create_ffi_sum_func, create_ffi_table_func, }; +use crate::function_registry::FFI_WeakFunctionRegistry; mod async_provider; pub mod catalog; @@ -60,17 +59,17 @@ pub mod utils; /// module. pub struct ForeignLibraryModule { /// Construct an opinionated catalog provider - pub create_catalog: extern "C" fn() -> FFI_CatalogProvider, + pub create_catalog: extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_CatalogProvider, /// Constructs the table provider - pub create_table: extern "C" fn(synchronous: bool) -> FFI_TableProvider, + pub create_table: extern "C" fn(synchronous: bool, function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider, /// Create a scalar UDF pub create_scalar_udf: extern "C" fn() -> FFI_ScalarUDF, pub create_nullary_udf: extern "C" fn() -> FFI_ScalarUDF, - pub create_table_function: extern "C" fn() -> FFI_TableFunction, + pub create_table_function: extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableFunction, /// Create an aggregate UDAF using sum pub create_sum_udaf: extern "C" fn() -> FFI_AggregateUDF, @@ -111,10 +110,10 @@ pub fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { /// Here we only wish to create a simple table provider as an example. /// We create an in-memory table and convert it to it's FFI counterpart. -extern "C" fn construct_table_provider(synchronous: bool) -> FFI_TableProvider { +extern "C" fn construct_table_provider(synchronous: bool, function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { match synchronous { - true => create_sync_table_provider(), - false => create_async_table_provider(), + true => create_sync_table_provider(function_registry), + false => create_async_table_provider(function_registry), } } diff --git a/datafusion/ffi/src/tests/sync_provider.rs b/datafusion/ffi/src/tests/sync_provider.rs index ff85e0b15b39..f5981640f451 100644 --- a/datafusion/ffi/src/tests/sync_provider.rs +++ b/datafusion/ffi/src/tests/sync_provider.rs @@ -16,13 +16,12 @@ // under the License. use std::sync::Arc; - +use datafusion_catalog::MemTable; use crate::table_provider::FFI_TableProvider; -use datafusion::datasource::MemTable; - +use crate::function_registry::FFI_WeakFunctionRegistry; use super::{create_record_batch, create_test_schema}; -pub(crate) fn create_sync_table_provider() -> FFI_TableProvider { +pub(crate) fn create_sync_table_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { let schema = create_test_schema(); // It is useful to create these as multiple record batches @@ -35,5 +34,5 @@ pub(crate) fn create_sync_table_provider() -> FFI_TableProvider { let table_provider = MemTable::try_new(schema, vec![batches]).unwrap(); - FFI_TableProvider::new(Arc::new(table_provider), true, None) + FFI_TableProvider::new(Arc::new(table_provider), true, None, function_registry) } diff --git a/datafusion/ffi/src/tests/udf_udaf_udwf.rs b/datafusion/ffi/src/tests/udf_udaf_udwf.rs index 55e31ef3ab77..3f43bf0e0266 100644 --- a/datafusion/ffi/src/tests/udf_udaf_udwf.rs +++ b/datafusion/ffi/src/tests/udf_udaf_udwf.rs @@ -19,16 +19,18 @@ use crate::{ udaf::FFI_AggregateUDF, udf::FFI_ScalarUDF, udtf::FFI_TableFunction, udwf::FFI_WindowUDF, }; -use datafusion::{ - catalog::TableFunctionImpl, - functions::math::{abs::AbsFunc, random::RandomFunc}, - functions_aggregate::{stddev::Stddev, sum::Sum}, - functions_table::generate_series::RangeFunc, - functions_window::rank::Rank, - logical_expr::{AggregateUDF, ScalarUDF, WindowUDF}, -}; + use std::sync::Arc; +use datafusion_functions::math::abs::AbsFunc; +use datafusion_functions_aggregate::stddev::Stddev; +use datafusion_functions_aggregate::sum::Sum; +use datafusion_functions_table::generate_series::RangeFunc; +use datafusion_functions_window::rank::Rank; +use datafusion_catalog::TableFunctionImpl; +use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use datafusion_functions::math::random::RandomFunc; +use crate::function_registry::FFI_WeakFunctionRegistry; pub(crate) extern "C" fn create_ffi_abs_func() -> FFI_ScalarUDF { let udf: Arc = Arc::new(AbsFunc::new().into()); @@ -42,10 +44,10 @@ pub(crate) extern "C" fn create_ffi_random_func() -> FFI_ScalarUDF { udf.into() } -pub(crate) extern "C" fn create_ffi_table_func() -> FFI_TableFunction { +pub(crate) extern "C" fn create_ffi_table_func(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableFunction { let udtf: Arc = Arc::new(RangeFunc {}); - FFI_TableFunction::new(udtf, None) + FFI_TableFunction::new(udtf, None, function_registry) } pub(crate) extern "C" fn create_ffi_sum_func() -> FFI_AggregateUDF { @@ -64,7 +66,7 @@ pub(crate) extern "C" fn create_ffi_rank_func() -> FFI_WindowUDF { let udwf: Arc = Arc::new( Rank::new( "rank_demo".to_string(), - datafusion::functions_window::rank::RankType::Basic, + datafusion_functions_window::rank::RankType::Basic, ) .into(), ); diff --git a/datafusion/ffi/src/tests/utils.rs b/datafusion/ffi/src/tests/utils.rs index 6465b17d9b60..4df0334c21fe 100644 --- a/datafusion/ffi/src/tests/utils.rs +++ b/datafusion/ffi/src/tests/utils.rs @@ -17,7 +17,7 @@ use crate::tests::ForeignLibraryModuleRef; use abi_stable::library::RootModule; -use datafusion::error::{DataFusionError, Result}; +use datafusion_common::error::{DataFusionError, Result}; use std::path::Path; /// Compute the path to the library. It would be preferable to simply use diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 1cead1ea1da7..9636c3004654 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -26,9 +26,7 @@ use abi_stable::{ }; use arrow::{datatypes::Schema, ffi::FFI_ArrowSchema}; use arrow_schema::FieldRef; -use datafusion_common::{ - error::DataFusionError, -}; +use datafusion_common::error::DataFusionError; use datafusion_expr::function::AccumulatorArgs; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -107,7 +105,7 @@ impl TryFrom for ForeignAccumulatorArgs { .exprs .into_iter() .map(|expr| { - Arc::new(ForeignPhysicalExpr::from(expr)) as Arc + >::from(expr) }) .collect(); diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 914ae9858c1b..266468cebd46 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -30,9 +30,7 @@ use arrow::{ error::ArrowError, ffi::to_ffi, }; -use datafusion_common::{ - error::{DataFusionError, Result}, -}; +use datafusion_common::error::{DataFusionError, Result}; use datafusion_expr::{EmitTo, GroupsAccumulator}; /// A stable struct for sharing [`GroupsAccumulator`] across FFI boundaries. diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index b81bda4dda08..b7c944313ff8 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -22,8 +22,13 @@ use abi_stable::{ StableAbi, }; -use datafusion_common::error::Result; +use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; +use crate::{ + df_result, rresult_return, + table_provider::{FFI_TableProvider, ForeignTableProvider}, +}; use datafusion_catalog::{TableFunctionImpl, TableProvider}; +use datafusion_common::error::Result; use datafusion_expr::Expr; use datafusion_proto::{ logical_plan::{ @@ -33,11 +38,6 @@ use datafusion_proto::{ }; use prost::Message; use tokio::runtime::Handle; -use crate::{ - df_result, rresult_return, - table_provider::{FFI_TableProvider, ForeignTableProvider}, -}; -use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; /// A stable struct for sharing a [`TableFunctionImpl`] across FFI boundaries. #[repr(C)] @@ -89,7 +89,8 @@ unsafe extern "C" fn call_fn_wrapper( udtf: &FFI_TableFunction, args: RVec, ) -> RResult { - let function_registry = ForeignWeakFunctionRegistry::try_from(&udtf.function_registry)?; + let function_registry = udtf.function_registry.clone(); + let foreign_registry = ForeignWeakFunctionRegistry::from(&function_registry); let runtime = udtf.runtime(); let udtf = udtf.inner(); @@ -98,11 +99,19 @@ unsafe extern "C" fn call_fn_wrapper( let proto_filters = rresult_return!(LogicalExprList::decode(args.as_ref())); - let args = - rresult_return!(parse_exprs(proto_filters.expr.iter(), &function_registry, &codec)); + let args = rresult_return!(parse_exprs( + proto_filters.expr.iter(), + &foreign_registry, + &codec + )); let table_provider = rresult_return!(udtf.call(&args)); - RResult::ROk(FFI_TableProvider::new(table_provider, false, runtime)) + RResult::ROk(FFI_TableProvider::new( + table_provider, + false, + runtime, + function_registry, + )) } unsafe extern "C" fn release_fn_wrapper(udtf: &mut FFI_TableFunction) { @@ -125,13 +134,16 @@ impl Clone for FFI_TableFunction { } impl FFI_TableFunction { - pub fn new(udtf: Arc, runtime: Option, function_registry: impl Into) -> Self { - let function_registry = function_registry.into(); + pub fn new( + udtf: Arc, + runtime: Option, + function_registry: FFI_WeakFunctionRegistry, + ) -> Self { let private_data = Box::new(TableFunctionPrivateData { udtf, runtime }); Self { call: call_fn_wrapper, - function_registry: function_registry.into(), + function_registry, clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, @@ -198,17 +210,18 @@ impl TableFunctionImpl for ForeignTableFunction { #[cfg(test)] mod tests { + use super::*; use arrow::{ array::{ record_batch, ArrayRef, Float64Array, RecordBatch, StringArray, UInt64Array, }, datatypes::{DataType, Field, Schema}, }; + use datafusion::prelude::SessionContext; use datafusion::{ catalog::MemTable, common::exec_err, prelude::lit, scalar::ScalarValue, }; - use datafusion::prelude::SessionContext; - use super::*; + use datafusion_expr::registry::FunctionRegistry; #[derive(Debug)] struct TestUDTF {} @@ -293,8 +306,12 @@ mod tests { let original_udtf = Arc::new(TestUDTF {}) as Arc; let ctx = Arc::new(SessionContext::default()); - let local_udtf: FFI_TableFunction = - FFI_TableFunction::new(Arc::clone(&original_udtf), None, Arc::clone(&ctx))?; + let function_registry = Arc::clone(&ctx) as Arc; + let local_udtf: FFI_TableFunction = FFI_TableFunction::new( + Arc::clone(&original_udtf), + None, + function_registry.into(), + ); let foreign_udf: ForeignTableFunction = local_udtf.into(); diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index 16ae5c955e9a..af284d80a87b 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -26,9 +26,7 @@ use arrow::{ ffi::FFI_ArrowSchema, }; use arrow_schema::FieldRef; -use datafusion_common::{ - error::{DataFusionError, Result}, -}; +use datafusion_common::error::{DataFusionError, Result}; use datafusion_expr::function::PartitionEvaluatorArgs; use datafusion_physical_plan::{expressions::Column, PhysicalExpr}; @@ -134,7 +132,7 @@ impl TryFrom for ForeignPartitionEvaluatorArgs { .input_exprs .into_iter() .map(|expr| { - Arc::new(ForeignPhysicalExpr::from(expr)) as Arc + >::from(expr) }) .collect::>(); diff --git a/datafusion/ffi/tests/ffi_integration.rs b/datafusion/ffi/tests/ffi_integration.rs index eb53e76bfb9b..5176a9acc6f7 100644 --- a/datafusion/ffi/tests/ffi_integration.rs +++ b/datafusion/ffi/tests/ffi_integration.rs @@ -26,6 +26,7 @@ mod tests { use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; use std::sync::Arc; + use datafusion_expr::registry::FunctionRegistry; /// It is important that this test is in the `tests` directory and not in the /// library directory so we can verify we are building a dynamic library and @@ -33,20 +34,21 @@ mod tests { async fn test_table_provider(synchronous: bool) -> Result<()> { let table_provider_module = get_module()?; + let ctx = Arc::new(SessionContext::new()); + let function_registry = Arc::clone(&ctx) as Arc; + // By calling the code below, the table provided will be created within // the module's code. let ffi_table_provider = table_provider_module.create_table().ok_or( DataFusionError::NotImplemented( "External table provider failed to implement create_table".to_string(), ), - )?(synchronous); + )?(synchronous, function_registry.into()); // In order to access the table provider within this executable, we need to // turn it into a `ForeignTableProvider`. let foreign_table_provider: ForeignTableProvider = (&ffi_table_provider).into(); - let ctx = SessionContext::new(); - // Display the data to show the full cycle works. ctx.register_table("external_table", Arc::new(foreign_table_provider))?; let df = ctx.table("external_table").await?; @@ -73,6 +75,8 @@ mod tests { #[tokio::test] async fn test_catalog() -> Result<()> { let module = get_module()?; + let ctx = Arc::new(SessionContext::default()); + let function_registry = Arc::clone(&ctx) as Arc; let ffi_catalog = module @@ -80,10 +84,9 @@ mod tests { .ok_or(DataFusionError::NotImplemented( "External catalog provider failed to implement create_catalog" .to_string(), - ))?(); + ))?(function_registry.into()); let foreign_catalog: ForeignCatalogProvider = (&ffi_catalog).into(); - let ctx = SessionContext::default(); let _ = ctx.register_catalog("fruit", Arc::new(foreign_catalog)); let df = ctx.table("fruit.apple.purchases").await?; diff --git a/datafusion/ffi/tests/ffi_udtf.rs b/datafusion/ffi/tests/ffi_udtf.rs index 8c1c64a092e1..c5f2d38c34b1 100644 --- a/datafusion/ffi/tests/ffi_udtf.rs +++ b/datafusion/ffi/tests/ffi_udtf.rs @@ -25,7 +25,7 @@ mod tests { use arrow::array::{create_array, ArrayRef}; use datafusion::error::{DataFusionError, Result}; use datafusion::prelude::SessionContext; - + use datafusion_expr::registry::FunctionRegistry; use datafusion_ffi::tests::utils::get_module; use datafusion_ffi::udtf::ForeignTableFunction; @@ -36,17 +36,19 @@ mod tests { async fn test_user_defined_table_function() -> Result<()> { let module = get_module()?; + let ctx = Arc::new(SessionContext::default()); + let function_registry = Arc::clone(&ctx) as Arc; + let ffi_table_func = module .create_table_function() .ok_or(DataFusionError::NotImplemented( "External table function provider failed to implement create_table_function" .to_string(), - ))?(); + ))?(function_registry.into()); let foreign_table_func: ForeignTableFunction = ffi_table_func.into(); let udtf = Arc::new(foreign_table_func); - let ctx = SessionContext::default(); ctx.register_udtf("my_range", udtf); let result = ctx From 8e1e5975c94a17544d2f5f57844be9a1dceda0ec Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 4 Nov 2025 14:53:55 -0500 Subject: [PATCH 19/69] add text about library marker approach --- datafusion/ffi/src/lib.rs | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index 65126f116e8f..f2fb2ac17b56 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -59,6 +59,18 @@ pub extern "C" fn version() -> u64 { static LIBRARY_MARKER: u8 = 0; +/// This utility is used to determine if two FFI structs are within +/// the same library. It is possible that the interplay between +/// foreign and local functions calls create one FFI struct that +/// references another. It is helpful to determine if a foreign +/// struct is truly foreign or in the same library. If we are in the +/// same library, then we can access the underlying types directly. +/// +/// This function works by checking the address of the library +/// marker. Each library that implements the FFI code will have +/// a different address for the marker. By checking the marker +/// address we can determine if a struct is truly Foreign or is +/// actually within the same originating library. pub extern "C" fn get_library_marker_id() -> u64 { &LIBRARY_MARKER as *const u8 as u64 } From f71ad1a12b7916d717ab966bb35a691148205608 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 4 Nov 2025 16:27:10 -0500 Subject: [PATCH 20/69] clippy --- .../ffi/ffi_module_loader/src/main.rs | 8 +++--- datafusion/ffi/Cargo.toml | 7 ++++- datafusion/ffi/src/physical_expr/mod.rs | 8 +++--- .../ffi/src/physical_expr/partitioning.rs | 6 ++--- datafusion/ffi/src/physical_expr/sort.rs | 4 +-- datafusion/ffi/src/tests/async_provider.rs | 27 ++++++++++++------- datafusion/ffi/src/tests/catalog.rs | 14 ++++++---- datafusion/ffi/src/tests/mod.rs | 20 +++++++++----- datafusion/ffi/src/tests/sync_provider.rs | 12 +++++---- datafusion/ffi/src/tests/udf_udaf_udwf.rs | 15 ++++++----- datafusion/ffi/src/udaf/accumulator_args.rs | 6 ++--- .../ffi/src/udwf/partition_evaluator_args.rs | 6 ++--- datafusion/ffi/tests/ffi_integration.rs | 2 +- 13 files changed, 78 insertions(+), 57 deletions(-) diff --git a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs index 6e376ca866e8..882fc746ca25 100644 --- a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs +++ b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs @@ -23,6 +23,7 @@ use datafusion::{ }; use abi_stable::library::{development_utils::compute_library_path, RootModule}; +use datafusion::execution::FunctionRegistry; use datafusion_ffi::table_provider::ForeignTableProvider; use ffi_module_interface::TableProviderModuleRef; @@ -39,6 +40,9 @@ async fn main() -> Result<()> { TableProviderModuleRef::load_from_directory(&library_path) .map_err(|e| DataFusionError::External(Box::new(e)))?; + let ctx = Arc::new(SessionContext::new()); + let function_registry = Arc::clone(&ctx) as Arc; + // By calling the code below, the table provided will be created within // the module's code. let ffi_table_provider = @@ -46,14 +50,12 @@ async fn main() -> Result<()> { .create_table() .ok_or(DataFusionError::NotImplemented( "External table provider failed to implement create_table".to_string(), - ))?(); + ))?(function_registry.into()); // In order to access the table provider within this executable, we need to // turn it into a `ForeignTableProvider`. let foreign_table_provider: ForeignTableProvider = (&ffi_table_provider).into(); - let ctx = SessionContext::new(); - // Display the data to show the full cycle works. ctx.register_table("external_table", Arc::new(foreign_table_provider))?; let df = ctx.table("external_table").await?; diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index c202f372bca1..c8aee693a37d 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -72,5 +72,10 @@ datafusion = { workspace = true, default-features = false, features = ["sql"] } doc-comment = { workspace = true } [features] -integration-tests = ["dep:datafusion-functions"] +integration-tests = [ + "dep:datafusion-functions", + "dep:datafusion-functions-aggregate", + "dep:datafusion-functions-table", + "dep:datafusion-functions-window", +] tarpaulin_include = [] # Exists only to prevent warnings on stable and still have accurate coverage diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index bb3b6f4eb682..b3b9e8237390 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -472,7 +472,7 @@ impl From for Arc { let children = unsafe { (expr.children)(&expr) .into_iter() - .map(|child| >::from(child)) + .map(>::from) .collect() }; @@ -552,7 +552,7 @@ impl PhysicalExpr for ForeignPhysicalExpr { unsafe { let children = children.into_iter().map(FFI_PhysicalExpr::from).collect(); df_result!((self.expr.new_with_children)(&self.expr, &children) - .map(|expr| >::from(expr))) + .map(>::from)) } } @@ -658,9 +658,7 @@ impl PhysicalExpr for ForeignPhysicalExpr { fn snapshot(&self) -> Result>> { unsafe { let result = df_result!((self.expr.snapshot)(&self.expr))?; - Ok(result - .map(|ffi_expr| >::from(ffi_expr)) - .into()) + Ok(result.map(>::from).into()) } } diff --git a/datafusion/ffi/src/physical_expr/partitioning.rs b/datafusion/ffi/src/physical_expr/partitioning.rs index 5bc5b1302094..adf7d9892c72 100644 --- a/datafusion/ffi/src/physical_expr/partitioning.rs +++ b/datafusion/ffi/src/physical_expr/partitioning.rs @@ -1,4 +1,4 @@ -use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; +use crate::physical_expr::FFI_PhysicalExpr; use abi_stable::std_types::RVec; use abi_stable::StableAbi; use datafusion_physical_expr::{Partitioning, PhysicalExpr}; @@ -40,9 +40,7 @@ impl From for Partitioning { FFI_Partitioning::Hash(exprs, size) => { let exprs = exprs .into_iter() - .map(|expr| { - >::from(expr) - }) + .map(>::from) .collect(); Self::Hash(exprs, size) } diff --git a/datafusion/ffi/src/physical_expr/sort.rs b/datafusion/ffi/src/physical_expr/sort.rs index a8e6b0d2109e..3d67ff4c863c 100644 --- a/datafusion/ffi/src/physical_expr/sort.rs +++ b/datafusion/ffi/src/physical_expr/sort.rs @@ -16,14 +16,14 @@ // under the License. use crate::expr::expr_properties::FFI_SortOptions; -use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; +use crate::physical_expr::FFI_PhysicalExpr; use abi_stable::std_types::RVec; use abi_stable::StableAbi; use arrow_schema::SortOptions; use datafusion_common::{exec_datafusion_err, DataFusionError}; use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; -use std::sync::Arc; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use std::sync::Arc; #[repr(C)] #[derive(Debug, StableAbi)] diff --git a/datafusion/ffi/src/tests/async_provider.rs b/datafusion/ffi/src/tests/async_provider.rs index fe5551f23999..102d6feb03f2 100644 --- a/datafusion/ffi/src/tests/async_provider.rs +++ b/datafusion/ffi/src/tests/async_provider.rs @@ -27,23 +27,23 @@ use std::{any::Any, fmt::Debug, sync::Arc}; +use super::create_record_batch; +use crate::function_registry::FFI_WeakFunctionRegistry; use crate::table_provider::FFI_TableProvider; use arrow::array::RecordBatch; use arrow::datatypes::Schema; use async_trait::async_trait; -use datafusion_common::{exec_err, error::Result}; -use futures::Stream; -use tokio::{ - runtime::Handle, - sync::{broadcast, mpsc}, -}; use datafusion_catalog::{Session, TableProvider}; +use datafusion_common::{error::Result, exec_err}; use datafusion_execution::RecordBatchStream; use datafusion_expr::Expr; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_plan::ExecutionPlan; -use crate::function_registry::FFI_WeakFunctionRegistry; -use super::create_record_batch; +use futures::Stream; +use tokio::{ + runtime::Handle, + sync::{broadcast, mpsc}, +}; #[derive(Debug)] pub struct AsyncTableProvider { @@ -274,7 +274,14 @@ impl Stream for AsyncTestRecordBatchStream { } } -pub(crate) fn create_async_table_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { +pub(crate) fn create_async_table_provider( + function_registry: FFI_WeakFunctionRegistry, +) -> FFI_TableProvider { let (table_provider, tokio_rt) = start_async_provider(); - FFI_TableProvider::new(Arc::new(table_provider), true, Some(tokio_rt), function_registry) + FFI_TableProvider::new( + Arc::new(table_provider), + true, + Some(tokio_rt), + function_registry, + ) } diff --git a/datafusion/ffi/src/tests/catalog.rs b/datafusion/ffi/src/tests/catalog.rs index 69774807dd9d..3c75a50f40de 100644 --- a/datafusion/ffi/src/tests/catalog.rs +++ b/datafusion/ffi/src/tests/catalog.rs @@ -28,13 +28,15 @@ use std::{any::Any, fmt::Debug, sync::Arc}; use crate::catalog_provider::FFI_CatalogProvider; +use crate::function_registry::FFI_WeakFunctionRegistry; use arrow::datatypes::Schema; use async_trait::async_trait; -use datafusion_catalog::{CatalogProvider, MemTable, MemoryCatalogProvider, MemorySchemaProvider, SchemaProvider, TableProvider}; -use datafusion_common::error::{DataFusionError, Result} -; +use datafusion_catalog::{ + CatalogProvider, MemTable, MemoryCatalogProvider, MemorySchemaProvider, + SchemaProvider, TableProvider, +}; +use datafusion_common::error::{DataFusionError, Result}; use datafusion_common::exec_err; -use crate::function_registry::FFI_WeakFunctionRegistry; /// This schema provider is intended only for unit tests. It prepopulates with one /// table and only allows for tables named sales and purchases. @@ -173,7 +175,9 @@ impl CatalogProvider for FixedCatalogProvider { } } -pub(crate) extern "C" fn create_catalog_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_CatalogProvider { +pub(crate) extern "C" fn create_catalog_provider( + function_registry: FFI_WeakFunctionRegistry, +) -> FFI_CatalogProvider { let catalog_provider = Arc::new(FixedCatalogProvider::default()); FFI_CatalogProvider::new(catalog_provider, None, function_registry) } diff --git a/datafusion/ffi/src/tests/mod.rs b/datafusion/ffi/src/tests/mod.rs index 9ed98387317e..8ba910575547 100644 --- a/datafusion/ffi/src/tests/mod.rs +++ b/datafusion/ffi/src/tests/mod.rs @@ -34,16 +34,16 @@ use crate::udaf::FFI_AggregateUDF; use crate::udwf::FFI_WindowUDF; use super::{table_provider::FFI_TableProvider, udf::FFI_ScalarUDF}; +use crate::function_registry::FFI_WeakFunctionRegistry; use arrow::array::RecordBatch; -use async_provider::create_async_table_provider; use arrow::datatypes::{DataType, Field, Schema}; +use async_provider::create_async_table_provider; use datafusion_common::record_batch; use sync_provider::create_sync_table_provider; use udf_udaf_udwf::{ create_ffi_abs_func, create_ffi_random_func, create_ffi_rank_func, create_ffi_stddev_func, create_ffi_sum_func, create_ffi_table_func, }; -use crate::function_registry::FFI_WeakFunctionRegistry; mod async_provider; pub mod catalog; @@ -59,17 +59,22 @@ pub mod utils; /// module. pub struct ForeignLibraryModule { /// Construct an opinionated catalog provider - pub create_catalog: extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_CatalogProvider, + pub create_catalog: + extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_CatalogProvider, /// Constructs the table provider - pub create_table: extern "C" fn(synchronous: bool, function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider, + pub create_table: extern "C" fn( + synchronous: bool, + function_registry: FFI_WeakFunctionRegistry, + ) -> FFI_TableProvider, /// Create a scalar UDF pub create_scalar_udf: extern "C" fn() -> FFI_ScalarUDF, pub create_nullary_udf: extern "C" fn() -> FFI_ScalarUDF, - pub create_table_function: extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableFunction, + pub create_table_function: + extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableFunction, /// Create an aggregate UDAF using sum pub create_sum_udaf: extern "C" fn() -> FFI_AggregateUDF, @@ -110,7 +115,10 @@ pub fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { /// Here we only wish to create a simple table provider as an example. /// We create an in-memory table and convert it to it's FFI counterpart. -extern "C" fn construct_table_provider(synchronous: bool, function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { +extern "C" fn construct_table_provider( + synchronous: bool, + function_registry: FFI_WeakFunctionRegistry, +) -> FFI_TableProvider { match synchronous { true => create_sync_table_provider(function_registry), false => create_async_table_provider(function_registry), diff --git a/datafusion/ffi/src/tests/sync_provider.rs b/datafusion/ffi/src/tests/sync_provider.rs index f5981640f451..16c2e7cc1e61 100644 --- a/datafusion/ffi/src/tests/sync_provider.rs +++ b/datafusion/ffi/src/tests/sync_provider.rs @@ -15,13 +15,15 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; -use datafusion_catalog::MemTable; -use crate::table_provider::FFI_TableProvider; -use crate::function_registry::FFI_WeakFunctionRegistry; use super::{create_record_batch, create_test_schema}; +use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::table_provider::FFI_TableProvider; +use datafusion_catalog::MemTable; +use std::sync::Arc; -pub(crate) fn create_sync_table_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { +pub(crate) fn create_sync_table_provider( + function_registry: FFI_WeakFunctionRegistry, +) -> FFI_TableProvider { let schema = create_test_schema(); // It is useful to create these as multiple record batches diff --git a/datafusion/ffi/src/tests/udf_udaf_udwf.rs b/datafusion/ffi/src/tests/udf_udaf_udwf.rs index 3f43bf0e0266..4e0a60547a16 100644 --- a/datafusion/ffi/src/tests/udf_udaf_udwf.rs +++ b/datafusion/ffi/src/tests/udf_udaf_udwf.rs @@ -20,17 +20,16 @@ use crate::{ udwf::FFI_WindowUDF, }; - -use std::sync::Arc; +use crate::function_registry::FFI_WeakFunctionRegistry; +use datafusion_catalog::TableFunctionImpl; +use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use datafusion_functions::math::abs::AbsFunc; +use datafusion_functions::math::random::RandomFunc; use datafusion_functions_aggregate::stddev::Stddev; use datafusion_functions_aggregate::sum::Sum; use datafusion_functions_table::generate_series::RangeFunc; use datafusion_functions_window::rank::Rank; -use datafusion_catalog::TableFunctionImpl; -use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; -use datafusion_functions::math::random::RandomFunc; -use crate::function_registry::FFI_WeakFunctionRegistry; +use std::sync::Arc; pub(crate) extern "C" fn create_ffi_abs_func() -> FFI_ScalarUDF { let udf: Arc = Arc::new(AbsFunc::new().into()); @@ -44,7 +43,9 @@ pub(crate) extern "C" fn create_ffi_random_func() -> FFI_ScalarUDF { udf.into() } -pub(crate) extern "C" fn create_ffi_table_func(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableFunction { +pub(crate) extern "C" fn create_ffi_table_func( + function_registry: FFI_WeakFunctionRegistry, +) -> FFI_TableFunction { let udtf: Arc = Arc::new(RangeFunc {}); FFI_TableFunction::new(udtf, None, function_registry) diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 9636c3004654..65ed1ff09cb6 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use crate::arrow_wrappers::WrappedSchema; use crate::physical_expr::sort::FFI_PhysicalSortExpr; -use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; +use crate::physical_expr::FFI_PhysicalExpr; use abi_stable::{ std_types::{RString, RVec}, StableAbi, @@ -104,9 +104,7 @@ impl TryFrom for ForeignAccumulatorArgs { let exprs = value .exprs .into_iter() - .map(|expr| { - >::from(expr) - }) + .map(>::from) .collect(); let return_field = Arc::new((&value.return_field.0).try_into()?); diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index af284d80a87b..2030cf0c304c 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -18,7 +18,7 @@ use std::{collections::HashMap, sync::Arc}; use crate::arrow_wrappers::WrappedSchema; -use crate::physical_expr::{FFI_PhysicalExpr, ForeignPhysicalExpr}; +use crate::physical_expr::FFI_PhysicalExpr; use abi_stable::{std_types::RVec, StableAbi}; use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef}, @@ -131,9 +131,7 @@ impl TryFrom for ForeignPartitionEvaluatorArgs { let input_exprs = value .input_exprs .into_iter() - .map(|expr| { - >::from(expr) - }) + .map(>::from) .collect::>(); let input_fields = input_exprs diff --git a/datafusion/ffi/tests/ffi_integration.rs b/datafusion/ffi/tests/ffi_integration.rs index 5176a9acc6f7..f01a49f217be 100644 --- a/datafusion/ffi/tests/ffi_integration.rs +++ b/datafusion/ffi/tests/ffi_integration.rs @@ -21,12 +21,12 @@ mod tests { use datafusion::error::{DataFusionError, Result}; use datafusion::prelude::SessionContext; + use datafusion_expr::registry::FunctionRegistry; use datafusion_ffi::catalog_provider::ForeignCatalogProvider; use datafusion_ffi::table_provider::ForeignTableProvider; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; use std::sync::Arc; - use datafusion_expr::registry::FunctionRegistry; /// It is important that this test is in the `tests` directory and not in the /// library directory so we can verify we are building a dynamic library and From e7263d8950454b59226f249b5919e3b2d50fa411 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 4 Nov 2025 16:39:25 -0500 Subject: [PATCH 21/69] switch catalog provider to use library marker --- .../ffi/ffi_example_table_provider/src/lib.rs | 4 +++- datafusion/ffi/src/catalog_provider.rs | 21 ++++++++++++++----- datafusion/ffi/src/physical_expr/mod.rs | 2 ++ datafusion/ffi/tests/ffi_integration.rs | 6 +++--- 4 files changed, 24 insertions(+), 9 deletions(-) diff --git a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs index 33c2d5b6993d..c8362d1993c1 100644 --- a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs @@ -35,7 +35,9 @@ fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { /// Here we only wish to create a simple table provider as an example. /// We create an in-memory table and convert it to it's FFI counterpart. -extern "C" fn construct_simple_table_provider(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableProvider { +extern "C" fn construct_simple_table_provider( + function_registry: FFI_WeakFunctionRegistry, +) -> FFI_TableProvider { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Float64, true), diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index 2c399283a09c..c404bd64c932 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -73,6 +73,10 @@ pub struct FFI_CatalogProvider { /// Internal data. This is only to be accessed by the provider of the plan. /// A [`ForeignCatalogProvider`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_CatalogProvider {} @@ -84,9 +88,9 @@ struct ProviderPrivateData { } impl FFI_CatalogProvider { - unsafe fn inner(&self) -> &Arc { + fn inner(&self) -> &Arc { let private_data = self.private_data as *const ProviderPrivateData; - &(*private_data).provider + unsafe { &(*private_data).provider } } unsafe fn runtime(&self) -> Option { @@ -186,6 +190,7 @@ unsafe extern "C" fn clone_fn_wrapper( release: release_fn_wrapper, version: super::version, private_data, + library_marker_id: crate::get_library_marker_id, } } @@ -214,6 +219,7 @@ impl FFI_CatalogProvider { release: release_fn_wrapper, version: super::version, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -228,9 +234,14 @@ pub struct ForeignCatalogProvider(FFI_CatalogProvider); unsafe impl Send for ForeignCatalogProvider {} unsafe impl Sync for ForeignCatalogProvider {} -impl From<&FFI_CatalogProvider> for ForeignCatalogProvider { +impl From<&FFI_CatalogProvider> for Arc { fn from(provider: &FFI_CatalogProvider) -> Self { - Self(provider.clone()) + if (provider.library_marker_id)() == crate::get_library_marker_id() { + return Arc::clone(provider.inner()); + } + + Arc::new(ForeignCatalogProvider(provider.clone())) + as Arc } } @@ -328,7 +339,7 @@ mod tests { let ffi_catalog = FFI_CatalogProvider::new(catalog, None, function_registry.into()); - let foreign_catalog: ForeignCatalogProvider = (&ffi_catalog).into(); + let foreign_catalog: Arc = (&ffi_catalog).into(); let prior_schema_names = foreign_catalog.schema_names(); assert_eq!(prior_schema_names.len(), 1); diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs index b3b9e8237390..036e908dcf91 100644 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ b/datafusion/ffi/src/physical_expr/mod.rs @@ -136,6 +136,8 @@ pub struct FFI_PhysicalExpr { /// A [`ForeignExecutionPlan`] should never attempt to access this data. pub private_data: *mut c_void, + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. pub library_marker_id: extern "C" fn() -> u64, } diff --git a/datafusion/ffi/tests/ffi_integration.rs b/datafusion/ffi/tests/ffi_integration.rs index f01a49f217be..2d460e7216cf 100644 --- a/datafusion/ffi/tests/ffi_integration.rs +++ b/datafusion/ffi/tests/ffi_integration.rs @@ -21,8 +21,8 @@ mod tests { use datafusion::error::{DataFusionError, Result}; use datafusion::prelude::SessionContext; + use datafusion_catalog::CatalogProvider; use datafusion_expr::registry::FunctionRegistry; - use datafusion_ffi::catalog_provider::ForeignCatalogProvider; use datafusion_ffi::table_provider::ForeignTableProvider; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; @@ -85,9 +85,9 @@ mod tests { "External catalog provider failed to implement create_catalog" .to_string(), ))?(function_registry.into()); - let foreign_catalog: ForeignCatalogProvider = (&ffi_catalog).into(); + let foreign_catalog: Arc = (&ffi_catalog).into(); - let _ = ctx.register_catalog("fruit", Arc::new(foreign_catalog)); + let _ = ctx.register_catalog("fruit", foreign_catalog); let df = ctx.table("fruit.apple.purchases").await?; From e0996b21d395a8fde053be00ba3aed8d2c2161c1 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 4 Nov 2025 18:04:36 -0500 Subject: [PATCH 22/69] switch execution plan to use library marker --- datafusion/ffi/src/execution_plan.rs | 65 ++++++++++++++++++---------- datafusion/ffi/src/session/mod.rs | 6 +-- datafusion/ffi/src/table_provider.rs | 18 +++----- 3 files changed, 51 insertions(+), 38 deletions(-) diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index a31e2ea7dbef..f6fbe6971549 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -17,6 +17,10 @@ use std::{ffi::c_void, pin::Pin, sync::Arc}; +use crate::{ + df_result, plan_properties::FFI_PlanProperties, + record_batch_stream::FFI_RecordBatchStream, rresult, +}; use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, @@ -27,11 +31,6 @@ use datafusion_physical_plan::DisplayFormatType; use datafusion_physical_plan::{DisplayAs, ExecutionPlan, PlanProperties}; use tokio::runtime::Handle; -use crate::{ - df_result, plan_properties::FFI_PlanProperties, - record_batch_stream::FFI_RecordBatchStream, rresult, -}; - /// A stable struct for sharing a [`ExecutionPlan`] across FFI boundaries. #[repr(C)] #[derive(Debug, StableAbi)] @@ -63,6 +62,10 @@ pub struct FFI_ExecutionPlan { /// Internal data. This is only to be accessed by the provider of the plan. /// A [`ForeignExecutionPlan`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_ExecutionPlan {} @@ -74,11 +77,17 @@ pub struct ExecutionPlanPrivateData { pub runtime: Option, } +impl FFI_ExecutionPlan { + fn inner(&self) -> &Arc { + let private_data = self.private_data as *const ExecutionPlanPrivateData; + unsafe { &(*private_data).plan } + } +} + unsafe extern "C" fn properties_fn_wrapper( plan: &FFI_ExecutionPlan, ) -> FFI_PlanProperties { - let private_data = plan.private_data as *const ExecutionPlanPrivateData; - let plan = &(*private_data).plan; + let plan = plan.inner(); plan.properties().into() } @@ -117,10 +126,7 @@ unsafe extern "C" fn execute_fn_wrapper( } unsafe extern "C" fn name_fn_wrapper(plan: &FFI_ExecutionPlan) -> RString { - let private_data = plan.private_data as *const ExecutionPlanPrivateData; - let plan = &(*private_data).plan; - - plan.name().into() + plan.inner().name().into() } unsafe extern "C" fn release_fn_wrapper(plan: &mut FFI_ExecutionPlan) { @@ -166,6 +172,7 @@ impl FFI_ExecutionPlan { clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -216,10 +223,14 @@ impl DisplayAs for ForeignExecutionPlan { } } -impl TryFrom<&FFI_ExecutionPlan> for ForeignExecutionPlan { +impl TryFrom<&FFI_ExecutionPlan> for Arc { type Error = DataFusionError; fn try_from(plan: &FFI_ExecutionPlan) -> Result { + if (plan.library_marker_id)() == crate::get_library_marker_id() { + return Ok(Arc::clone(plan.inner())); + } + unsafe { let name = (plan.name)(plan).into(); @@ -228,16 +239,17 @@ impl TryFrom<&FFI_ExecutionPlan> for ForeignExecutionPlan { let children_rvec = (plan.children)(plan); let children = children_rvec .iter() - .map(ForeignExecutionPlan::try_from) - .map(|child| child.map(|c| Arc::new(c) as Arc)) + .map(>::try_from) .collect::>>()?; - Ok(Self { + let plan = ForeignExecutionPlan { name, plan: plan.clone(), properties, children, - }) + }; + + Ok(Arc::new(plan)) } } } @@ -369,6 +381,10 @@ mod tests { } } + extern "C" fn mock_library_marker_id() -> u64 { + crate::get_library_marker_id() + 1 + } + #[test] fn test_round_trip_ffi_execution_plan() -> Result<()> { let schema = @@ -378,14 +394,17 @@ mod tests { let original_plan = Arc::new(EmptyExec::new(schema)); let original_name = original_plan.name().to_string(); - let local_plan = FFI_ExecutionPlan::new(original_plan, ctx.task_ctx(), None); + let mut local_plan = FFI_ExecutionPlan::new(original_plan, ctx.task_ctx(), None); + + // Force round trip to go through foreign provider + local_plan.library_marker_id = mock_library_marker_id; - let foreign_plan: ForeignExecutionPlan = (&local_plan).try_into()?; + let foreign_plan: Arc = (&local_plan).try_into()?; assert!(original_name == foreign_plan.name()); let display = datafusion::physical_plan::display::DisplayableExecutionPlan::new( - &foreign_plan, + foreign_plan.as_ref(), ); let buf = display.one_line().to_string(); @@ -406,11 +425,11 @@ mod tests { // Version 1: Adding child to the foreign plan let child_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); let child_local = FFI_ExecutionPlan::new(child_plan, ctx.task_ctx(), None); - let child_foreign = Arc::new(ForeignExecutionPlan::try_from(&child_local)?); + let child_foreign = >::try_from(&child_local)?; let parent_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); let parent_local = FFI_ExecutionPlan::new(parent_plan, ctx.task_ctx(), None); - let parent_foreign = Arc::new(ForeignExecutionPlan::try_from(&parent_local)?); + let parent_foreign = >::try_from(&parent_local)?; assert_eq!(parent_foreign.children().len(), 0); assert_eq!(child_foreign.children().len(), 0); @@ -421,12 +440,12 @@ mod tests { // Version 2: Adding child to the local plan let child_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); let child_local = FFI_ExecutionPlan::new(child_plan, ctx.task_ctx(), None); - let child_foreign = Arc::new(ForeignExecutionPlan::try_from(&child_local)?); + let child_foreign = >::try_from(&child_local)?; let parent_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); let parent_plan = parent_plan.with_new_children(vec![child_foreign])?; let parent_local = FFI_ExecutionPlan::new(parent_plan, ctx.task_ctx(), None); - let parent_foreign = Arc::new(ForeignExecutionPlan::try_from(&parent_local)?); + let parent_foreign = >::try_from(&parent_local)?; assert_eq!(parent_foreign.children().len(), 1); diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index f58375c9a898..f6461564dd80 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -16,7 +16,7 @@ // under the License. use crate::arrow_wrappers::WrappedSchema; -use crate::execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}; +use crate::execution_plan::FFI_ExecutionPlan; use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; use crate::session::config::{FFI_SessionConfig, ForeignSessionConfig}; use crate::session::task::FFI_TaskContext; @@ -460,9 +460,9 @@ impl Session for ForeignSession { ) .await )?; - let physical_plan = ForeignExecutionPlan::try_from(&physical_plan)?; + let physical_plan = >::try_from(&physical_plan)?; - Ok(Arc::new(physical_plan)) + Ok(physical_plan) } } diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 9027c1f18162..6789007a7592 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -40,7 +40,7 @@ use crate::session::{FFI_Session, ForeignSession}; use crate::{ arrow_wrappers::WrappedSchema, df_result, - execution_plan::{FFI_ExecutionPlan, ForeignExecutionPlan}, + execution_plan::FFI_ExecutionPlan, insert_op::FFI_InsertOp, rresult_return, table_source::{FFI_TableProviderFilterPushDown, FFI_TableType}, @@ -291,14 +291,8 @@ unsafe extern "C" fn insert_into_fn_wrapper( async move { let session = rresult_return!(session); - // let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); - // let session = SessionStateBuilder::new() - // .with_default_features() - // .with_config(config.0) - // .build(); - // let ctx = SessionContext::new_with_state(session); - let input = rresult_return!(ForeignExecutionPlan::try_from(&input).map(Arc::new)); + let input = rresult_return!(>::try_from(&input)); let insert_op = InsertOp::from(insert_op); @@ -445,10 +439,10 @@ impl TableProvider for ForeignTableProvider { ) .await; - ForeignExecutionPlan::try_from(&df_result!(maybe_plan)?)? + >::try_from(&df_result!(maybe_plan)?)? }; - Ok(Arc::new(plan)) + Ok(plan) } /// Tests whether the table provider can make use of a filter expression @@ -500,10 +494,10 @@ impl TableProvider for ForeignTableProvider { let maybe_plan = (self.0.insert_into)(&self.0, &session, &input, insert_op).await; - ForeignExecutionPlan::try_from(&df_result!(maybe_plan)?)? + >::try_from(&df_result!(maybe_plan)?)? }; - Ok(Arc::new(plan)) + Ok(plan) } } From 369303fcbdc9406be80dd50ba76352cea41d76fd Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 5 Nov 2025 07:26:23 -0500 Subject: [PATCH 23/69] switch function registry to use library marker --- .../ffi/ffi_module_loader/src/main.rs | 2 +- datafusion/ffi/src/catalog_provider.rs | 3 +- datafusion/ffi/src/function_registry.rs | 34 +++++++++++++----- datafusion/ffi/src/schema_provider.rs | 3 +- datafusion/ffi/src/session/mod.rs | 11 +++--- datafusion/ffi/src/table_provider.rs | 35 ++++++++++++------- datafusion/ffi/src/udtf.rs | 12 ++++--- datafusion/ffi/tests/ffi_integration.rs | 6 ++-- datafusion/ffi/tests/ffi_udtf.rs | 3 +- 9 files changed, 73 insertions(+), 36 deletions(-) diff --git a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs index 882fc746ca25..9420e68a21cc 100644 --- a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs +++ b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs @@ -41,7 +41,7 @@ async fn main() -> Result<()> { .map_err(|e| DataFusionError::External(Box::new(e)))?; let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = Arc::clone(&ctx) as Arc; // By calling the code below, the table provided will be created within // the module's code. diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index c404bd64c932..1c959eef6adc 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -334,7 +334,8 @@ mod tests { .is_none()); let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let ffi_catalog = FFI_CatalogProvider::new(catalog, None, function_registry.into()); diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 850f4e5ecc5e..04f87b158c13 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -23,7 +23,7 @@ use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, }; -use datafusion_common::not_impl_err; +use datafusion_common::{exec_datafusion_err, not_impl_err, DataFusionError}; use datafusion_expr::expr_rewriter::FunctionRewrite; use datafusion_expr::planner::ExprPlanner; use datafusion_expr::registry::FunctionRegistry; @@ -62,19 +62,28 @@ pub struct FFI_WeakFunctionRegistry { /// Internal data. This is only to be accessed by the provider of the plan. /// A [`ForeignWeakFunctionRegistry`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_WeakFunctionRegistry {} unsafe impl Sync for FFI_WeakFunctionRegistry {} struct RegistryPrivateData { - registry: Weak, + registry: Weak, } impl FFI_WeakFunctionRegistry { - unsafe fn inner(&self) -> Result, RString> { + fn inner(&self) -> Result, DataFusionError> { let private_data = self.private_data as *const RegistryPrivateData; - (*private_data).registry.upgrade().ok_or_else(|| "Unable to access FunctionRegistry via FFI. Ensure owning object has not gone out of scope.".into()) + unsafe { + (*private_data) + .registry + .upgrade() + .ok_or_else(|| exec_datafusion_err!("Unable to access FunctionRegistry via FFI. Ensure owning object has not gone out of scope.")) + } } } @@ -152,6 +161,7 @@ unsafe extern "C" fn clone_fn_wrapper( release: release_fn_wrapper, version: super::version, private_data, + library_marker_id: crate::get_library_marker_id, } } @@ -161,9 +171,9 @@ impl Drop for FFI_WeakFunctionRegistry { } } -impl From> for FFI_WeakFunctionRegistry { +impl From> for FFI_WeakFunctionRegistry { /// Creates a new [`FFI_WeakFunctionRegistry`]. - fn from(registry: Arc) -> Self { + fn from(registry: Arc) -> Self { let registry = Arc::downgrade(®istry); let private_data = Box::new(RegistryPrivateData { registry }); @@ -180,6 +190,7 @@ impl From> for FFI_WeakFunctionRegistry { release: release_fn_wrapper, version: super::version, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -194,9 +205,14 @@ pub struct ForeignWeakFunctionRegistry(FFI_WeakFunctionRegistry); unsafe impl Send for ForeignWeakFunctionRegistry {} unsafe impl Sync for ForeignWeakFunctionRegistry {} -impl From<&FFI_WeakFunctionRegistry> for ForeignWeakFunctionRegistry { - fn from(provider: &FFI_WeakFunctionRegistry) -> Self { - Self(provider.clone()) +impl TryFrom<&FFI_WeakFunctionRegistry> for Arc { + type Error = DataFusionError; + fn try_from(value: &FFI_WeakFunctionRegistry) -> Result { + if (value.library_marker_id)() == crate::get_library_marker_id() { + return value.inner(); + } + + Ok(Arc::new(ForeignWeakFunctionRegistry(value.clone()))) } } diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index 4ea26dd65ad5..e461b2a5a499 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -348,7 +348,8 @@ mod tests { .unwrap() .is_none()); let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let ffi_schema_provider = FFI_SchemaProvider::new(schema_provider, None, function_registry.into()); diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index f6461564dd80..fd9c2a662f0e 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -17,7 +17,7 @@ use crate::arrow_wrappers::WrappedSchema; use crate::execution_plan::FFI_ExecutionPlan; -use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; +use crate::function_registry::FFI_WeakFunctionRegistry; use crate::session::config::{FFI_SessionConfig, ForeignSessionConfig}; use crate::session::task::FFI_TaskContext; use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; @@ -130,9 +130,10 @@ impl FFI_Session { (*private_data).session } - unsafe fn function_registry(&self) -> Arc { - let registry = ForeignWeakFunctionRegistry::from(&self.function_registry); - Arc::new(registry) as Arc + fn function_registry( + &self, + ) -> Result, DataFusionError> { + (&self.function_registry).try_into() } unsafe fn runtime(&self) -> &Option { @@ -182,7 +183,7 @@ unsafe extern "C" fn create_physical_expr_fn_wrapper( expr_serialized: RVec, schema: WrappedSchema, ) -> RResult, RString> { - let function_registry = session.function_registry(); + let function_registry = rresult_return!(session.function_registry()); let session = session.inner(); let codec = DefaultLogicalExtensionCodec {}; diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 6789007a7592..896fe1ec02bc 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -35,7 +35,7 @@ use datafusion_proto::{ use prost::Message; use tokio::runtime::Handle; -use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; +use crate::function_registry::FFI_WeakFunctionRegistry; use crate::session::{FFI_Session, ForeignSession}; use crate::{ arrow_wrappers::WrappedSchema, @@ -47,6 +47,7 @@ use crate::{ }; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::dml::InsertOp; +use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType}; use datafusion_physical_plan::ExecutionPlan; @@ -181,7 +182,7 @@ unsafe extern "C" fn table_type_fn_wrapper( fn supports_filters_pushdown_internal( provider: &Arc, filters_serialized: &[u8], - function_registry: &ForeignWeakFunctionRegistry, + function_registry: &dyn FunctionRegistry, ) -> Result> { let codec = DefaultLogicalExtensionCodec {}; @@ -209,14 +210,19 @@ unsafe extern "C" fn supports_filters_pushdown_fn_wrapper( provider: &FFI_TableProvider, filters_serialized: RVec, ) -> RResult, RString> { - let function_registry = - ForeignWeakFunctionRegistry::from(&provider.function_registry); + let function_registry = rresult_return!( + >::try_from(&provider.function_registry) + ); let private_data = provider.private_data as *const ProviderPrivateData; let provider = &(*private_data).provider; - supports_filters_pushdown_internal(provider, &filters_serialized, &function_registry) - .map_err(|e| e.to_string().into()) - .into() + supports_filters_pushdown_internal( + provider, + &filters_serialized, + function_registry.as_ref(), + ) + .map_err(|e| e.to_string().into()) + .into() } unsafe extern "C" fn scan_fn_wrapper( @@ -227,13 +233,15 @@ unsafe extern "C" fn scan_fn_wrapper( limit: ROption, ) -> FfiFuture> { let function_registry = - ForeignWeakFunctionRegistry::from(&provider.function_registry); + >::try_from(&provider.function_registry) + .expect(""); let session = ForeignSession::try_from(session); let private_data = provider.private_data as *mut ProviderPrivateData; let internal_provider = &(*private_data).provider; let runtime = &(*private_data).runtime; async move { + // let function_registry = rresult_return!(function_registry); let session = rresult_return!(session); // let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); // let session = SessionStateBuilder::new() @@ -253,7 +261,7 @@ unsafe extern "C" fn scan_fn_wrapper( rresult_return!(parse_exprs( proto_filters.expr.iter(), - &function_registry, + function_registry.as_ref(), &codec )) } @@ -530,7 +538,8 @@ mod tests { )?; let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); @@ -574,7 +583,8 @@ mod tests { )?; let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); @@ -623,7 +633,8 @@ mod tests { )?; let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1]])?); diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index b7c944313ff8..504b20bc5999 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -22,13 +22,14 @@ use abi_stable::{ StableAbi, }; -use crate::function_registry::{FFI_WeakFunctionRegistry, ForeignWeakFunctionRegistry}; +use crate::function_registry::FFI_WeakFunctionRegistry; use crate::{ df_result, rresult_return, table_provider::{FFI_TableProvider, ForeignTableProvider}, }; use datafusion_catalog::{TableFunctionImpl, TableProvider}; use datafusion_common::error::Result; +use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::Expr; use datafusion_proto::{ logical_plan::{ @@ -90,7 +91,9 @@ unsafe extern "C" fn call_fn_wrapper( args: RVec, ) -> RResult { let function_registry = udtf.function_registry.clone(); - let foreign_registry = ForeignWeakFunctionRegistry::from(&function_registry); + let foreign_registry = rresult_return!( + >::try_from(&udtf.function_registry) + ); let runtime = udtf.runtime(); let udtf = udtf.inner(); @@ -101,7 +104,7 @@ unsafe extern "C" fn call_fn_wrapper( let args = rresult_return!(parse_exprs( proto_filters.expr.iter(), - &foreign_registry, + foreign_registry.as_ref(), &codec )); @@ -306,7 +309,8 @@ mod tests { let original_udtf = Arc::new(TestUDTF {}) as Arc; let ctx = Arc::new(SessionContext::default()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let local_udtf: FFI_TableFunction = FFI_TableFunction::new( Arc::clone(&original_udtf), None, diff --git a/datafusion/ffi/tests/ffi_integration.rs b/datafusion/ffi/tests/ffi_integration.rs index 2d460e7216cf..86295b98cec5 100644 --- a/datafusion/ffi/tests/ffi_integration.rs +++ b/datafusion/ffi/tests/ffi_integration.rs @@ -35,7 +35,8 @@ mod tests { let table_provider_module = get_module()?; let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; // By calling the code below, the table provided will be created within // the module's code. @@ -76,7 +77,8 @@ mod tests { async fn test_catalog() -> Result<()> { let module = get_module()?; let ctx = Arc::new(SessionContext::default()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let ffi_catalog = module diff --git a/datafusion/ffi/tests/ffi_udtf.rs b/datafusion/ffi/tests/ffi_udtf.rs index c5f2d38c34b1..a661959d227c 100644 --- a/datafusion/ffi/tests/ffi_udtf.rs +++ b/datafusion/ffi/tests/ffi_udtf.rs @@ -37,7 +37,8 @@ mod tests { let module = get_module()?; let ctx = Arc::new(SessionContext::default()); - let function_registry = Arc::clone(&ctx) as Arc; + let function_registry = + Arc::clone(&ctx) as Arc; let ffi_table_func = module .create_table_function() From e516ec40ab5805585752efe6cd18ed9f9b54c753 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 5 Nov 2025 08:02:43 -0500 Subject: [PATCH 24/69] switch schema provider to use library marker --- datafusion/ffi/src/catalog_provider.rs | 15 +++++++++------ datafusion/ffi/src/schema_provider.rs | 20 +++++++++++++++----- 2 files changed, 24 insertions(+), 11 deletions(-) diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index 1c959eef6adc..c531598fdd87 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -128,7 +128,7 @@ unsafe extern "C" fn register_schema_fn_wrapper( schema: &FFI_SchemaProvider, ) -> RResult, RString> { let runtime = provider.runtime(); - let schema = Arc::new(ForeignSchemaProvider::from(schema)); + let schema = >::from(schema); let returned_schema = rresult_return!(provider .inner() @@ -271,7 +271,8 @@ impl CatalogProvider for ForeignCatalogProvider { (self.0.schema)(&self.0, name.into()).into(); maybe_provider.map(|provider| { - Arc::new(ForeignSchemaProvider(provider)) as Arc + >::from(&provider) + as Arc }) } } @@ -294,8 +295,9 @@ impl CatalogProvider for ForeignCatalogProvider { df_result!((self.0.register_schema)(&self.0, name.into(), schema))? .into(); - Ok(returned_schema - .map(|s| Arc::new(ForeignSchemaProvider(s)) as Arc)) + Ok(returned_schema.map(|s| { + >::from(&s) as Arc + })) } } @@ -309,8 +311,9 @@ impl CatalogProvider for ForeignCatalogProvider { df_result!((self.0.deregister_schema)(&self.0, name.into(), cascade))? .into(); - Ok(returned_schema - .map(|s| Arc::new(ForeignSchemaProvider(s)) as Arc)) + Ok(returned_schema.map(|s| { + >::from(&s) as Arc + })) } } } diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index e461b2a5a499..c24a670400f6 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -79,6 +79,10 @@ pub struct FFI_SchemaProvider { /// Internal data. This is only to be accessed by the provider of the plan. /// A [`ForeignSchemaProvider`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_SchemaProvider {} @@ -90,9 +94,9 @@ struct ProviderPrivateData { } impl FFI_SchemaProvider { - unsafe fn inner(&self) -> &Arc { + fn inner(&self) -> &Arc { let private_data = self.private_data as *const ProviderPrivateData; - &(*private_data).provider + unsafe { &(*private_data).provider } } unsafe fn runtime(&self) -> Option { @@ -194,6 +198,7 @@ unsafe extern "C" fn clone_fn_wrapper( deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, function_registry: provider.function_registry.clone(), + library_marker_id: crate::get_library_marker_id, } } @@ -225,6 +230,7 @@ impl FFI_SchemaProvider { deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, function_registry, + library_marker_id: crate::get_library_marker_id, } } } @@ -239,9 +245,13 @@ pub struct ForeignSchemaProvider(pub FFI_SchemaProvider); unsafe impl Send for ForeignSchemaProvider {} unsafe impl Sync for ForeignSchemaProvider {} -impl From<&FFI_SchemaProvider> for ForeignSchemaProvider { +impl From<&FFI_SchemaProvider> for Arc { fn from(provider: &FFI_SchemaProvider) -> Self { - Self(provider.clone()) + if (provider.library_marker_id)() == crate::get_library_marker_id() { + return Arc::clone(provider.inner()); + } + + Arc::new(ForeignSchemaProvider(provider.clone())) } } @@ -354,7 +364,7 @@ mod tests { let ffi_schema_provider = FFI_SchemaProvider::new(schema_provider, None, function_registry.into()); - let foreign_schema_provider: ForeignSchemaProvider = + let foreign_schema_provider: Arc = (&ffi_schema_provider).into(); let prior_table_names = foreign_schema_provider.table_names(); From 6309f455df30f1823f132423812b236f351ec01e Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 5 Nov 2025 08:13:15 -0500 Subject: [PATCH 25/69] switch session config to use library marker --- datafusion/ffi/src/session/config.rs | 15 ++++++--------- datafusion/ffi/src/session/mod.rs | 4 ++-- datafusion/ffi/src/session/task.rs | 8 ++++---- datafusion/ffi/src/table_provider.rs | 7 ------- 4 files changed, 12 insertions(+), 22 deletions(-) diff --git a/datafusion/ffi/src/session/config.rs b/datafusion/ffi/src/session/config.rs index bf46a1e77a6c..60a7286e32a4 100644 --- a/datafusion/ffi/src/session/config.rs +++ b/datafusion/ffi/src/session/config.rs @@ -57,7 +57,6 @@ pub struct FFI_SessionConfig { pub release: unsafe extern "C" fn(arg: &mut Self), /// Internal data. This is only to be accessed by the provider of the plan. - /// A [`ForeignSessionConfig`] should never attempt to access this data. pub private_data: *mut c_void, } @@ -148,10 +147,8 @@ impl Drop for FFI_SessionConfig { /// A wrapper struct for accessing [`SessionConfig`] across a FFI boundary. /// The [`SessionConfig`] will be generated from a hash map of the config /// options in the provider and will be reconstructed on this side of the -/// interface.s -pub struct ForeignSessionConfig(pub SessionConfig); - -impl TryFrom<&FFI_SessionConfig> for ForeignSessionConfig { +/// interface. +impl TryFrom<&FFI_SessionConfig> for SessionConfig { type Error = DataFusionError; fn try_from(config: &FFI_SessionConfig) -> Result { @@ -162,7 +159,7 @@ impl TryFrom<&FFI_SessionConfig> for ForeignSessionConfig { options_map.insert(kv_pair.0.to_string(), kv_pair.1.to_string()); }); - Ok(Self(SessionConfig::from_string_hash_map(&options_map)?)) + SessionConfig::from_string_hash_map(&options_map) } } @@ -177,11 +174,11 @@ mod tests { let ffi_config: FFI_SessionConfig = (&session_config).into(); - let foreign_config: ForeignSessionConfig = (&ffi_config).try_into()?; + let foreign_config: SessionConfig = (&ffi_config).try_into()?; - let returned_options = foreign_config.0.options().entries(); + let returned_options = foreign_config.options().entries(); - assert!(original_options.len() == returned_options.len()); + assert_eq!(original_options.len(), returned_options.len()); Ok(()) } diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index fd9c2a662f0e..54b18b8509b8 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -18,7 +18,7 @@ use crate::arrow_wrappers::WrappedSchema; use crate::execution_plan::FFI_ExecutionPlan; use crate::function_registry::FFI_WeakFunctionRegistry; -use crate::session::config::{FFI_SessionConfig, ForeignSessionConfig}; +use crate::session::config::FFI_SessionConfig; use crate::session::task::FFI_TaskContext; use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; @@ -366,7 +366,7 @@ impl TryFrom<&FFI_Session> for ForeignSession { table_options_from_rhashmap((session.table_options)(session)); let config = (session.config)(session); - let config = ForeignSessionConfig::try_from(&config)?.0; + let config = SessionConfig::try_from(&config)?; let scalar_functions = (session.scalar_functions)(session) .into_iter() diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index c2d418677eb3..edab56ba7175 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -15,13 +15,14 @@ // specific language governing permissions and limitations // under the License. -use crate::session::config::{FFI_SessionConfig, ForeignSessionConfig}; +use crate::session::config::FFI_SessionConfig; use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use abi_stable::pmr::ROption; use abi_stable::std_types::RHashMap; use abi_stable::{std_types::RString, StableAbi}; +use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; @@ -145,9 +146,8 @@ impl From for TaskContext { let task_id = (ffi_ctx.task_id)(&ffi_ctx).map(|s| s.to_string()).into(); let sesion_id = (ffi_ctx.session_id)(&ffi_ctx).into(); let session_config = (ffi_ctx.session_config)(&ffi_ctx); - let session_config = ForeignSessionConfig::try_from(&session_config) - .map(|v| v.0) - .unwrap_or_default(); + let session_config = + SessionConfig::try_from(&session_config).unwrap_or_default(); let scalar_functions = (ffi_ctx.scalar_functions)(&ffi_ctx) .into_iter() diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 896fe1ec02bc..7a22cb257529 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -241,14 +241,7 @@ unsafe extern "C" fn scan_fn_wrapper( let runtime = &(*private_data).runtime; async move { - // let function_registry = rresult_return!(function_registry); let session = rresult_return!(session); - // let config = rresult_return!(ForeignSessionConfig::try_from(&session_config)); - // let session = SessionStateBuilder::new() - // .with_default_features() - // .with_config(config.0) - // .build(); - // let ctx = SessionContext::new_with_state(session); let filters = match filters_serialized.is_empty() { true => vec![], From 5f5d1f69aed840c11ab0fba3a895de5541a056b2 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 5 Nov 2025 08:38:42 -0500 Subject: [PATCH 26/69] Working on using session locally when possible --- datafusion/ffi/src/session/mod.rs | 23 +++++++++++++++++++---- datafusion/ffi/src/table_provider.rs | 9 +++++---- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 54b18b8509b8..409fc635932c 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -114,20 +114,24 @@ pub struct FFI_Session { /// Internal data. This is only to be accessed by the provider of the plan. /// A [`ForeignSession`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_Session {} unsafe impl Sync for FFI_Session {} struct SessionPrivateData<'a> { - session: &'a (dyn Session + Send), + session: &'a (dyn Session + Send + Sync), runtime: Option, } impl FFI_Session { - unsafe fn inner(&self) -> &(dyn Session + Send) { + fn inner(&self) -> &(dyn Session + Send + Sync) { let private_data = self.private_data as *const SessionPrivateData; - (*private_data).session + unsafe { (*private_data).session } } fn function_registry( @@ -300,6 +304,7 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_Session) -> FFI_Session { release: release_fn_wrapper, version: super::version, private_data, + library_marker_id: crate::get_library_marker_id, } } @@ -312,7 +317,7 @@ impl Drop for FFI_Session { impl FFI_Session { /// Creates a new [`FFI_Session`]. pub fn new( - session: &(dyn Session + Send), + session: &(dyn Session + Send + Sync), function_registry: FFI_WeakFunctionRegistry, runtime: Option, ) -> Self { @@ -335,6 +340,7 @@ impl FFI_Session { release: release_fn_wrapper, version: super::version, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -358,6 +364,15 @@ pub struct ForeignSession { unsafe impl Send for ForeignSession {} unsafe impl Sync for ForeignSession {} +impl FFI_Session { + pub fn as_local(&self) -> Option<&(dyn Session + Send + Sync)> { + if (self.library_marker_id)() == crate::get_library_marker_id() { + return Some(self.inner()); + } + None + } +} + impl TryFrom<&FFI_Session> for ForeignSession { type Error = DataFusionError; fn try_from(session: &FFI_Session) -> Result { diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 7a22cb257529..08bf4817aa8c 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -285,13 +285,14 @@ unsafe extern "C" fn insert_into_fn_wrapper( ) -> FfiFuture> { let private_data = provider.private_data as *mut ProviderPrivateData; let internal_provider = &(*private_data).provider; - // let session_config = session_config.clone(); - let session = ForeignSession::try_from(session); + let session = session.clone(); let input = input.clone(); let runtime = &(*private_data).runtime; async move { - let session = rresult_return!(session); + let local_session = session.as_local(); + let foreign_session = rresult_return!(ForeignSession::try_from(&session)); + let session = local_session.unwrap_or(&foreign_session); let input = rresult_return!(>::try_from(&input)); @@ -299,7 +300,7 @@ unsafe extern "C" fn insert_into_fn_wrapper( let plan = rresult_return!( internal_provider - .insert_into(&session, input, insert_op) + .insert_into(session, input, insert_op) .await ); From e68b25bc32213500e1b8a163044705b5d7098088 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 5 Nov 2025 21:04:42 -0500 Subject: [PATCH 27/69] switch table provider to use library marker --- .../ffi/ffi_module_loader/src/main.rs | 6 +- datafusion/ffi/src/schema_provider.rs | 4 +- datafusion/ffi/src/table_provider.rs | 68 +++++++++++-------- datafusion/ffi/src/udtf.rs | 8 +-- datafusion/ffi/tests/ffi_integration.rs | 7 +- 5 files changed, 49 insertions(+), 44 deletions(-) diff --git a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs index 9420e68a21cc..6f9a6ee4c6c7 100644 --- a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs +++ b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs @@ -23,8 +23,8 @@ use datafusion::{ }; use abi_stable::library::{development_utils::compute_library_path, RootModule}; +use datafusion::catalog::TableProvider; use datafusion::execution::FunctionRegistry; -use datafusion_ffi::table_provider::ForeignTableProvider; use ffi_module_interface::TableProviderModuleRef; #[tokio::main] @@ -54,10 +54,10 @@ async fn main() -> Result<()> { // In order to access the table provider within this executable, we need to // turn it into a `ForeignTableProvider`. - let foreign_table_provider: ForeignTableProvider = (&ffi_table_provider).into(); + let foreign_table_provider: Arc = (&ffi_table_provider).into(); // Display the data to show the full cycle works. - ctx.register_table("external_table", Arc::new(foreign_table_provider))?; + ctx.register_table("external_table", foreign_table_provider)?; let df = ctx.table("external_table").await?; df.show().await?; diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index c24a670400f6..75437d8d1f09 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -289,9 +289,7 @@ impl SchemaProvider for ForeignSchemaProvider { let table: Option = df_result!((self.0.table)(&self.0, name.into()).await)?.into(); - let table = table.as_ref().map(|t| { - Arc::new(ForeignTableProvider::from(t)) as Arc - }); + let table = table.as_ref().map(>::from); Ok(table) } diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 08bf4817aa8c..e8187a4575eb 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -153,34 +153,44 @@ pub struct FFI_TableProvider { /// Internal data. This is only to be accessed by the provider of the plan. /// A [`ForeignExecutionPlan`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_TableProvider {} unsafe impl Sync for FFI_TableProvider {} struct ProviderPrivateData { - provider: Arc, + provider: Arc, runtime: Option, } -unsafe extern "C" fn schema_fn_wrapper(provider: &FFI_TableProvider) -> WrappedSchema { - let private_data = provider.private_data as *const ProviderPrivateData; - let provider = &(*private_data).provider; +impl FFI_TableProvider { + fn inner(&self) -> &Arc { + let private_data = self.private_data as *const ProviderPrivateData; + unsafe { &(*private_data).provider } + } + + fn runtime(&self) -> &Option { + let private_data = self.private_data as *const ProviderPrivateData; + unsafe { &(*private_data).runtime } + } +} - provider.schema().into() +unsafe extern "C" fn schema_fn_wrapper(provider: &FFI_TableProvider) -> WrappedSchema { + provider.inner().schema().into() } unsafe extern "C" fn table_type_fn_wrapper( provider: &FFI_TableProvider, ) -> FFI_TableType { - let private_data = provider.private_data as *const ProviderPrivateData; - let provider = &(*private_data).provider; - - provider.table_type().into() + provider.inner().table_type().into() } fn supports_filters_pushdown_internal( - provider: &Arc, + provider: &Arc, filters_serialized: &[u8], function_registry: &dyn FunctionRegistry, ) -> Result> { @@ -213,11 +223,9 @@ unsafe extern "C" fn supports_filters_pushdown_fn_wrapper( let function_registry = rresult_return!( >::try_from(&provider.function_registry) ); - let private_data = provider.private_data as *const ProviderPrivateData; - let provider = &(*private_data).provider; supports_filters_pushdown_internal( - provider, + provider.inner(), &filters_serialized, function_registry.as_ref(), ) @@ -236,9 +244,8 @@ unsafe extern "C" fn scan_fn_wrapper( >::try_from(&provider.function_registry) .expect(""); let session = ForeignSession::try_from(session); - let private_data = provider.private_data as *mut ProviderPrivateData; - let internal_provider = &(*private_data).provider; - let runtime = &(*private_data).runtime; + let internal_provider = Arc::clone(provider.inner()); + let runtime = provider.runtime().clone(); async move { let session = rresult_return!(session); @@ -283,11 +290,10 @@ unsafe extern "C" fn insert_into_fn_wrapper( input: &FFI_ExecutionPlan, insert_op: FFI_InsertOp, ) -> FfiFuture> { - let private_data = provider.private_data as *mut ProviderPrivateData; - let internal_provider = &(*private_data).provider; + let internal_provider = Arc::clone(provider.inner()); let session = session.clone(); let input = input.clone(); - let runtime = &(*private_data).runtime; + let runtime = provider.runtime().clone(); async move { let local_session = session.as_local(); @@ -338,6 +344,7 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_TableProvider) -> FFI_Table release: release_fn_wrapper, version: super::version, private_data, + library_marker_id: crate::get_library_marker_id, } } @@ -350,7 +357,7 @@ impl Drop for FFI_TableProvider { impl FFI_TableProvider { /// Creates a new [`FFI_TableProvider`]. pub fn new( - provider: Arc, + provider: Arc, can_support_pushdown_filters: bool, runtime: Option, function_registry: FFI_WeakFunctionRegistry, @@ -371,6 +378,7 @@ impl FFI_TableProvider { release: release_fn_wrapper, version: super::version, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -385,9 +393,13 @@ pub struct ForeignTableProvider(pub FFI_TableProvider); unsafe impl Send for ForeignTableProvider {} unsafe impl Sync for ForeignTableProvider {} -impl From<&FFI_TableProvider> for ForeignTableProvider { +impl From<&FFI_TableProvider> for Arc { fn from(provider: &FFI_TableProvider) -> Self { - Self(provider.clone()) + if (provider.library_marker_id)() == crate::get_library_marker_id() { + Arc::clone(provider.inner()) as Arc + } else { + Arc::new(ForeignTableProvider(provider.clone())) + } } } @@ -541,9 +553,9 @@ mod tests { let ffi_provider = FFI_TableProvider::new(provider, true, None, function_registry.into()); - let foreign_table_provider: ForeignTableProvider = (&ffi_provider).into(); + let foreign_table_provider: Arc = (&ffi_provider).into(); - ctx.register_table("t", Arc::new(foreign_table_provider))?; + ctx.register_table("t", foreign_table_provider)?; let df = ctx.table("t").await?; @@ -586,9 +598,9 @@ mod tests { let ffi_provider = FFI_TableProvider::new(provider, true, None, function_registry.into()); - let foreign_table_provider: ForeignTableProvider = (&ffi_provider).into(); + let foreign_table_provider: Arc = (&ffi_provider).into(); - ctx.register_table("t", Arc::new(foreign_table_provider))?; + ctx.register_table("t", foreign_table_provider)?; let result = ctx .sql("INSERT INTO t VALUES (128.0);") @@ -635,9 +647,9 @@ mod tests { let ffi_provider = FFI_TableProvider::new(provider, true, None, function_registry.into()); - let foreign_table_provider: ForeignTableProvider = (&ffi_provider).into(); + let foreign_table_provider: Arc = (&ffi_provider).into(); - ctx.register_table("t", Arc::new(foreign_table_provider))?; + ctx.register_table("t", foreign_table_provider)?; let result = ctx .sql("SELECT COUNT(*) as cnt FROM t") diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index 504b20bc5999..3924c1fb6f8f 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -23,10 +23,7 @@ use abi_stable::{ }; use crate::function_registry::FFI_WeakFunctionRegistry; -use crate::{ - df_result, rresult_return, - table_provider::{FFI_TableProvider, ForeignTableProvider}, -}; +use crate::{df_result, rresult_return, table_provider::FFI_TableProvider}; use datafusion_catalog::{TableFunctionImpl, TableProvider}; use datafusion_common::error::Result; use datafusion_expr::registry::FunctionRegistry; @@ -205,9 +202,8 @@ impl TableFunctionImpl for ForeignTableFunction { let table_provider = unsafe { (self.0.call)(&self.0, filters_serialized) }; let table_provider = df_result!(table_provider)?; - let table_provider: ForeignTableProvider = (&table_provider).into(); - Ok(Arc::new(table_provider)) + Ok((&table_provider).into()) } } diff --git a/datafusion/ffi/tests/ffi_integration.rs b/datafusion/ffi/tests/ffi_integration.rs index 86295b98cec5..92f88790f6c9 100644 --- a/datafusion/ffi/tests/ffi_integration.rs +++ b/datafusion/ffi/tests/ffi_integration.rs @@ -21,9 +21,8 @@ mod tests { use datafusion::error::{DataFusionError, Result}; use datafusion::prelude::SessionContext; - use datafusion_catalog::CatalogProvider; + use datafusion_catalog::{CatalogProvider, TableProvider}; use datafusion_expr::registry::FunctionRegistry; - use datafusion_ffi::table_provider::ForeignTableProvider; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; use std::sync::Arc; @@ -48,10 +47,10 @@ mod tests { // In order to access the table provider within this executable, we need to // turn it into a `ForeignTableProvider`. - let foreign_table_provider: ForeignTableProvider = (&ffi_table_provider).into(); + let foreign_table_provider: Arc = (&ffi_table_provider).into(); // Display the data to show the full cycle works. - ctx.register_table("external_table", Arc::new(foreign_table_provider))?; + ctx.register_table("external_table", foreign_table_provider)?; let df = ctx.table("external_table").await?; let results = df.collect().await?; From 085c7b4bc09564f75a56fda44d82f66d9d27a6e9 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 5 Nov 2025 22:19:48 -0500 Subject: [PATCH 28/69] switch groups accumulator to use library marker --- datafusion/ffi/src/udaf/groups_accumulator.rs | 22 +++++++++++++++---- datafusion/ffi/src/udaf/mod.rs | 10 +++------ 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 266468cebd46..e7f2335fed91 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -84,6 +84,10 @@ pub struct FFI_GroupsAccumulator { /// Internal data. This is only to be accessed by the provider of the accumulator. /// A [`ForeignGroupsAccumulator`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_GroupsAccumulator {} @@ -234,6 +238,7 @@ impl From> for FFI_GroupsAccumulator { release: release_fn_wrapper, private_data: Box::into_raw(Box::new(private_data)) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -258,9 +263,18 @@ pub struct ForeignGroupsAccumulator { unsafe impl Send for ForeignGroupsAccumulator {} unsafe impl Sync for ForeignGroupsAccumulator {} -impl From for ForeignGroupsAccumulator { +impl From for Box { fn from(accumulator: FFI_GroupsAccumulator) -> Self { - Self { accumulator } + if (accumulator.library_marker_id)() == crate::get_library_marker_id() { + unsafe { + let private_data = Box::from_raw( + accumulator.private_data as *mut GroupsAccumulatorPrivateData, + ); + private_data.accumulator + } + } else { + Box::new(ForeignGroupsAccumulator { accumulator }) + } } } @@ -434,14 +448,14 @@ mod tests { }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::bool_op::BooleanGroupsAccumulator; - use super::{FFI_EmitTo, FFI_GroupsAccumulator, ForeignGroupsAccumulator}; + use super::{FFI_EmitTo, FFI_GroupsAccumulator}; #[test] fn test_foreign_avg_accumulator() -> Result<()> { let boxed_accum: Box = Box::new(BooleanGroupsAccumulator::new(|a, b| a && b, true)); let ffi_accum: FFI_GroupsAccumulator = boxed_accum.into(); - let mut foreign_accum: ForeignGroupsAccumulator = ffi_accum.into(); + let mut foreign_accum: Box = ffi_accum.into(); // Send in an array to evaluate. We want a mean of 30 and standard deviation of 4. let values = create_array!(Boolean, vec![true, true, true, false, true, true]); diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index ff9104981989..5a0e5b0c974a 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -36,7 +36,7 @@ use datafusion_common::error::Result; use datafusion_common::exec_datafusion_err; use datafusion_expr::{AggregateUDF, AggregateUDFImpl, Signature}; use datafusion_proto_common::from_proto::parse_proto_fields_to_fields; -use groups_accumulator::{FFI_GroupsAccumulator, ForeignGroupsAccumulator}; +use groups_accumulator::FFI_GroupsAccumulator; use std::hash::{Hash, Hasher}; use std::{ffi::c_void, sync::Arc}; @@ -520,12 +520,8 @@ impl AggregateUDFImpl for ForeignAggregateUDF { let args = FFI_AccumulatorArgs::try_from(args)?; unsafe { - df_result!((self.udaf.create_groups_accumulator)(&self.udaf, args)).map( - |accum| { - Box::new(ForeignGroupsAccumulator::from(accum)) - as Box - }, - ) + df_result!((self.udaf.create_groups_accumulator)(&self.udaf, args)) + .map(>::from) } } From 4b1ba3f06a54a47cc525ea5977f33cb76f2cb259 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Thu, 6 Nov 2025 23:22:52 -0500 Subject: [PATCH 29/69] switch UDAF to use library marker --- datafusion/ffi/src/function_registry.rs | 8 ++--- datafusion/ffi/src/session/mod.rs | 10 ++++--- datafusion/ffi/src/session/task.rs | 8 ++--- datafusion/ffi/src/udaf/mod.rs | 39 +++++++++++++++---------- datafusion/ffi/tests/ffi_udaf.rs | 13 +++++---- 5 files changed, 44 insertions(+), 34 deletions(-) diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 04f87b158c13..80679a8bf33c 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; +use crate::udaf::FFI_AggregateUDF; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use crate::{df_result, rresult_return}; @@ -27,7 +27,7 @@ use datafusion_common::{exec_datafusion_err, not_impl_err, DataFusionError}; use datafusion_expr::expr_rewriter::FunctionRewrite; use datafusion_expr::planner::ExprPlanner; use datafusion_expr::registry::FunctionRegistry; -use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use datafusion_expr::{AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF}; use log::warn; use std::collections::HashSet; use std::sync::Weak; @@ -266,8 +266,8 @@ impl FunctionRegistry for ForeignWeakFunctionRegistry { fn udaf(&self, name: &str) -> datafusion_common::Result> { let udaf = df_result!(unsafe { (self.0.udaf)(&self.0, name.into()) })?; - let udaf = ForeignAggregateUDF::try_from(&udaf)?; - Ok(Arc::new(udaf.into())) + let udaf = >::try_from(&udaf)?; + Ok(Arc::new(AggregateUDF::new_from_shared_impl(udaf))) } fn udwf(&self, name: &str) -> datafusion_common::Result> { diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 409fc635932c..09ac8cf41a16 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -20,7 +20,7 @@ use crate::execution_plan::FFI_ExecutionPlan; use crate::function_registry::FFI_WeakFunctionRegistry; use crate::session::config::FFI_SessionConfig; use crate::session::task::FFI_TaskContext; -use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; +use crate::udaf::FFI_AggregateUDF; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use crate::{df_result, rresult, rresult_return}; @@ -41,7 +41,9 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::registry::FunctionRegistry; -use datafusion_expr::{AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF}; +use datafusion_expr::{ + AggregateUDF, AggregateUDFImpl, Expr, LogicalPlan, ScalarUDF, WindowUDF, +}; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::ExecutionPlan; use datafusion_proto::bytes::{logical_plan_from_bytes, logical_plan_to_bytes}; @@ -397,11 +399,11 @@ impl TryFrom<&FFI_Session> for ForeignSession { let aggregate_functions = (session.aggregate_functions)(session) .into_iter() .map(|kv_pair| { - let udaf = ForeignAggregateUDF::try_from(&kv_pair.1)?; + let udaf = >::try_from(&kv_pair.1)?; Ok(( kv_pair.0.into_string(), - Arc::new(AggregateUDF::new_from_impl(udaf)), + Arc::new(AggregateUDF::new_from_shared_impl(udaf)), )) }) .collect::>()?; diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index edab56ba7175..e7ee035b84b4 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -16,7 +16,7 @@ // under the License. use crate::session::config::FFI_SessionConfig; -use crate::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; +use crate::udaf::FFI_AggregateUDF; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use abi_stable::pmr::ROption; @@ -25,7 +25,7 @@ use abi_stable::{std_types::RString, StableAbi}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use datafusion_expr::{AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF}; use std::{ffi::c_void, sync::Arc}; /// A stable struct for sharing [`TaskContext`] across FFI boundaries. @@ -169,7 +169,7 @@ impl From for TaskContext { let aggregate_functions = (ffi_ctx.aggregate_functions)(&ffi_ctx) .into_iter() .filter_map(|kv_pair| { - let udaf = ForeignAggregateUDF::try_from(&kv_pair.1); + let udaf = >::try_from(&kv_pair.1); if let Err(err) = &udaf { log::error!("Unable to create AggregateUDF in FFI: {err}") @@ -178,7 +178,7 @@ impl From for TaskContext { udaf.ok().map(|udaf| { ( kv_pair.0.into_string(), - Arc::new(AggregateUDF::new_from_impl(udaf)), + Arc::new(AggregateUDF::new_from_shared_impl(udaf)), ) }) }) diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index 5a0e5b0c974a..774a35bf3b16 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -142,6 +142,10 @@ pub struct FFI_AggregateUDF { /// Internal data. This is only to be accessed by the provider of the udaf. /// A [`ForeignAggregateUDF`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_AggregateUDF {} @@ -152,9 +156,9 @@ pub struct AggregateUDFPrivateData { } impl FFI_AggregateUDF { - unsafe fn inner(&self) -> &Arc { + fn inner(&self) -> &Arc { let private_data = self.private_data as *const AggregateUDFPrivateData; - &(*private_data).udaf + unsafe { &(*private_data).udaf } } } @@ -364,6 +368,7 @@ impl From> for FFI_AggregateUDF { clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -403,18 +408,22 @@ impl Hash for ForeignAggregateUDF { } } -impl TryFrom<&FFI_AggregateUDF> for ForeignAggregateUDF { +impl TryFrom<&FFI_AggregateUDF> for Arc { type Error = DataFusionError; fn try_from(udaf: &FFI_AggregateUDF) -> Result { + if (udaf.library_marker_id)() == crate::get_library_marker_id() { + return Ok(Arc::clone(udaf.inner().inner())); + } + let signature = Signature::user_defined((&udaf.volatility).into()); let aliases = udaf.aliases.iter().map(|s| s.to_string()).collect(); - Ok(Self { + Ok(Arc::new(ForeignAggregateUDF { udaf: udaf.clone(), signature, aliases, - }) + })) } } @@ -552,11 +561,9 @@ impl AggregateUDFImpl for ForeignAggregateUDF { ))? .into_option(); - let result = result - .map(|func| ForeignAggregateUDF::try_from(&func)) - .transpose()?; - - Ok(result.map(|func| Arc::new(func) as Arc)) + result + .map(|func| >::try_from(&func)) + .transpose() } } @@ -662,8 +669,8 @@ mod tests { let local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); - let foreign_udaf: ForeignAggregateUDF = (&local_udaf).try_into()?; - Ok(foreign_udaf.into()) + let foreign_udaf: Arc = (&local_udaf).try_into()?; + Ok(AggregateUDF::new_from_shared_impl(foreign_udaf)) } #[test] @@ -676,8 +683,8 @@ mod tests { let local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); // Convert back to native format - let foreign_udaf: ForeignAggregateUDF = (&local_udaf).try_into()?; - let foreign_udaf: AggregateUDF = foreign_udaf.into(); + let foreign_udaf: Arc = (&local_udaf).try_into()?; + let foreign_udaf = AggregateUDF::new_from_shared_impl(foreign_udaf); assert_eq!(original_name, foreign_udaf.name()); Ok(()) @@ -730,8 +737,8 @@ mod tests { let local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); // Convert back to native format - let foreign_udaf: ForeignAggregateUDF = (&local_udaf).try_into()?; - let foreign_udaf: AggregateUDF = foreign_udaf.into(); + let foreign_udaf: Arc = (&local_udaf).try_into()?; + let foreign_udaf = AggregateUDF::new_from_shared_impl(foreign_udaf); let metadata: HashMap = [("a_key".to_string(), "a_value".to_string())] diff --git a/datafusion/ffi/tests/ffi_udaf.rs b/datafusion/ffi/tests/ffi_udaf.rs index ffd99bac62ec..17db7a64561a 100644 --- a/datafusion/ffi/tests/ffi_udaf.rs +++ b/datafusion/ffi/tests/ffi_udaf.rs @@ -24,9 +24,9 @@ mod tests { use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::AggregateUDF; use datafusion::prelude::{col, SessionContext}; - + use datafusion_expr::AggregateUDFImpl; use datafusion_ffi::tests::utils::get_module; - use datafusion_ffi::udaf::ForeignAggregateUDF; + use std::sync::Arc; #[tokio::test] async fn test_ffi_udaf() -> Result<()> { @@ -38,9 +38,9 @@ mod tests { .ok_or(DataFusionError::NotImplemented( "External table provider failed to implement create_udaf".to_string(), ))?(); - let foreign_sum_func: ForeignAggregateUDF = (&ffi_sum_func).try_into()?; + let foreign_sum_func: Arc = (&ffi_sum_func).try_into()?; - let udaf: AggregateUDF = foreign_sum_func.into(); + let udaf = AggregateUDF::new_from_shared_impl(foreign_sum_func); let ctx = SessionContext::default(); let record_batch = record_batch!( @@ -80,9 +80,10 @@ mod tests { .ok_or(DataFusionError::NotImplemented( "External table provider failed to implement create_udaf".to_string(), ))?(); - let foreign_stddev_func: ForeignAggregateUDF = (&ffi_stddev_func).try_into()?; + let foreign_stddev_func: Arc = + (&ffi_stddev_func).try_into()?; - let udaf: AggregateUDF = foreign_stddev_func.into(); + let udaf = AggregateUDF::new_from_shared_impl(foreign_stddev_func); let ctx = SessionContext::default(); let record_batch = record_batch!( From 78e65c5717a32862d6604a9f9191946c76dab627 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 7 Nov 2025 06:52:02 -0500 Subject: [PATCH 30/69] switch accumulator to use library marker --- datafusion/ffi/src/udaf/accumulator.rs | 25 +++++++++++++++++++------ datafusion/ffi/src/udaf/mod.rs | 12 +++++------- 2 files changed, 24 insertions(+), 13 deletions(-) diff --git a/datafusion/ffi/src/udaf/accumulator.rs b/datafusion/ffi/src/udaf/accumulator.rs index f7efa88dc90d..9196c90cefa2 100644 --- a/datafusion/ffi/src/udaf/accumulator.rs +++ b/datafusion/ffi/src/udaf/accumulator.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ffi::c_void, ops::Deref}; - use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, @@ -28,6 +26,7 @@ use datafusion_common::{ }; use datafusion_expr::Accumulator; use prost::Message; +use std::{ffi::c_void, ops::Deref}; use crate::{arrow_wrappers::WrappedArray, df_result, rresult, rresult_return}; @@ -70,6 +69,10 @@ pub struct FFI_Accumulator { /// Internal data. This is only to be accessed by the provider of the accumulator. /// A [`ForeignAccumulator`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_Accumulator {} @@ -193,6 +196,7 @@ impl From> for FFI_Accumulator { supports_retract_batch, release: release_fn_wrapper, private_data: Box::into_raw(Box::new(private_data)) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -217,9 +221,18 @@ pub struct ForeignAccumulator { unsafe impl Send for ForeignAccumulator {} unsafe impl Sync for ForeignAccumulator {} -impl From for ForeignAccumulator { +impl From for Box { fn from(accumulator: FFI_Accumulator) -> Self { - Self { accumulator } + if (accumulator.library_marker_id)() == crate::get_library_marker_id() { + unsafe { + let private_data = Box::from_raw( + accumulator.private_data as *mut AccumulatorPrivateData, + ); + private_data.accumulator + } + } else { + Box::new(ForeignAccumulator { accumulator }) + } } } @@ -313,7 +326,7 @@ mod tests { scalar::ScalarValue, }; - use super::{FFI_Accumulator, ForeignAccumulator}; + use super::FFI_Accumulator; #[test] fn test_foreign_avg_accumulator() -> Result<()> { @@ -323,7 +336,7 @@ mod tests { let boxed_accum: Box = Box::new(original_accum); let ffi_accum: FFI_Accumulator = boxed_accum.into(); - let mut foreign_accum: ForeignAccumulator = ffi_accum.into(); + let mut foreign_accum: Box = ffi_accum.into(); // Send in an array to average. There are 5 values and it should average to 30.0 let values = create_array!(Float64, vec![10., 20., 30., 40., 50.]); diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index 774a35bf3b16..30482ed467a7 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -19,7 +19,7 @@ use abi_stable::{ std_types::{ROption, RResult, RStr, RString, RVec}, StableAbi, }; -use accumulator::{FFI_Accumulator, ForeignAccumulator}; +use accumulator::FFI_Accumulator; use accumulator_args::{FFI_AccumulatorArgs, ForeignAccumulatorArgs}; use arrow::datatypes::{DataType, Field}; use arrow::ffi::FFI_ArrowSchema; @@ -465,9 +465,8 @@ impl AggregateUDFImpl for ForeignAggregateUDF { fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { let args = acc_args.try_into()?; unsafe { - df_result!((self.udaf.accumulator)(&self.udaf, args)).map(|accum| { - Box::new(ForeignAccumulator::from(accum)) as Box - }) + df_result!((self.udaf.accumulator)(&self.udaf, args)) + .map(>::from) } } @@ -544,9 +543,8 @@ impl AggregateUDFImpl for ForeignAggregateUDF { ) -> Result> { let args = args.try_into()?; unsafe { - df_result!((self.udaf.create_sliding_accumulator)(&self.udaf, args)).map( - |accum| Box::new(ForeignAccumulator::from(accum)) as Box, - ) + df_result!((self.udaf.create_sliding_accumulator)(&self.udaf, args)) + .map(>::from) } } From 1a09d919458e8fe1d5919c0c84140fb6fc58f018 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 7 Nov 2025 08:00:30 -0500 Subject: [PATCH 31/69] switch window evaluator to use library marker --- datafusion/ffi/src/udwf/mod.rs | 7 ++--- .../ffi/src/udwf/partition_evaluator.rs | 29 ++++++++++++++----- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index 7e53a4e9cbe0..6ec8330a8586 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -33,7 +33,7 @@ use datafusion_expr::{ LimitEffect, PartitionEvaluator, Signature, WindowUDF, WindowUDFImpl, }; use datafusion_physical_expr::PhysicalExpr; -use partition_evaluator::{FFI_PartitionEvaluator, ForeignPartitionEvaluator}; +use partition_evaluator::FFI_PartitionEvaluator; use partition_evaluator_args::{ FFI_PartitionEvaluatorArgs, ForeignPartitionEvaluatorArgs, }; @@ -322,10 +322,7 @@ impl WindowUDFImpl for ForeignWindowUDF { (self.udf.partition_evaluator)(&self.udf, args) }; - df_result!(evaluator).map(|evaluator| { - Box::new(ForeignPartitionEvaluator::from(evaluator)) - as Box - }) + df_result!(evaluator).map(>::from) } fn field(&self, field_args: WindowUDFFieldArgs) -> Result { diff --git a/datafusion/ffi/src/udwf/partition_evaluator.rs b/datafusion/ffi/src/udwf/partition_evaluator.rs index b8cdb61e8ea6..b5c4b42baa95 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ffi::c_void, ops::Range}; - use crate::{arrow_wrappers::WrappedArray, df_result, rresult, rresult_return}; use abi_stable::{ std_types::{RResult, RString, RVec}, @@ -29,6 +27,7 @@ use datafusion_common::{ }; use datafusion_expr::{window_state::WindowAggState, PartitionEvaluator}; use prost::Message; +use std::{ffi::c_void, ops::Range}; use super::range::FFI_Range; @@ -76,6 +75,10 @@ pub struct FFI_PartitionEvaluator { /// Internal data. This is only to be accessed by the provider of the evaluator. /// A [`ForeignPartitionEvaluator`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_PartitionEvaluator {} @@ -86,14 +89,14 @@ pub struct PartitionEvaluatorPrivateData { } impl FFI_PartitionEvaluator { - unsafe fn inner_mut(&mut self) -> &mut Box { + fn inner_mut(&mut self) -> &mut Box { let private_data = self.private_data as *mut PartitionEvaluatorPrivateData; - &mut (*private_data).evaluator + unsafe { &mut (*private_data).evaluator } } - unsafe fn inner(&self) -> &(dyn PartitionEvaluator + 'static) { + fn inner(&self) -> &(dyn PartitionEvaluator + 'static) { let private_data = self.private_data as *mut PartitionEvaluatorPrivateData; - (*private_data).evaluator.as_ref() + unsafe { (*private_data).evaluator.as_ref() } } } @@ -195,6 +198,7 @@ impl From> for FFI_PartitionEvaluator { uses_window_frame, release: release_fn_wrapper, private_data: Box::into_raw(Box::new(private_data)) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -219,9 +223,18 @@ pub struct ForeignPartitionEvaluator { unsafe impl Send for ForeignPartitionEvaluator {} unsafe impl Sync for ForeignPartitionEvaluator {} -impl From for ForeignPartitionEvaluator { +impl From for Box { fn from(evaluator: FFI_PartitionEvaluator) -> Self { - Self { evaluator } + if (evaluator.library_marker_id)() == crate::get_library_marker_id() { + unsafe { + let private_data = Box::from_raw( + evaluator.private_data as *mut PartitionEvaluatorPrivateData, + ); + private_data.evaluator + } + } else { + Box::new(ForeignPartitionEvaluator { evaluator }) + } } } From ce1bb9613d7df26c810796ea52102ede76867455 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 7 Nov 2025 08:25:38 -0500 Subject: [PATCH 32/69] switch udwf to use library marker --- datafusion/ffi/src/function_registry.rs | 10 +++--- datafusion/ffi/src/session/mod.rs | 7 ++-- datafusion/ffi/src/session/task.rs | 10 +++--- datafusion/ffi/src/udwf/mod.rs | 48 +++++++++++++++---------- datafusion/ffi/tests/ffi_udwf.rs | 7 ++-- 5 files changed, 49 insertions(+), 33 deletions(-) diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 80679a8bf33c..34890220e613 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -17,7 +17,7 @@ use crate::udaf::FFI_AggregateUDF; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; -use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; +use crate::udwf::FFI_WindowUDF; use crate::{df_result, rresult_return}; use abi_stable::{ std_types::{RResult, RString, RVec}, @@ -27,7 +27,9 @@ use datafusion_common::{exec_datafusion_err, not_impl_err, DataFusionError}; use datafusion_expr::expr_rewriter::FunctionRewrite; use datafusion_expr::planner::ExprPlanner; use datafusion_expr::registry::FunctionRegistry; -use datafusion_expr::{AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF}; +use datafusion_expr::{ + AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF, WindowUDFImpl, +}; use log::warn; use std::collections::HashSet; use std::sync::Weak; @@ -273,8 +275,8 @@ impl FunctionRegistry for ForeignWeakFunctionRegistry { fn udwf(&self, name: &str) -> datafusion_common::Result> { let udwf = df_result!(unsafe { (self.0.udwf)(&self.0, name.into()) })?; - let udwf = ForeignWindowUDF::try_from(&udwf)?; - Ok(Arc::new(udwf.into())) + let udwf = >::try_from(&udwf)?; + Ok(Arc::new(WindowUDF::new_from_shared_impl(udwf))) } fn register_udf( diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 09ac8cf41a16..715808f0266d 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -22,7 +22,7 @@ use crate::session::config::FFI_SessionConfig; use crate::session::task::FFI_TaskContext; use crate::udaf::FFI_AggregateUDF; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; -use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; +use crate::udwf::FFI_WindowUDF; use crate::{df_result, rresult, rresult_return}; use abi_stable::std_types::{RHashMap, RStr}; use abi_stable::{ @@ -43,6 +43,7 @@ use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{ AggregateUDF, AggregateUDFImpl, Expr, LogicalPlan, ScalarUDF, WindowUDF, + WindowUDFImpl, }; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::ExecutionPlan; @@ -410,11 +411,11 @@ impl TryFrom<&FFI_Session> for ForeignSession { let window_functions = (session.window_functions)(session) .into_iter() .map(|kv_pair| { - let udwf = ForeignWindowUDF::try_from(&kv_pair.1)?; + let udwf = >::try_from(&kv_pair.1)?; Ok(( kv_pair.0.into_string(), - Arc::new(WindowUDF::new_from_impl(udwf)), + Arc::new(WindowUDF::new_from_shared_impl(udwf)), )) }) .collect::>()?; diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index e7ee035b84b4..5e620a9bf202 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -18,14 +18,16 @@ use crate::session::config::FFI_SessionConfig; use crate::udaf::FFI_AggregateUDF; use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; -use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; +use crate::udwf::FFI_WindowUDF; use abi_stable::pmr::ROption; use abi_stable::std_types::RHashMap; use abi_stable::{std_types::RString, StableAbi}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_expr::{AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF}; +use datafusion_expr::{ + AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF, WindowUDFImpl, +}; use std::{ffi::c_void, sync::Arc}; /// A stable struct for sharing [`TaskContext`] across FFI boundaries. @@ -186,7 +188,7 @@ impl From for TaskContext { let window_functions = (ffi_ctx.window_functions)(&ffi_ctx) .into_iter() .filter_map(|kv_pair| { - let udwf = ForeignWindowUDF::try_from(&kv_pair.1); + let udwf = >::try_from(&kv_pair.1); if let Err(err) = &udwf { log::error!("Unable to create WindowUDF in FFI: {err}") @@ -195,7 +197,7 @@ impl From for TaskContext { udwf.ok().map(|udwf| { ( kv_pair.0.into_string(), - Arc::new(WindowUDF::new_from_impl(udwf)), + Arc::new(WindowUDF::new_from_shared_impl(udwf)), ) }) }) diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index 6ec8330a8586..f7ff2aa4ac28 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -99,6 +99,10 @@ pub struct FFI_WindowUDF { /// Internal data. This is only to be accessed by the provider of the udf. /// A [`ForeignWindowUDF`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_WindowUDF {} @@ -109,9 +113,9 @@ pub struct WindowUDFPrivateData { } impl FFI_WindowUDF { - unsafe fn inner(&self) -> &Arc { + fn inner(&self) -> &Arc { let private_data = self.private_data as *const WindowUDFPrivateData; - &(*private_data).udf + unsafe { &(*private_data).udf } } } @@ -195,6 +199,7 @@ unsafe extern "C" fn clone_fn_wrapper(udwf: &FFI_WindowUDF) -> FFI_WindowUDF { clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } @@ -230,6 +235,7 @@ impl From> for FFI_WindowUDF { clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -270,21 +276,25 @@ impl Hash for ForeignWindowUDF { } } -impl TryFrom<&FFI_WindowUDF> for ForeignWindowUDF { +impl TryFrom<&FFI_WindowUDF> for Arc { type Error = DataFusionError; fn try_from(udf: &FFI_WindowUDF) -> Result { - let name = udf.name.to_owned().into(); - let signature = Signature::user_defined((&udf.volatility).into()); - - let aliases = udf.aliases.iter().map(|s| s.to_string()).collect(); - - Ok(Self { - name, - udf: udf.clone(), - aliases, - signature, - }) + if (udf.library_marker_id)() == crate::get_library_marker_id() { + Ok(Arc::clone(udf.inner().inner())) + } else { + let name = udf.name.to_owned().into(); + let signature = Signature::user_defined((&udf.volatility).into()); + + let aliases = udf.aliases.iter().map(|s| s.to_string()).collect(); + + Ok(Arc::new(ForeignWindowUDF { + name, + udf: udf.clone(), + aliases, + signature, + })) + } } } @@ -384,7 +394,7 @@ impl From<&FFI_SortOptions> for SortOptions { #[cfg(feature = "integration-tests")] mod tests { use crate::tests::create_record_batch; - use crate::udwf::{FFI_WindowUDF, ForeignWindowUDF}; + use crate::udwf::FFI_WindowUDF; use arrow::array::{create_array, ArrayRef}; use datafusion::functions_window::lead_lag::{lag_udwf, WindowShift}; use datafusion::logical_expr::expr::Sort; @@ -399,8 +409,8 @@ mod tests { let local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); - let foreign_udwf: ForeignWindowUDF = (&local_udwf).try_into()?; - Ok(foreign_udwf.into()) + let foreign_udwf: Arc = (&local_udwf).try_into()?; + Ok(WindowUDF::new_from_shared_impl(foreign_udwf)) } #[test] @@ -412,8 +422,8 @@ mod tests { let local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); // Convert back to native format - let foreign_udwf: ForeignWindowUDF = (&local_udwf).try_into()?; - let foreign_udwf: WindowUDF = foreign_udwf.into(); + let foreign_udwf: Arc = (&local_udwf).try_into()?; + let foreign_udwf = WindowUDF::new_from_shared_impl(foreign_udwf); assert_eq!(original_name, foreign_udwf.name()); Ok(()) diff --git a/datafusion/ffi/tests/ffi_udwf.rs b/datafusion/ffi/tests/ffi_udwf.rs index 18ffd0c5bcb7..d663a17982ee 100644 --- a/datafusion/ffi/tests/ffi_udwf.rs +++ b/datafusion/ffi/tests/ffi_udwf.rs @@ -24,9 +24,10 @@ mod tests { use datafusion::logical_expr::expr::Sort; use datafusion::logical_expr::{col, ExprFunctionExt, WindowUDF}; use datafusion::prelude::SessionContext; + use datafusion_expr::WindowUDFImpl; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; - use datafusion_ffi::udwf::ForeignWindowUDF; + use std::sync::Arc; #[tokio::test] async fn test_rank_udwf() -> Result<()> { @@ -39,9 +40,9 @@ mod tests { "External table provider failed to implement create_scalar_udf" .to_string(), ))?(); - let foreign_rank_func: ForeignWindowUDF = (&ffi_rank_func).try_into()?; + let foreign_rank_func: Arc = (&ffi_rank_func).try_into()?; - let udwf: WindowUDF = foreign_rank_func.into(); + let udwf = WindowUDF::new_from_shared_impl(foreign_rank_func); let ctx = SessionContext::default(); let df = ctx.read_batch(create_record_batch(-5, 5))?; From 11b3161ecaff0dd81bd0f1e122bd03bfc77566c1 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 7 Nov 2025 08:36:31 -0500 Subject: [PATCH 33/69] switch scalar udf to use library marker --- datafusion/ffi/src/function_registry.rs | 8 ++-- datafusion/ffi/src/session/mod.rs | 10 ++--- datafusion/ffi/src/session/task.rs | 8 ++-- datafusion/ffi/src/udf/mod.rs | 59 +++++++++++++++---------- datafusion/ffi/tests/ffi_udf.rs | 13 +++--- 5 files changed, 54 insertions(+), 44 deletions(-) diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 34890220e613..4ff1b621a9ed 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -16,7 +16,7 @@ // under the License. use crate::udaf::FFI_AggregateUDF; -use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; +use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; use crate::{df_result, rresult_return}; use abi_stable::{ @@ -28,7 +28,7 @@ use datafusion_expr::expr_rewriter::FunctionRewrite; use datafusion_expr::planner::ExprPlanner; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{ - AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF, WindowUDFImpl, + AggregateUDF, AggregateUDFImpl, ScalarUDF, ScalarUDFImpl, WindowUDF, WindowUDFImpl, }; use log::warn; use std::collections::HashSet; @@ -261,8 +261,8 @@ impl FunctionRegistry for ForeignWeakFunctionRegistry { fn udf(&self, name: &str) -> datafusion_common::Result> { let udf = df_result!(unsafe { (self.0.udf)(&self.0, name.into()) })?; - let udf = ForeignScalarUDF::try_from(&udf)?; - Ok(Arc::new(udf.into())) + let udf = >::try_from(&udf)?; + Ok(Arc::new(ScalarUDF::new_from_shared_impl(udf))) } fn udaf(&self, name: &str) -> datafusion_common::Result> { diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 715808f0266d..99aeaa5de069 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -21,7 +21,7 @@ use crate::function_registry::FFI_WeakFunctionRegistry; use crate::session::config::FFI_SessionConfig; use crate::session::task::FFI_TaskContext; use crate::udaf::FFI_AggregateUDF; -use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; +use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; use crate::{df_result, rresult, rresult_return}; use abi_stable::std_types::{RHashMap, RStr}; @@ -42,8 +42,8 @@ use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{ - AggregateUDF, AggregateUDFImpl, Expr, LogicalPlan, ScalarUDF, WindowUDF, - WindowUDFImpl, + AggregateUDF, AggregateUDFImpl, Expr, LogicalPlan, ScalarUDF, ScalarUDFImpl, + WindowUDF, WindowUDFImpl, }; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::ExecutionPlan; @@ -389,11 +389,11 @@ impl TryFrom<&FFI_Session> for ForeignSession { let scalar_functions = (session.scalar_functions)(session) .into_iter() .map(|kv_pair| { - let udf = ForeignScalarUDF::try_from(&kv_pair.1)?; + let udf = >::try_from(&kv_pair.1)?; Ok(( kv_pair.0.into_string(), - Arc::new(ScalarUDF::new_from_impl(udf)), + Arc::new(ScalarUDF::new_from_shared_impl(udf)), )) }) .collect::>()?; diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index 5e620a9bf202..b306d97470ec 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -17,7 +17,7 @@ use crate::session::config::FFI_SessionConfig; use crate::udaf::FFI_AggregateUDF; -use crate::udf::{FFI_ScalarUDF, ForeignScalarUDF}; +use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; use abi_stable::pmr::ROption; use abi_stable::std_types::RHashMap; @@ -26,7 +26,7 @@ use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{ - AggregateUDF, AggregateUDFImpl, ScalarUDF, WindowUDF, WindowUDFImpl, + AggregateUDF, AggregateUDFImpl, ScalarUDF, ScalarUDFImpl, WindowUDF, WindowUDFImpl, }; use std::{ffi::c_void, sync::Arc}; @@ -154,7 +154,7 @@ impl From for TaskContext { let scalar_functions = (ffi_ctx.scalar_functions)(&ffi_ctx) .into_iter() .filter_map(|kv_pair| { - let udf = ForeignScalarUDF::try_from(&kv_pair.1); + let udf = >::try_from(&kv_pair.1); if let Err(err) = &udf { log::error!("Unable to create WindowUDF in FFI: {err}") @@ -163,7 +163,7 @@ impl From for TaskContext { udf.ok().map(|udf| { ( kv_pair.0.into_string(), - Arc::new(ScalarUDF::new_from_impl(udf)), + Arc::new(ScalarUDF::new_from_shared_impl(udf)), ) }) }) diff --git a/datafusion/ffi/src/udf/mod.rs b/datafusion/ffi/src/udf/mod.rs index a834066041f9..0f34400c77db 100644 --- a/datafusion/ffi/src/udf/mod.rs +++ b/datafusion/ffi/src/udf/mod.rs @@ -108,6 +108,10 @@ pub struct FFI_ScalarUDF { /// Internal data. This is only to be accessed by the provider of the udf. /// A [`ForeignScalarUDF`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_ScalarUDF {} @@ -117,12 +121,18 @@ pub struct ScalarUDFPrivateData { pub udf: Arc, } +impl FFI_ScalarUDF { + fn inner(&self) -> &Arc { + let private_data = self.private_data as *const ScalarUDFPrivateData; + unsafe { &(*private_data).udf } + } +} + unsafe extern "C" fn return_type_fn_wrapper( udf: &FFI_ScalarUDF, arg_types: RVec, ) -> RResult { - let private_data = udf.private_data as *const ScalarUDFPrivateData; - let udf = &(*private_data).udf; + let udf = udf.inner(); let arg_types = rresult_return!(rvec_wrapped_to_vec_datatype(&arg_types)); @@ -138,8 +148,7 @@ unsafe extern "C" fn return_field_from_args_fn_wrapper( udf: &FFI_ScalarUDF, args: FFI_ReturnFieldArgs, ) -> RResult { - let private_data = udf.private_data as *const ScalarUDFPrivateData; - let udf = &(*private_data).udf; + let udf = udf.inner(); let args: ForeignReturnFieldArgsOwned = rresult_return!((&args).try_into()); let args_ref: ForeignReturnFieldArgs = (&args).into(); @@ -156,8 +165,7 @@ unsafe extern "C" fn coerce_types_fn_wrapper( udf: &FFI_ScalarUDF, arg_types: RVec, ) -> RResult, RString> { - let private_data = udf.private_data as *const ScalarUDFPrivateData; - let udf = &(*private_data).udf; + let udf = udf.inner(); let arg_types = rresult_return!(rvec_wrapped_to_vec_datatype(&arg_types)); @@ -173,8 +181,7 @@ unsafe extern "C" fn invoke_with_args_fn_wrapper( number_rows: usize, return_field: WrappedSchema, ) -> RResult { - let private_data = udf.private_data as *const ScalarUDFPrivateData; - let udf = &(*private_data).udf; + let udf = udf.inner(); let args = args .into_iter() @@ -224,10 +231,9 @@ unsafe extern "C" fn release_fn_wrapper(udf: &mut FFI_ScalarUDF) { } unsafe extern "C" fn clone_fn_wrapper(udf: &FFI_ScalarUDF) -> FFI_ScalarUDF { - let private_data = udf.private_data as *const ScalarUDFPrivateData; - let udf_data = &(*private_data); + let udf = udf.inner(); - Arc::clone(&udf_data.udf).into() + Arc::clone(udf).into() } impl Clone for FFI_ScalarUDF { @@ -263,6 +269,7 @@ impl From> for FFI_ScalarUDF { clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -321,21 +328,25 @@ impl Hash for ForeignScalarUDF { } } -impl TryFrom<&FFI_ScalarUDF> for ForeignScalarUDF { +impl TryFrom<&FFI_ScalarUDF> for Arc { type Error = DataFusionError; fn try_from(udf: &FFI_ScalarUDF) -> Result { - let name = udf.name.to_owned().into(); - let signature = Signature::user_defined((&udf.volatility).into()); - - let aliases = udf.aliases.iter().map(|s| s.to_string()).collect(); - - Ok(Self { - name, - udf: udf.clone(), - aliases, - signature, - }) + if (udf.library_marker_id)() == crate::get_library_marker_id() { + Ok(Arc::clone(udf.inner().inner())) + } else { + let name = udf.name.to_owned().into(); + let signature = Signature::user_defined((&udf.volatility).into()); + + let aliases = udf.aliases.iter().map(|s| s.to_string()).collect(); + + Ok(Arc::new(ForeignScalarUDF { + name, + udf: udf.clone(), + aliases, + signature, + })) + } } } @@ -457,7 +468,7 @@ mod tests { let local_udf: FFI_ScalarUDF = Arc::clone(&original_udf).into(); - let foreign_udf: ForeignScalarUDF = (&local_udf).try_into()?; + let foreign_udf: Arc = (&local_udf).try_into()?; assert_eq!(original_udf.name(), foreign_udf.name()); diff --git a/datafusion/ffi/tests/ffi_udf.rs b/datafusion/ffi/tests/ffi_udf.rs index fd6a84bcf5b0..0bf770cd9b93 100644 --- a/datafusion/ffi/tests/ffi_udf.rs +++ b/datafusion/ffi/tests/ffi_udf.rs @@ -19,16 +19,15 @@ /// when the feature integration-tests is built #[cfg(feature = "integration-tests")] mod tests { - use arrow::datatypes::DataType; use datafusion::common::record_batch; use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::ScalarUDF; use datafusion::prelude::{col, SessionContext}; - + use datafusion_expr::ScalarUDFImpl; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; - use datafusion_ffi::udf::ForeignScalarUDF; + use std::sync::Arc; /// This test validates that we can load an external module and use a scalar /// udf defined in it via the foreign function interface. In this case we are @@ -44,9 +43,9 @@ mod tests { "External table provider failed to implement create_scalar_udf" .to_string(), ))?(); - let foreign_abs_func: ForeignScalarUDF = (&ffi_abs_func).try_into()?; + let foreign_abs_func: Arc = (&ffi_abs_func).try_into()?; - let udf: ScalarUDF = foreign_abs_func.into(); + let udf = ScalarUDF::new_from_shared_impl(foreign_abs_func); let ctx = SessionContext::default(); let df = ctx.read_batch(create_record_batch(-5, 5))?; @@ -82,9 +81,9 @@ mod tests { "External table provider failed to implement create_scalar_udf" .to_string(), ))?(); - let foreign_abs_func: ForeignScalarUDF = (&ffi_abs_func).try_into()?; + let foreign_abs_func: Arc = (&ffi_abs_func).try_into()?; - let udf: ScalarUDF = foreign_abs_func.into(); + let udf = ScalarUDF::new_from_shared_impl(foreign_abs_func); let ctx = SessionContext::default(); let df = ctx.read_batch(create_record_batch(-5, 5))?; From 18c241a55908bd28aa58aa3b0c5c5c53a1aa8d37 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 7 Nov 2025 08:44:25 -0500 Subject: [PATCH 34/69] switch table function to use library marker --- datafusion/ffi/src/udtf.rs | 31 ++++++++++++------------------- datafusion/ffi/tests/ffi_udtf.rs | 8 +++----- 2 files changed, 15 insertions(+), 24 deletions(-) diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index 3924c1fb6f8f..45f8faedd1a1 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -61,6 +61,10 @@ pub struct FFI_TableFunction { /// Internal data. This is only to be accessed by the provider of the udtf. /// A [`ForeignTableFunction`] should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_TableFunction {} @@ -147,26 +151,11 @@ impl FFI_TableFunction { clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } -// impl From> for FFI_TableFunction { -// fn from(udtf: Arc) -> Self { -// let private_data = Box::new(TableFunctionPrivateData { -// udtf, -// runtime: None, -// }); -// -// Self { -// call: call_fn_wrapper, -// clone: clone_fn_wrapper, -// release: release_fn_wrapper, -// private_data: Box::into_raw(private_data) as *mut c_void, -// } -// } -// } - impl Drop for FFI_TableFunction { fn drop(&mut self) { unsafe { (self.release)(self) } @@ -185,9 +174,13 @@ pub struct ForeignTableFunction(FFI_TableFunction); unsafe impl Send for ForeignTableFunction {} unsafe impl Sync for ForeignTableFunction {} -impl From for ForeignTableFunction { +impl From for Arc { fn from(value: FFI_TableFunction) -> Self { - Self(value) + if (value.library_marker_id)() == crate::get_library_marker_id() { + Arc::clone(value.inner()) + } else { + Arc::new(ForeignTableFunction(value)) + } } } @@ -313,7 +306,7 @@ mod tests { function_registry.into(), ); - let foreign_udf: ForeignTableFunction = local_udtf.into(); + let foreign_udf: Arc = local_udtf.into(); let table = foreign_udf.call(&[lit(6_u64), lit("one"), lit(2.0), lit(3_u64)])?; diff --git a/datafusion/ffi/tests/ffi_udtf.rs b/datafusion/ffi/tests/ffi_udtf.rs index a661959d227c..205934650aa4 100644 --- a/datafusion/ffi/tests/ffi_udtf.rs +++ b/datafusion/ffi/tests/ffi_udtf.rs @@ -25,9 +25,9 @@ mod tests { use arrow::array::{create_array, ArrayRef}; use datafusion::error::{DataFusionError, Result}; use datafusion::prelude::SessionContext; + use datafusion_catalog::TableFunctionImpl; use datafusion_expr::registry::FunctionRegistry; use datafusion_ffi::tests::utils::get_module; - use datafusion_ffi::udtf::ForeignTableFunction; /// This test validates that we can load an external module and use a scalar /// udf defined in it via the foreign function interface. In this case we are @@ -46,11 +46,9 @@ mod tests { "External table function provider failed to implement create_table_function" .to_string(), ))?(function_registry.into()); - let foreign_table_func: ForeignTableFunction = ffi_table_func.into(); + let foreign_table_func: Arc = ffi_table_func.into(); - let udtf = Arc::new(foreign_table_func); - - ctx.register_udtf("my_range", udtf); + ctx.register_udtf("my_range", foreign_table_func); let result = ctx .sql("SELECT * FROM my_range(5)") From a0c53fb74e245171f57cd6f0fd6673b240b0e39a Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 7 Nov 2025 22:50:12 -0500 Subject: [PATCH 35/69] Avoid double free --- datafusion/ffi/src/udaf/accumulator.rs | 16 ++++++----- datafusion/ffi/src/udaf/groups_accumulator.rs | 27 ++++++++++--------- 2 files changed, 25 insertions(+), 18 deletions(-) diff --git a/datafusion/ffi/src/udaf/accumulator.rs b/datafusion/ffi/src/udaf/accumulator.rs index 9196c90cefa2..8f94cf820056 100644 --- a/datafusion/ffi/src/udaf/accumulator.rs +++ b/datafusion/ffi/src/udaf/accumulator.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::{arrow_wrappers::WrappedArray, df_result, rresult, rresult_return}; use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, @@ -26,10 +27,9 @@ use datafusion_common::{ }; use datafusion_expr::Accumulator; use prost::Message; +use std::ptr::null_mut; use std::{ffi::c_void, ops::Deref}; -use crate::{arrow_wrappers::WrappedArray, df_result, rresult, rresult_return}; - /// A stable struct for sharing [`Accumulator`] across FFI boundaries. /// For an explanation of each field, see the corresponding function /// defined in [`Accumulator`]. @@ -176,9 +176,11 @@ unsafe extern "C" fn retract_batch_fn_wrapper( } unsafe extern "C" fn release_fn_wrapper(accumulator: &mut FFI_Accumulator) { - let private_data = - Box::from_raw(accumulator.private_data as *mut AccumulatorPrivateData); - drop(private_data); + if !accumulator.private_data.is_null() { + let private_data = + Box::from_raw(accumulator.private_data as *mut AccumulatorPrivateData); + drop(private_data); + } } impl From> for FFI_Accumulator { @@ -222,12 +224,14 @@ unsafe impl Send for ForeignAccumulator {} unsafe impl Sync for ForeignAccumulator {} impl From for Box { - fn from(accumulator: FFI_Accumulator) -> Self { + fn from(mut accumulator: FFI_Accumulator) -> Self { if (accumulator.library_marker_id)() == crate::get_library_marker_id() { unsafe { let private_data = Box::from_raw( accumulator.private_data as *mut AccumulatorPrivateData, ); + // We must set this to null to avoid a double free + accumulator.private_data = null_mut(); private_data.accumulator } } else { diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index e7f2335fed91..98ff533962e2 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ffi::c_void, ops::Deref, sync::Arc}; - use crate::{ arrow_wrappers::{WrappedArray, WrappedSchema}, df_result, rresult, rresult_return, @@ -32,6 +30,8 @@ use arrow::{ }; use datafusion_common::error::{DataFusionError, Result}; use datafusion_expr::{EmitTo, GroupsAccumulator}; +use std::ptr::null_mut; +use std::{ffi::c_void, ops::Deref, sync::Arc}; /// A stable struct for sharing [`GroupsAccumulator`] across FFI boundaries. /// For an explanation of each field, see the corresponding function @@ -217,9 +217,11 @@ unsafe extern "C" fn convert_to_state_fn_wrapper( } unsafe extern "C" fn release_fn_wrapper(accumulator: &mut FFI_GroupsAccumulator) { - let private_data = - Box::from_raw(accumulator.private_data as *mut GroupsAccumulatorPrivateData); - drop(private_data); + if !accumulator.private_data.is_null() { + let private_data = + Box::from_raw(accumulator.private_data as *mut GroupsAccumulatorPrivateData); + drop(private_data); + } } impl From> for FFI_GroupsAccumulator { @@ -243,11 +245,11 @@ impl From> for FFI_GroupsAccumulator { } } -impl Drop for FFI_GroupsAccumulator { - fn drop(&mut self) { - unsafe { (self.release)(self) } - } -} +// impl Drop for FFI_GroupsAccumulator { +// fn drop(&mut self) { +// unsafe { (self.release)(self) } +// } +// } /// This struct is used to access an UDF provided by a foreign /// library across a FFI boundary. @@ -264,12 +266,13 @@ unsafe impl Send for ForeignGroupsAccumulator {} unsafe impl Sync for ForeignGroupsAccumulator {} impl From for Box { - fn from(accumulator: FFI_GroupsAccumulator) -> Self { + fn from(mut accumulator: FFI_GroupsAccumulator) -> Self { if (accumulator.library_marker_id)() == crate::get_library_marker_id() { unsafe { let private_data = Box::from_raw( accumulator.private_data as *mut GroupsAccumulatorPrivateData, ); + accumulator.private_data = null_mut(); private_data.accumulator } } else { @@ -451,7 +454,7 @@ mod tests { use super::{FFI_EmitTo, FFI_GroupsAccumulator}; #[test] - fn test_foreign_avg_accumulator() -> Result<()> { + fn test_foreign_bool_groups_accumulator() -> Result<()> { let boxed_accum: Box = Box::new(BooleanGroupsAccumulator::new(|a, b| a && b, true)); let ffi_accum: FFI_GroupsAccumulator = boxed_accum.into(); From 069ffbdd2f8b0f4ea012f76b8810b3c290bd79a8 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 07:42:16 -0500 Subject: [PATCH 36/69] keep config instead of config options --- datafusion/ffi/src/session/config.rs | 51 +++++++++++++--------------- 1 file changed, 23 insertions(+), 28 deletions(-) diff --git a/datafusion/ffi/src/session/config.rs b/datafusion/ffi/src/session/config.rs index 60a7286e32a4..42d1f8c49856 100644 --- a/datafusion/ffi/src/session/config.rs +++ b/datafusion/ffi/src/session/config.rs @@ -19,16 +19,9 @@ use abi_stable::{ std_types::{RHashMap, RString}, StableAbi, }; -use datafusion_common::{ - config::ConfigOptions, - error::{DataFusionError, Result}, -}; +use datafusion_common::error::{DataFusionError, Result}; use datafusion_execution::config::SessionConfig; -use std::sync::Arc; -use std::{ - collections::HashMap, - ffi::{c_char, c_void, CString}, -}; +use std::{collections::HashMap, ffi::c_void}; /// A stable struct for sharing [`SessionConfig`] across FFI boundaries. /// Instead of attempting to expose the entire SessionConfig interface, we @@ -58,16 +51,26 @@ pub struct FFI_SessionConfig { /// Internal data. This is only to be accessed by the provider of the plan. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } unsafe impl Send for FFI_SessionConfig {} unsafe impl Sync for FFI_SessionConfig {} +impl FFI_SessionConfig { + fn inner(&self) -> &SessionConfig { + let private_data = self.private_data as *mut SessionConfigPrivateData; + unsafe { &(*private_data).config } + } +} + unsafe extern "C" fn config_options_fn_wrapper( config: &FFI_SessionConfig, ) -> RHashMap { - let private_data = config.private_data as *mut SessionConfigPrivateData; - let config_options = &(*private_data).config; + let config_options = config.inner().options(); let mut options = RHashMap::default(); for config_entry in config_options.entries() { @@ -87,7 +90,7 @@ unsafe extern "C" fn release_fn_wrapper(config: &mut FFI_SessionConfig) { unsafe extern "C" fn clone_fn_wrapper(config: &FFI_SessionConfig) -> FFI_SessionConfig { let old_private_data = config.private_data as *mut SessionConfigPrivateData; - let old_config = Arc::clone(&(*old_private_data).config); + let old_config = (*old_private_data).config.clone(); let private_data = Box::new(SessionConfigPrivateData { config: old_config }); @@ -96,31 +99,18 @@ unsafe extern "C" fn clone_fn_wrapper(config: &FFI_SessionConfig) -> FFI_Session private_data: Box::into_raw(private_data) as *mut c_void, clone: clone_fn_wrapper, release: release_fn_wrapper, + library_marker_id: crate::get_library_marker_id, } } struct SessionConfigPrivateData { - pub config: Arc, + pub config: SessionConfig, } impl From<&SessionConfig> for FFI_SessionConfig { fn from(session: &SessionConfig) -> Self { - let mut config_keys = Vec::new(); - let mut config_values = Vec::new(); - for config_entry in session.options().entries() { - if let Some(value) = config_entry.value { - let key_cstr = CString::new(config_entry.key).unwrap_or_default(); - let key_ptr = key_cstr.into_raw() as *const c_char; - config_keys.push(key_ptr); - - config_values - .push(CString::new(value).unwrap_or_default().into_raw() - as *const c_char); - } - } - let private_data = Box::new(SessionConfigPrivateData { - config: Arc::clone(session.options()), + config: session.clone(), }); Self { @@ -128,6 +118,7 @@ impl From<&SessionConfig> for FFI_SessionConfig { private_data: Box::into_raw(private_data) as *mut c_void, clone: clone_fn_wrapper, release: release_fn_wrapper, + library_marker_id: crate::get_library_marker_id, } } } @@ -152,6 +143,10 @@ impl TryFrom<&FFI_SessionConfig> for SessionConfig { type Error = DataFusionError; fn try_from(config: &FFI_SessionConfig) -> Result { + if (config.library_marker_id)() == crate::get_library_marker_id() { + return Ok(config.inner().clone()); + } + let config_options = unsafe { (config.config_options)(config) }; let mut options_map = HashMap::new(); From 8a14d6fd37fd1fd126e38f6616b8bd08611cfb97 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 07:48:53 -0500 Subject: [PATCH 37/69] plug mem leak --- datafusion/ffi/src/udaf/groups_accumulator.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 98ff533962e2..904d2f7e593b 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -245,11 +245,11 @@ impl From> for FFI_GroupsAccumulator { } } -// impl Drop for FFI_GroupsAccumulator { -// fn drop(&mut self) { -// unsafe { (self.release)(self) } -// } -// } +impl Drop for FFI_GroupsAccumulator { + fn drop(&mut self) { + unsafe { (self.release)(self) } + } +} /// This struct is used to access an UDF provided by a foreign /// library across a FFI boundary. From c4451e539c3333f7303733ef576098f1b259f9cb Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 08:01:48 -0500 Subject: [PATCH 38/69] Check if task context is a local library --- datafusion/execution/src/task.rs | 2 +- datafusion/ffi/src/session/task.rs | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index c2a6cfe2c833..c8ee4f0823ec 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -32,7 +32,7 @@ use std::{collections::HashMap, sync::Arc}; /// information. /// /// [`SessionContext`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionContext.html -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct TaskContext { /// Session Id session_id: String, diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index b306d97470ec..10aadfba4623 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -54,6 +54,10 @@ pub struct FFI_TaskContext { /// Internal data. This is only to be accessed by the provider of the plan. /// The foreign library should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } struct TaskContextPrivateData { @@ -138,6 +142,7 @@ impl From> for FFI_TaskContext { window_functions: window_functions_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -145,6 +150,10 @@ impl From> for FFI_TaskContext { impl From for TaskContext { fn from(ffi_ctx: FFI_TaskContext) -> Self { unsafe { + if (ffi_ctx.library_marker_id)() == crate::get_library_marker_id() { + return ffi_ctx.inner().clone(); + } + let task_id = (ffi_ctx.task_id)(&ffi_ctx).map(|s| s.to_string()).into(); let sesion_id = (ffi_ctx.session_id)(&ffi_ctx).into(); let session_config = (ffi_ctx.session_config)(&ffi_ctx); From 3fb668dcca0321b4a7c7e5b75ae6aa7b44af068c Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 08:32:42 -0500 Subject: [PATCH 39/69] Use mock marker ID for unit tests --- datafusion/ffi/src/catalog_provider.rs | 3 +- datafusion/ffi/src/execution_plan.rs | 9 +-- datafusion/ffi/src/lib.rs | 8 +++ datafusion/ffi/src/plan_properties.rs | 62 ++++++++++--------- datafusion/ffi/src/schema_provider.rs | 3 +- datafusion/ffi/src/session/config.rs | 3 +- datafusion/ffi/src/session/task.rs | 33 ---------- datafusion/ffi/src/table_provider.rs | 9 ++- datafusion/ffi/src/udaf/accumulator.rs | 3 +- datafusion/ffi/src/udaf/groups_accumulator.rs | 3 +- datafusion/ffi/src/udaf/mod.rs | 9 ++- datafusion/ffi/src/udf/mod.rs | 3 +- datafusion/ffi/src/udtf.rs | 3 +- datafusion/ffi/src/udwf/mod.rs | 6 +- 14 files changed, 73 insertions(+), 84 deletions(-) diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index c531598fdd87..1d33aa5e13c0 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -340,8 +340,9 @@ mod tests { let function_registry = Arc::clone(&ctx) as Arc; - let ffi_catalog = + let mut ffi_catalog = FFI_CatalogProvider::new(catalog, None, function_registry.into()); + ffi_catalog.library_marker_id = crate::mock_foreign_marker_id; let foreign_catalog: Arc = (&ffi_catalog).into(); diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index f6fbe6971549..d4e8b9a795fc 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -381,10 +381,6 @@ mod tests { } } - extern "C" fn mock_library_marker_id() -> u64 { - crate::get_library_marker_id() + 1 - } - #[test] fn test_round_trip_ffi_execution_plan() -> Result<()> { let schema = @@ -397,7 +393,7 @@ mod tests { let mut local_plan = FFI_ExecutionPlan::new(original_plan, ctx.task_ctx(), None); // Force round trip to go through foreign provider - local_plan.library_marker_id = mock_library_marker_id; + local_plan.library_marker_id = crate::mock_foreign_marker_id; let foreign_plan: Arc = (&local_plan).try_into()?; @@ -424,7 +420,8 @@ mod tests { // Version 1: Adding child to the foreign plan let child_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); - let child_local = FFI_ExecutionPlan::new(child_plan, ctx.task_ctx(), None); + let mut child_local = FFI_ExecutionPlan::new(child_plan, ctx.task_ctx(), None); + child_local.library_marker_id = crate::mock_foreign_marker_id; let child_foreign = >::try_from(&child_local)?; let parent_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index f2fb2ac17b56..2e37344a9ca3 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -75,5 +75,13 @@ pub extern "C" fn get_library_marker_id() -> u64 { &LIBRARY_MARKER as *const u8 as u64 } +/// For unit testing in this crate we need to trick the providers +/// into thinking we have a foreign call. We do this by overwriting +/// their `library_marker_id` function to return a different value. +#[cfg(test)] +pub(crate) extern "C" fn mock_foreign_marker_id() -> u64 { + get_library_marker_id() + 1 +} + #[cfg(doctest)] doc_comment::doctest!("../README.md", readme_example_test); diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index ffb00f8d61f1..b51a75d717b7 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -58,57 +58,53 @@ pub struct FFI_PlanProperties { /// Internal data. This is only to be accessed by the provider of the plan. /// The foreign library should never attempt to access this data. pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, } struct PlanPropertiesPrivateData { props: PlanProperties, } +impl FFI_PlanProperties { + fn inner(&self) -> &PlanProperties { + let private_data = self.private_data as *const PlanPropertiesPrivateData; + unsafe { &(*private_data).props } + } +} + unsafe extern "C" fn output_partitioning_fn_wrapper( properties: &FFI_PlanProperties, ) -> FFI_Partitioning { - let private_data = properties.private_data as *const PlanPropertiesPrivateData; - let props = &(*private_data).props; - // - // let codec = DefaultPhysicalExtensionCodec {}; - // let partitioning_data = - // rresult_return!(serialize_partitioning(props.output_partitioning(), &codec)); - // let output_partitioning = partitioning_data.encode_to_vec(); - - // ROk(output_partitioning.into()) - (&props.partitioning).into() + (&properties.inner().partitioning).into() } unsafe extern "C" fn emission_type_fn_wrapper( properties: &FFI_PlanProperties, ) -> FFI_EmissionType { - let private_data = properties.private_data as *const PlanPropertiesPrivateData; - let props = &(*private_data).props; - props.emission_type.into() + (&properties.inner().emission_type).into() } unsafe extern "C" fn boundedness_fn_wrapper( properties: &FFI_PlanProperties, ) -> FFI_Boundedness { - let private_data = properties.private_data as *const PlanPropertiesPrivateData; - let props = &(*private_data).props; - props.boundedness.into() + (&properties.inner().boundedness).into() } unsafe extern "C" fn output_ordering_fn_wrapper( properties: &FFI_PlanProperties, ) -> ROption { - let private_data = properties.private_data as *const PlanPropertiesPrivateData; - let props = &(*private_data).props; - - props.output_ordering().map(FFI_LexOrdering::from).into() + properties + .inner() + .output_ordering() + .map(FFI_LexOrdering::from) + .into() } unsafe extern "C" fn schema_fn_wrapper(properties: &FFI_PlanProperties) -> WrappedSchema { - let private_data = properties.private_data as *const PlanPropertiesPrivateData; - let props = &(*private_data).props; - - let schema: SchemaRef = Arc::clone(props.eq_properties.schema()); + let schema: SchemaRef = Arc::clone(properties.inner().eq_properties.schema()); schema.into() } @@ -138,6 +134,7 @@ impl From<&PlanProperties> for FFI_PlanProperties { schema: schema_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, } } } @@ -146,6 +143,10 @@ impl TryFrom for PlanProperties { type Error = DataFusionError; fn try_from(ffi_props: FFI_PlanProperties) -> Result { + if (ffi_props.library_marker_id)() == crate::get_library_marker_id() { + return Ok(ffi_props.inner().clone()); + } + let ffi_schema = unsafe { (ffi_props.schema)(&ffi_props) }; let schema = (&ffi_schema.0).try_into()?; @@ -185,14 +186,14 @@ pub enum FFI_Boundedness { Unbounded { requires_infinite_memory: bool }, } -impl From for FFI_Boundedness { - fn from(value: Boundedness) -> Self { +impl From<&Boundedness> for FFI_Boundedness { + fn from(value: &Boundedness) -> Self { match value { Boundedness::Bounded => FFI_Boundedness::Bounded, Boundedness::Unbounded { requires_infinite_memory, } => FFI_Boundedness::Unbounded { - requires_infinite_memory, + requires_infinite_memory: *requires_infinite_memory, }, } } @@ -221,8 +222,8 @@ pub enum FFI_EmissionType { Both, } -impl From for FFI_EmissionType { - fn from(value: EmissionType) -> Self { +impl From<&EmissionType> for FFI_EmissionType { + fn from(value: &EmissionType) -> Self { match value { EmissionType::Incremental => FFI_EmissionType::Incremental, EmissionType::Final => FFI_EmissionType::Final, @@ -264,7 +265,8 @@ mod tests { Boundedness::Bounded, ); - let local_props_ptr = FFI_PlanProperties::from(&original_props); + let mut local_props_ptr = FFI_PlanProperties::from(&original_props); + local_props_ptr.library_marker_id = crate::mock_foreign_marker_id; let foreign_props: PlanProperties = local_props_ptr.try_into()?; diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index 75437d8d1f09..fa9bcfaf7749 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -359,8 +359,9 @@ mod tests { let function_registry = Arc::clone(&ctx) as Arc; - let ffi_schema_provider = + let mut ffi_schema_provider = FFI_SchemaProvider::new(schema_provider, None, function_registry.into()); + ffi_schema_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_schema_provider: Arc = (&ffi_schema_provider).into(); diff --git a/datafusion/ffi/src/session/config.rs b/datafusion/ffi/src/session/config.rs index 42d1f8c49856..afef05ce19ed 100644 --- a/datafusion/ffi/src/session/config.rs +++ b/datafusion/ffi/src/session/config.rs @@ -167,7 +167,8 @@ mod tests { let session_config = SessionConfig::new(); let original_options = session_config.options().entries(); - let ffi_config: FFI_SessionConfig = (&session_config).into(); + let mut ffi_config: FFI_SessionConfig = (&session_config).into(); + ffi_config.library_marker_id = crate::mock_foreign_marker_id; let foreign_config: SessionConfig = (&ffi_config).try_into()?; diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task.rs index 10aadfba4623..e61879da5a13 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task.rs @@ -226,36 +226,3 @@ impl From for TaskContext { } } } -// -// #[cfg(test)] -// mod tests { -// use datafusion::{physical_expr::PhysicalSortExpr, physical_plan::Partitioning}; -// -// use super::*; -// -// #[test] -// fn test_round_trip_ffi_plan_properties() -> Result<()> { -// use arrow::datatypes::{DataType, Field, Schema}; -// let schema = -// Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, false)])); -// -// let mut eqp = EquivalenceProperties::new(Arc::clone(&schema)); -// let _ = eqp.reorder([PhysicalSortExpr::new_default( -// datafusion::physical_plan::expressions::col("a", &schema)?, -// )]); -// let original_ctx = TaskContext::new( -// eqp, -// Partitioning::RoundRobinBatch(3), -// EmissionType::Incremental, -// Boundedness::Bounded, -// ); -// -// let local_ctx_ptr = FFI_TaskContext::from(&original_ctx); -// -// let foreign_ctx: TaskContext = local_ctx_ptr.try_into()?; -// -// assert_eq!(format!("{foreign_props:?}"), format!("{original_props:?}")); -// -// Ok(()) -// } -// } diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index e8187a4575eb..e21355649176 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -550,8 +550,9 @@ mod tests { let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); - let ffi_provider = + let mut ffi_provider = FFI_TableProvider::new(provider, true, None, function_registry.into()); + ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); @@ -595,8 +596,9 @@ mod tests { let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); - let ffi_provider = + let mut ffi_provider = FFI_TableProvider::new(provider, true, None, function_registry.into()); + ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); @@ -644,8 +646,9 @@ mod tests { let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1]])?); - let ffi_provider = + let mut ffi_provider = FFI_TableProvider::new(provider, true, None, function_registry.into()); + ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); diff --git a/datafusion/ffi/src/udaf/accumulator.rs b/datafusion/ffi/src/udaf/accumulator.rs index 8f94cf820056..f11ef0e3ade8 100644 --- a/datafusion/ffi/src/udaf/accumulator.rs +++ b/datafusion/ffi/src/udaf/accumulator.rs @@ -339,7 +339,8 @@ mod tests { let original_supports_retract = original_accum.supports_retract_batch(); let boxed_accum: Box = Box::new(original_accum); - let ffi_accum: FFI_Accumulator = boxed_accum.into(); + let mut ffi_accum: FFI_Accumulator = boxed_accum.into(); + ffi_accum.library_marker_id = crate::mock_foreign_marker_id; let mut foreign_accum: Box = ffi_accum.into(); // Send in an array to average. There are 5 values and it should average to 30.0 diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 904d2f7e593b..537c6f4a8d20 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -457,7 +457,8 @@ mod tests { fn test_foreign_bool_groups_accumulator() -> Result<()> { let boxed_accum: Box = Box::new(BooleanGroupsAccumulator::new(|a, b| a && b, true)); - let ffi_accum: FFI_GroupsAccumulator = boxed_accum.into(); + let mut ffi_accum: FFI_GroupsAccumulator = boxed_accum.into(); + ffi_accum.library_marker_id = crate::mock_foreign_marker_id; let mut foreign_accum: Box = ffi_accum.into(); // Send in an array to evaluate. We want a mean of 30 and standard deviation of 4. diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index 30482ed467a7..49bae72825af 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -665,7 +665,8 @@ mod tests { ) -> Result { let original_udaf = Arc::new(AggregateUDF::from(original_udaf)); - let local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + let mut local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + local_udaf.library_marker_id = crate::mock_foreign_marker_id; let foreign_udaf: Arc = (&local_udaf).try_into()?; Ok(AggregateUDF::new_from_shared_impl(foreign_udaf)) @@ -678,7 +679,8 @@ mod tests { let original_udaf = Arc::new(AggregateUDF::from(original_udaf)); // Convert to FFI format - let local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + let mut local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + local_udaf.library_marker_id = crate::mock_foreign_marker_id; // Convert back to native format let foreign_udaf: Arc = (&local_udaf).try_into()?; @@ -732,7 +734,8 @@ mod tests { let original_udaf = Arc::new(AggregateUDF::from(original_udaf)); // Convert to FFI format - let local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + let mut local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + local_udaf.library_marker_id = crate::mock_foreign_marker_id; // Convert back to native format let foreign_udaf: Arc = (&local_udaf).try_into()?; diff --git a/datafusion/ffi/src/udf/mod.rs b/datafusion/ffi/src/udf/mod.rs index 0f34400c77db..6e06ea07fc57 100644 --- a/datafusion/ffi/src/udf/mod.rs +++ b/datafusion/ffi/src/udf/mod.rs @@ -466,7 +466,8 @@ mod tests { let original_udf = datafusion::functions::math::abs::AbsFunc::new(); let original_udf = Arc::new(ScalarUDF::from(original_udf)); - let local_udf: FFI_ScalarUDF = Arc::clone(&original_udf).into(); + let mut local_udf: FFI_ScalarUDF = Arc::clone(&original_udf).into(); + local_udf.library_marker_id = crate::mock_foreign_marker_id; let foreign_udf: Arc = (&local_udf).try_into()?; diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index 45f8faedd1a1..46e66217d839 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -300,11 +300,12 @@ mod tests { let ctx = Arc::new(SessionContext::default()); let function_registry = Arc::clone(&ctx) as Arc; - let local_udtf: FFI_TableFunction = FFI_TableFunction::new( + let mut local_udtf: FFI_TableFunction = FFI_TableFunction::new( Arc::clone(&original_udtf), None, function_registry.into(), ); + local_udtf.library_marker_id = crate::mock_foreign_marker_id; let foreign_udf: Arc = local_udtf.into(); diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index f7ff2aa4ac28..6b107e66a856 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -407,7 +407,8 @@ mod tests { ) -> datafusion::common::Result { let original_udwf = Arc::new(WindowUDF::from(original_udwf)); - let local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); + let mut local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); + local_udwf.library_marker_id = crate::mock_foreign_marker_id; let foreign_udwf: Arc = (&local_udwf).try_into()?; Ok(WindowUDF::new_from_shared_impl(foreign_udwf)) @@ -419,7 +420,8 @@ mod tests { let original_name = original_udwf.name().to_owned(); // Convert to FFI format - let local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); + let mut local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); + local_udwf.library_marker_id = crate::mock_foreign_marker_id; // Convert back to native format let foreign_udwf: Arc = (&local_udwf).try_into()?; From 04666fab7a13b849d3d2a0196e233623aca84263 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 09:59:14 -0500 Subject: [PATCH 40/69] Add explicit type --- datafusion/ffi/src/udaf/accumulator_args.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 65ed1ff09cb6..c32bdf94b2a3 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -101,10 +101,10 @@ impl TryFrom for ForeignAccumulatorArgs { type Error = DataFusionError; fn try_from(value: FFI_AccumulatorArgs) -> Result { - let exprs = value + let exprs: Vec> = value .exprs .into_iter() - .map(>::from) + .map(Into::into) .collect(); let return_field = Arc::new((&value.return_field.0).try_into()?); From 102e5af8c016bf9c365e57f8a857412f9b87880e Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 09:59:25 -0500 Subject: [PATCH 41/69] Add documentation about code coverage testing --- datafusion/ffi/README.md | 43 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/datafusion/ffi/README.md b/datafusion/ffi/README.md index 72070984f931..ea30e9eeb0f3 100644 --- a/datafusion/ffi/README.md +++ b/datafusion/ffi/README.md @@ -101,6 +101,48 @@ In this crate we have a variety of structs which closely mimic the behavior of their internal counterparts. To see detailed notes about how to use them, see the example in `FFI_TableProvider`. +## Testing Coverage + +Since this library contains a large amount of `unsafe` code, it is important +to ensure proper test coverage. To generate a coverage report, you can use +[tarpaulin] as follows. It is necessary to use the `integration-tests` feature +in order to properly generate coverage. + +```shell +cargo tarpaulin --package datafusion-ffi --tests --features integration-tests --out Html +``` + +While it is not normally required to check Rust code for memory leaks, this +crate does manual memory management due to the FFI boundary. You can test for +leaks using the generated unit tests. How you run these checks differs depending +on your OS. + +### Linux + +On Linux, you can install `cargo-valgrind` + +```shell +cargo valgrind test --features integration-tests -p datafusion-ffi +``` + +### MacOS + +You can find the generated binaries for your unit tests by running `cargo test`. + +```shell +cargo test --features integration-tests -p datafusion-ffi --no-run +``` + +This should generate output that shows the path to the test binaries. Then +you can run commands such as the following. The specific paths of the tests +will vary. + +```shell +leaks --atExit -- target/debug/deps/datafusion_ffi-e77a2604a85a8afe +leaks --atExit -- target/debug/deps/ffi_integration-e91b7127a59b71a7 +# ... +``` + [apache datafusion]: https://datafusion.apache.org/ [api docs]: http://docs.rs/datafusion-ffi/latest [rust abi]: https://doc.rust-lang.org/reference/abi.html @@ -110,3 +152,4 @@ the example in `FFI_TableProvider`. [bindgen]: https://crates.io/crates/bindgen [`datafusion-python`]: https://datafusion.apache.org/python/ [datafusion-contrib]: https://github.com/datafusion-contrib +[tarpaulin]: https://crates.io/crates/cargo-tarpaulin From d13afdb3f10d4cb1df273fc80dbfb0bb623b596a Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 10:08:14 -0500 Subject: [PATCH 42/69] plug mem leak --- datafusion/ffi/src/udaf/accumulator_args.rs | 7 ++----- datafusion/ffi/src/udwf/partition_evaluator.rs | 15 +++++++++------ 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index c32bdf94b2a3..244fc5901c66 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -101,11 +101,8 @@ impl TryFrom for ForeignAccumulatorArgs { type Error = DataFusionError; fn try_from(value: FFI_AccumulatorArgs) -> Result { - let exprs: Vec> = value - .exprs - .into_iter() - .map(Into::into) - .collect(); + let exprs: Vec> = + value.exprs.into_iter().map(Into::into).collect(); let return_field = Arc::new((&value.return_field.0).try_into()?); let schema = Schema::try_from(&value.schema.0)?; diff --git a/datafusion/ffi/src/udwf/partition_evaluator.rs b/datafusion/ffi/src/udwf/partition_evaluator.rs index b5c4b42baa95..ff28b0704efb 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use super::range::FFI_Range; use crate::{arrow_wrappers::WrappedArray, df_result, rresult, rresult_return}; use abi_stable::{ std_types::{RResult, RString, RVec}, @@ -27,10 +28,9 @@ use datafusion_common::{ }; use datafusion_expr::{window_state::WindowAggState, PartitionEvaluator}; use prost::Message; +use std::ptr::null_mut; use std::{ffi::c_void, ops::Range}; -use super::range::FFI_Range; - /// A stable struct for sharing [`PartitionEvaluator`] across FFI boundaries. /// For an explanation of each field, see the corresponding function /// defined in [`PartitionEvaluator`]. @@ -173,9 +173,11 @@ unsafe extern "C" fn get_range_fn_wrapper( } unsafe extern "C" fn release_fn_wrapper(evaluator: &mut FFI_PartitionEvaluator) { - let private_data = - Box::from_raw(evaluator.private_data as *mut PartitionEvaluatorPrivateData); - drop(private_data); + if !evaluator.private_data.is_null() { + let private_data = + Box::from_raw(evaluator.private_data as *mut PartitionEvaluatorPrivateData); + drop(private_data); + } } impl From> for FFI_PartitionEvaluator { @@ -224,12 +226,13 @@ unsafe impl Send for ForeignPartitionEvaluator {} unsafe impl Sync for ForeignPartitionEvaluator {} impl From for Box { - fn from(evaluator: FFI_PartitionEvaluator) -> Self { + fn from(mut evaluator: FFI_PartitionEvaluator) -> Self { if (evaluator.library_marker_id)() == crate::get_library_marker_id() { unsafe { let private_data = Box::from_raw( evaluator.private_data as *mut PartitionEvaluatorPrivateData, ); + evaluator.private_data = null_mut(); private_data.evaluator } } else { From 2be617ee47d1ffaf5fd16e8c8e17b0505e25defc Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 12:04:07 -0500 Subject: [PATCH 43/69] add test coverage for function registry --- datafusion/ffi/src/function_registry.rs | 54 +++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 4ff1b621a9ed..395ac894c719 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -329,6 +329,7 @@ impl FunctionRegistry for ForeignWeakFunctionRegistry { } fn expr_planners(&self) -> Vec> { + warn!("FFI Function Registry does not support expression planners."); vec![] } @@ -339,3 +340,56 @@ impl FunctionRegistry for ForeignWeakFunctionRegistry { not_impl_err!("Function Registry does not allow mutation via FFI") } } + +#[cfg(test)] +mod tests { + use super::*; + use datafusion::prelude::SessionContext; + use datafusion_expr::registry::FunctionRegistry; + + #[tokio::test] + async fn test_round_trip_ffi_function_registry() -> Result<(), DataFusionError> { + let ctx = Arc::new(SessionContext::new()); + let function_registry = + Arc::clone(&ctx) as Arc; + + let mut ffi_registry = FFI_WeakFunctionRegistry::from(function_registry); + ffi_registry.library_marker_id = crate::mock_foreign_marker_id; + + let foreign_registry: Arc = + (&ffi_registry).try_into()?; + + let udf_names = foreign_registry.udfs(); + assert!(!udf_names.is_empty()); + let udf = foreign_registry.udf(udf_names.iter().next().unwrap())?; + + let udaf_names = foreign_registry.udafs(); + assert!(!udaf_names.is_empty()); + let udaf = foreign_registry.udaf(udaf_names.iter().next().unwrap())?; + + let udwf_names = foreign_registry.udwfs(); + assert!(!udwf_names.is_empty()); + let udwf = foreign_registry.udwf(udwf_names.iter().next().unwrap())?; + + // The following tests exist to ensure that if we do add support + // for mutable function registry in the future that we have + // added test coverage. + + // Manually create foreign registry so we can make it mutable + let mut foreign_registry = ForeignWeakFunctionRegistry(ffi_registry); + + fn expect_not_implemented(input: Result) { + let Err(DataFusionError::NotImplemented(_)) = input else { + panic!("Expected not implemented feature"); + }; + } + expect_not_implemented(foreign_registry.register_udf(udf)); + expect_not_implemented(foreign_registry.register_udaf(udaf)); + expect_not_implemented(foreign_registry.register_udwf(udwf)); + expect_not_implemented(foreign_registry.deregister_udf("a")); + expect_not_implemented(foreign_registry.deregister_udaf("a")); + expect_not_implemented(foreign_registry.deregister_udwf("a")); + + Ok(()) + } +} From ad1b780f995febcd7d83003b911712ad58186724 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 12:04:17 -0500 Subject: [PATCH 44/69] improve test coverage for udtf --- datafusion/ffi/src/udtf.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index 46e66217d839..24366d5ae944 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -305,6 +305,10 @@ mod tests { None, function_registry.into(), ); + + // Add unit test coverage to check for memory leaks on clone + let _ = local_udtf.clone(); + local_udtf.library_marker_id = crate::mock_foreign_marker_id; let foreign_udf: Arc = local_udtf.into(); From bf0e4f8cbea3dd0c1bf611a324f51ac1b4f78f0f Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 14:25:15 -0500 Subject: [PATCH 45/69] Remove FFI_PhysicalExpr --- datafusion/ffi/src/lib.rs | 1 - datafusion/ffi/src/physical_expr/mod.rs | 695 ------------------ .../ffi/src/physical_expr/partitioning.rs | 49 -- datafusion/ffi/src/physical_expr/sort.rs | 74 -- datafusion/ffi/src/udaf/accumulator_args.rs | 63 +- .../ffi/src/udwf/partition_evaluator_args.rs | 30 +- 6 files changed, 64 insertions(+), 848 deletions(-) delete mode 100644 datafusion/ffi/src/physical_expr/mod.rs delete mode 100644 datafusion/ffi/src/physical_expr/partitioning.rs delete mode 100644 datafusion/ffi/src/physical_expr/sort.rs diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index 2e37344a9ca3..fe7ded25e123 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -30,7 +30,6 @@ pub mod execution_plan; pub mod expr; pub mod function_registry; pub mod insert_op; -pub mod physical_expr; pub mod plan_properties; pub mod record_batch_stream; pub mod schema_provider; diff --git a/datafusion/ffi/src/physical_expr/mod.rs b/datafusion/ffi/src/physical_expr/mod.rs deleted file mode 100644 index 036e908dcf91..000000000000 --- a/datafusion/ffi/src/physical_expr/mod.rs +++ /dev/null @@ -1,695 +0,0 @@ -// 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. - -pub mod partitioning; -pub mod sort; - -use crate::arrow_wrappers::WrappedArray; -use crate::expr::columnar_value::FFI_ColumnarValue; -use crate::expr::distribution::FFI_Distribution; -use crate::expr::expr_properties::FFI_ExprProperties; -use crate::expr::interval::FFI_Interval; -use crate::record_batch_stream::{ - record_batch_to_wrapped_array, wrapped_array_to_record_batch, -}; -use crate::util::FFIResult; -use crate::{arrow_wrappers::WrappedSchema, df_result, rresult, rresult_return}; -use abi_stable::std_types::RResult; -use abi_stable::{ - std_types::{ROption, RString, RVec}, - StableAbi, -}; -use arrow::array::{ArrayRef, BooleanArray, RecordBatch}; -use arrow::datatypes::SchemaRef; -use arrow_schema::ffi::FFI_ArrowSchema; -use arrow_schema::{DataType, Field, FieldRef, Schema}; -use datafusion_common::exec_datafusion_err; -use datafusion_common::Result; -use datafusion_expr::ColumnarValue; -use datafusion_expr_common::interval_arithmetic::Interval; -use datafusion_expr_common::sort_properties::ExprProperties; -use datafusion_expr_common::statistics::Distribution; -use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExpr}; -use std::fmt::{Display, Formatter}; -use std::hash::{DefaultHasher, Hash, Hasher}; -use std::{any::Any, ffi::c_void, sync::Arc}; - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_PhysicalExpr { - pub data_type: unsafe extern "C" fn( - &Self, - input_schema: WrappedSchema, - ) -> FFIResult, - - pub nullable: - unsafe extern "C" fn(&Self, input_schema: WrappedSchema) -> FFIResult, - - pub evaluate: - unsafe extern "C" fn(&Self, batch: WrappedArray) -> FFIResult, - - pub return_field: unsafe extern "C" fn( - &Self, - input_schema: WrappedSchema, - ) -> FFIResult, - - pub evaluate_selection: unsafe extern "C" fn( - &Self, - batch: WrappedArray, - selection: WrappedArray, - ) -> FFIResult, - - pub children: unsafe extern "C" fn(&Self) -> RVec, - - pub new_with_children: - unsafe extern "C" fn(&Self, children: &RVec) -> FFIResult, - - pub evaluate_bounds: unsafe extern "C" fn( - &Self, - children: &RVec, - ) -> FFIResult, - - pub propagate_constraints: - unsafe extern "C" fn( - &Self, - interval: &FFI_Interval, - children: &RVec, - ) -> FFIResult>>, - - pub evaluate_statistics: unsafe extern "C" fn( - &Self, - children: &RVec, - ) -> FFIResult, - - pub propagate_statistics: - unsafe extern "C" fn( - &Self, - parent: &FFI_Distribution, - children: &RVec, - ) -> FFIResult>>, - - pub get_properties: unsafe extern "C" fn( - &Self, - children: &RVec, - ) -> FFIResult, - - pub fmt_sql: unsafe extern "C" fn(&Self) -> FFIResult, - - pub snapshot: unsafe extern "C" fn(&Self) -> FFIResult>, - - pub snapshot_generation: unsafe extern "C" fn(&Self) -> u64, - - pub is_volatile_node: unsafe extern "C" fn(&Self) -> bool, - - // Display trait - pub display: unsafe extern "C" fn(&Self) -> RString, - - // Hash trait - pub hash: unsafe extern "C" fn(&Self) -> u64, - - /// Used to create a clone on the provider of the execution plan. This should - /// only need to be called by the receiver of the plan. - pub clone: unsafe extern "C" fn(plan: &Self) -> Self, - - /// Release the memory of the private data when it is no longer being used. - pub release: unsafe extern "C" fn(arg: &mut Self), - - /// Return the major DataFusion version number of this provider. - pub version: unsafe extern "C" fn() -> u64, - - /// Internal data. This is only to be accessed by the provider of the plan. - /// A [`ForeignExecutionPlan`] should never attempt to access this data. - pub private_data: *mut c_void, - - /// Utility to identify when FFI objects are accessed locally through - /// the foreign interface. - pub library_marker_id: extern "C" fn() -> u64, -} - -unsafe impl Send for FFI_PhysicalExpr {} -unsafe impl Sync for FFI_PhysicalExpr {} - -impl FFI_PhysicalExpr { - fn inner(&self) -> &Arc { - unsafe { - let private_data = self.private_data as *const PhysicalExprPrivateData; - &(*private_data).expr - } - } -} - -struct PhysicalExprPrivateData { - expr: Arc, -} - -unsafe extern "C" fn data_type_fn_wrapper( - expr: &FFI_PhysicalExpr, - input_schema: WrappedSchema, -) -> FFIResult { - let expr = expr.inner(); - let schema: SchemaRef = input_schema.into(); - let data_type = expr - .data_type(&schema) - .and_then(|dt| FFI_ArrowSchema::try_from(dt).map_err(Into::into)) - .map(WrappedSchema); - rresult!(data_type) -} - -unsafe extern "C" fn nullable_fn_wrapper( - expr: &FFI_PhysicalExpr, - input_schema: WrappedSchema, -) -> FFIResult { - let expr = expr.inner(); - let schema: SchemaRef = input_schema.into(); - rresult!(expr.nullable(&schema)) -} - -unsafe extern "C" fn evaluate_fn_wrapper( - expr: &FFI_PhysicalExpr, - batch: WrappedArray, -) -> FFIResult { - let batch = rresult_return!(wrapped_array_to_record_batch(batch)); - rresult!(expr - .inner() - .evaluate(&batch) - .and_then(FFI_ColumnarValue::try_from)) -} - -unsafe extern "C" fn return_field_fn_wrapper( - expr: &FFI_PhysicalExpr, - input_schema: WrappedSchema, -) -> FFIResult { - let expr = expr.inner(); - let schema: SchemaRef = input_schema.into(); - rresult!(expr - .return_field(&schema) - .and_then(|f| FFI_ArrowSchema::try_from(&f).map_err(Into::into)) - .map(WrappedSchema)) -} - -unsafe extern "C" fn evaluate_selection_fn_wrapper( - expr: &FFI_PhysicalExpr, - batch: WrappedArray, - selection: WrappedArray, -) -> FFIResult { - let batch = rresult_return!(wrapped_array_to_record_batch(batch)); - let selection: ArrayRef = rresult_return!(selection.try_into()); - let selection = rresult_return!(selection - .as_any() - .downcast_ref::() - .ok_or(exec_datafusion_err!("Unexpected selection array type"))); - rresult!(expr - .inner() - .evaluate_selection(&batch, selection) - .and_then(FFI_ColumnarValue::try_from)) -} - -unsafe extern "C" fn children_fn_wrapper( - expr: &FFI_PhysicalExpr, -) -> RVec { - let expr = expr.inner(); - let children = expr.children(); - children - .into_iter() - .map(|child| FFI_PhysicalExpr::from(Arc::clone(child))) - .collect() -} - -unsafe extern "C" fn new_with_children_fn_wrapper( - expr: &FFI_PhysicalExpr, - children: &RVec, -) -> FFIResult { - let expr = Arc::clone(expr.inner()); - let children = children - .iter() - .map(|e| >::from(e.clone())) - .collect::>(); - rresult!(expr.with_new_children(children).map(FFI_PhysicalExpr::from)) -} - -unsafe extern "C" fn evaluate_bounds_fn_wrapper( - expr: &FFI_PhysicalExpr, - children: &RVec, -) -> FFIResult { - let expr = expr.inner(); - let children = rresult_return!(children - .iter() - .map(Interval::try_from) - .collect::>>()); - let children_borrowed = children.iter().collect::>(); - - rresult!(expr - .evaluate_bounds(&children_borrowed) - .and_then(FFI_Interval::try_from)) -} - -unsafe extern "C" fn propagate_constraints_fn_wrapper( - expr: &FFI_PhysicalExpr, - interval: &FFI_Interval, - children: &RVec, -) -> FFIResult>> { - let expr = expr.inner(); - let interval = rresult_return!(Interval::try_from(interval)); - let children = rresult_return!(children - .iter() - .map(Interval::try_from) - .collect::>>()); - let children_borrowed = children.iter().collect::>(); - - let result = - rresult_return!(expr.propagate_constraints(&interval, &children_borrowed)); - - let result = rresult_return!(result - .map(|intervals| intervals - .into_iter() - .map(FFI_Interval::try_from) - .collect::>>()) - .transpose()); - - RResult::ROk(result.into()) -} - -unsafe extern "C" fn evaluate_statistics_fn_wrapper( - expr: &FFI_PhysicalExpr, - children: &RVec, -) -> FFIResult { - let expr = expr.inner(); - let children = rresult_return!(children - .iter() - .map(Distribution::try_from) - .collect::>>()); - let children_borrowed = children.iter().collect::>(); - rresult!(expr - .evaluate_statistics(&children_borrowed) - .and_then(|dist| FFI_Distribution::try_from(&dist))) -} - -unsafe extern "C" fn propagate_statistics_fn_wrapper( - expr: &FFI_PhysicalExpr, - parent: &FFI_Distribution, - children: &RVec, -) -> FFIResult>> { - let expr = expr.inner(); - let parent = rresult_return!(Distribution::try_from(parent)); - let children = rresult_return!(children - .iter() - .map(Distribution::try_from) - .collect::>>()); - let children_borrowed = children.iter().collect::>(); - - let result = rresult_return!(expr.propagate_statistics(&parent, &children_borrowed)); - let result = rresult_return!(result - .map(|dists| dists - .iter() - .map(FFI_Distribution::try_from) - .collect::>>()) - .transpose()); - - RResult::ROk(result.into()) -} - -unsafe extern "C" fn get_properties_fn_wrapper( - expr: &FFI_PhysicalExpr, - children: &RVec, -) -> FFIResult { - let expr = expr.inner(); - let children = rresult_return!(children - .iter() - .map(ExprProperties::try_from) - .collect::>>()); - rresult!(expr - .get_properties(&children) - .and_then(|p| FFI_ExprProperties::try_from(&p))) -} - -unsafe extern "C" fn fmt_sql_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFIResult { - let expr = expr.inner(); - let result = fmt_sql(expr.as_ref()).to_string(); - RResult::ROk(result.into()) -} - -unsafe extern "C" fn snapshot_fn_wrapper( - expr: &FFI_PhysicalExpr, -) -> FFIResult> { - let expr = expr.inner(); - rresult!(expr - .snapshot() - .map(|snapshot| snapshot.map(FFI_PhysicalExpr::from).into())) -} - -unsafe extern "C" fn snapshot_generation_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 { - let expr = expr.inner(); - expr.snapshot_generation() -} - -unsafe extern "C" fn is_volatile_node_fn_wrapper(expr: &FFI_PhysicalExpr) -> bool { - let expr = expr.inner(); - expr.is_volatile_node() -} -unsafe extern "C" fn display_fn_wrapper(expr: &FFI_PhysicalExpr) -> RString { - let expr = expr.inner(); - format!("{expr}").into() -} - -unsafe extern "C" fn hash_fn_wrapper(expr: &FFI_PhysicalExpr) -> u64 { - let expr = expr.inner(); - let mut hasher = DefaultHasher::new(); - expr.hash(&mut hasher); - hasher.finish() -} - -unsafe extern "C" fn release_fn_wrapper(expr: &mut FFI_PhysicalExpr) { - let private_data = Box::from_raw(expr.private_data as *mut PhysicalExprPrivateData); - drop(private_data); -} - -unsafe extern "C" fn clone_fn_wrapper(expr: &FFI_PhysicalExpr) -> FFI_PhysicalExpr { - let old_private_data = expr.private_data as *const PhysicalExprPrivateData; - - let private_data = Box::into_raw(Box::new(PhysicalExprPrivateData { - expr: Arc::clone(&(*old_private_data).expr), - })) as *mut c_void; - - FFI_PhysicalExpr { - data_type: data_type_fn_wrapper, - nullable: nullable_fn_wrapper, - evaluate: evaluate_fn_wrapper, - return_field: return_field_fn_wrapper, - evaluate_selection: evaluate_selection_fn_wrapper, - children: children_fn_wrapper, - new_with_children: new_with_children_fn_wrapper, - evaluate_bounds: evaluate_bounds_fn_wrapper, - propagate_constraints: propagate_constraints_fn_wrapper, - evaluate_statistics: evaluate_statistics_fn_wrapper, - propagate_statistics: propagate_statistics_fn_wrapper, - get_properties: get_properties_fn_wrapper, - fmt_sql: fmt_sql_fn_wrapper, - snapshot: snapshot_fn_wrapper, - snapshot_generation: snapshot_generation_fn_wrapper, - is_volatile_node: is_volatile_node_fn_wrapper, - display: display_fn_wrapper, - hash: hash_fn_wrapper, - clone: clone_fn_wrapper, - release: release_fn_wrapper, - version: super::version, - private_data, - library_marker_id: crate::get_library_marker_id, - } -} - -impl Drop for FFI_PhysicalExpr { - fn drop(&mut self) { - unsafe { (self.release)(self) } - } -} - -impl From> for FFI_PhysicalExpr { - /// Creates a new [`FFI_PhysicalExpr`]. - fn from(expr: Arc) -> Self { - let private_data = Box::new(PhysicalExprPrivateData { expr }); - - Self { - data_type: data_type_fn_wrapper, - nullable: nullable_fn_wrapper, - evaluate: evaluate_fn_wrapper, - return_field: return_field_fn_wrapper, - evaluate_selection: evaluate_selection_fn_wrapper, - children: children_fn_wrapper, - new_with_children: new_with_children_fn_wrapper, - evaluate_bounds: evaluate_bounds_fn_wrapper, - propagate_constraints: propagate_constraints_fn_wrapper, - evaluate_statistics: evaluate_statistics_fn_wrapper, - propagate_statistics: propagate_statistics_fn_wrapper, - get_properties: get_properties_fn_wrapper, - fmt_sql: fmt_sql_fn_wrapper, - snapshot: snapshot_fn_wrapper, - snapshot_generation: snapshot_generation_fn_wrapper, - is_volatile_node: is_volatile_node_fn_wrapper, - display: display_fn_wrapper, - hash: hash_fn_wrapper, - clone: clone_fn_wrapper, - release: release_fn_wrapper, - version: super::version, - private_data: Box::into_raw(private_data) as *mut c_void, - library_marker_id: crate::get_library_marker_id, - } - } -} - -/// This wrapper struct exists on the receiver side of the FFI interface, so it has -/// no guarantees about being able to access the data in `private_data`. Any functions -/// defined on this struct must only use the stable functions provided in -/// FFI_PhysicalExpr to interact with the foreign table provider. -#[derive(Debug)] -pub struct ForeignPhysicalExpr { - pub expr: FFI_PhysicalExpr, - children: Vec>, -} - -unsafe impl Send for ForeignPhysicalExpr {} -unsafe impl Sync for ForeignPhysicalExpr {} - -impl From for Arc { - fn from(expr: FFI_PhysicalExpr) -> Self { - if (expr.library_marker_id)() == crate::get_library_marker_id() { - // We are built in the same library so safe to access inner member - return Arc::clone(expr.inner()); - } - - let children = unsafe { - (expr.children)(&expr) - .into_iter() - .map(>::from) - .collect() - }; - - let expr = ForeignPhysicalExpr { expr, children }; - - Arc::new(expr) - } -} - -impl Clone for FFI_PhysicalExpr { - fn clone(&self) -> Self { - unsafe { (self.clone)(self) } - } -} - -impl PhysicalExpr for ForeignPhysicalExpr { - fn as_any(&self) -> &dyn Any { - self - } - - fn data_type(&self, input_schema: &Schema) -> Result { - unsafe { - let schema = WrappedSchema::from(Arc::new(input_schema.clone())); - df_result!((self.expr.data_type)(&self.expr, schema)) - .and_then(|d| DataType::try_from(&d.0).map_err(Into::into)) - } - } - - fn nullable(&self, input_schema: &Schema) -> Result { - unsafe { - let schema = WrappedSchema::from(Arc::new(input_schema.clone())); - df_result!((self.expr.nullable)(&self.expr, schema)) - } - } - - fn evaluate(&self, batch: &RecordBatch) -> Result { - unsafe { - let batch = df_result!(record_batch_to_wrapped_array(batch.clone()))?; - df_result!((self.expr.evaluate)(&self.expr, batch)) - .and_then(ColumnarValue::try_from) - } - } - - fn return_field(&self, input_schema: &Schema) -> Result { - unsafe { - let schema = WrappedSchema::from(Arc::new(input_schema.clone())); - let result = df_result!((self.expr.return_field)(&self.expr, schema))?; - Field::try_from(&result.0).map(Arc::new).map_err(Into::into) - } - } - - fn evaluate_selection( - &self, - batch: &RecordBatch, - selection: &BooleanArray, - ) -> Result { - unsafe { - let batch = df_result!(record_batch_to_wrapped_array(batch.clone()))?; - // This is not ideal - we are cloning the selection array - // This is not terrible since it will be a small array. - // The other alternative is to modify the trait signature. - let selection: ArrayRef = Arc::new(selection.clone()); - let selection = WrappedArray::try_from(&selection)?; - df_result!((self.expr.evaluate_selection)(&self.expr, batch, selection)) - .and_then(ColumnarValue::try_from) - } - } - - fn children(&self) -> Vec<&Arc> { - self.children.iter().collect() - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> Result> { - unsafe { - let children = children.into_iter().map(FFI_PhysicalExpr::from).collect(); - df_result!((self.expr.new_with_children)(&self.expr, &children) - .map(>::from)) - } - } - - fn evaluate_bounds(&self, children: &[&Interval]) -> Result { - unsafe { - let children = children - .iter() - .map(|interval| FFI_Interval::try_from(*interval)) - .collect::>>()?; - df_result!((self.expr.evaluate_bounds)(&self.expr, &children)) - .and_then(Interval::try_from) - } - } - - fn propagate_constraints( - &self, - interval: &Interval, - children: &[&Interval], - ) -> Result>> { - unsafe { - let interval = interval.try_into()?; - let children = children - .iter() - .map(|interval| FFI_Interval::try_from(*interval)) - .collect::>>()?; - let result = df_result!((self.expr.propagate_constraints)( - &self.expr, &interval, &children - ))?; - - let result: Option<_> = result - .map(|intervals| { - intervals - .into_iter() - .map(Interval::try_from) - .collect::>>() - }) - .into(); - result.transpose() - } - } - - fn evaluate_statistics(&self, children: &[&Distribution]) -> Result { - unsafe { - let children = children - .iter() - .map(|dist| FFI_Distribution::try_from(*dist)) - .collect::>>()?; - - let result = - df_result!((self.expr.evaluate_statistics)(&self.expr, &children))?; - Distribution::try_from(&result) - } - } - - fn propagate_statistics( - &self, - parent: &Distribution, - children: &[&Distribution], - ) -> Result>> { - unsafe { - let parent = FFI_Distribution::try_from(parent)?; - let children = children - .iter() - .map(|dist| FFI_Distribution::try_from(*dist)) - .collect::>>()?; - let result = df_result!((self.expr.propagate_statistics)( - &self.expr, &parent, &children - ))?; - - let result: Option>> = result - .map(|dists| { - dists - .iter() - .map(Distribution::try_from) - .collect::>>() - }) - .into(); - - result.transpose() - } - } - - fn get_properties(&self, children: &[ExprProperties]) -> Result { - unsafe { - let children = children - .iter() - .map(FFI_ExprProperties::try_from) - .collect::>>()?; - df_result!((self.expr.get_properties)(&self.expr, &children)) - .and_then(|p| ExprProperties::try_from(&p)) - } - } - - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - unsafe { - match (self.expr.fmt_sql)(&self.expr) { - RResult::ROk(sql) => write!(f, "{sql}"), - RResult::RErr(_) => Err(std::fmt::Error), - } - } - } - - fn snapshot(&self) -> Result>> { - unsafe { - let result = df_result!((self.expr.snapshot)(&self.expr))?; - Ok(result.map(>::from).into()) - } - } - - fn snapshot_generation(&self) -> u64 { - unsafe { (self.expr.snapshot_generation)(&self.expr) } - } - - fn is_volatile_node(&self) -> bool { - unsafe { (self.expr.is_volatile_node)(&self.expr) } - } -} - -impl Eq for ForeignPhysicalExpr {} -impl PartialEq for ForeignPhysicalExpr { - fn eq(&self, other: &Self) -> bool { - // FFI_PhysicalExpr cannot be compared, so identity equality is the best we can do. - std::ptr::eq(self, other) - } -} -impl Hash for ForeignPhysicalExpr { - fn hash(&self, state: &mut H) { - let value = unsafe { (self.expr.hash)(&self.expr) }; - value.hash(state) - } -} - -impl Display for ForeignPhysicalExpr { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - let display = unsafe { (self.expr.display)(&self.expr) }; - write!(f, "{display}") - } -} diff --git a/datafusion/ffi/src/physical_expr/partitioning.rs b/datafusion/ffi/src/physical_expr/partitioning.rs deleted file mode 100644 index adf7d9892c72..000000000000 --- a/datafusion/ffi/src/physical_expr/partitioning.rs +++ /dev/null @@ -1,49 +0,0 @@ -use crate::physical_expr::FFI_PhysicalExpr; -use abi_stable::std_types::RVec; -use abi_stable::StableAbi; -use datafusion_physical_expr::{Partitioning, PhysicalExpr}; -use std::sync::Arc; - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub enum FFI_Partitioning { - /// Allocate batches using a round-robin algorithm and the specified number of partitions - RoundRobinBatch(usize), - /// Allocate rows based on a hash of one of more expressions and the specified number of - /// partitions - Hash(RVec, usize), - /// Unknown partitioning scheme with a known number of partitions - UnknownPartitioning(usize), -} - -impl From<&Partitioning> for FFI_Partitioning { - fn from(partitioning: &Partitioning) -> Self { - match partitioning { - Partitioning::UnknownPartitioning(size) => Self::UnknownPartitioning(*size), - Partitioning::RoundRobinBatch(size) => Self::RoundRobinBatch(*size), - Partitioning::Hash(exprs, size) => { - let exprs = exprs.iter().cloned().map(Into::into).collect(); - Self::Hash(exprs, *size) - } - } - } -} - -impl From for Partitioning { - fn from(partitioning: FFI_Partitioning) -> Self { - match partitioning { - FFI_Partitioning::UnknownPartitioning(size) => { - Self::UnknownPartitioning(size) - } - FFI_Partitioning::RoundRobinBatch(size) => Self::RoundRobinBatch(size), - FFI_Partitioning::Hash(exprs, size) => { - let exprs = exprs - .into_iter() - .map(>::from) - .collect(); - Self::Hash(exprs, size) - } - } - } -} diff --git a/datafusion/ffi/src/physical_expr/sort.rs b/datafusion/ffi/src/physical_expr/sort.rs deleted file mode 100644 index 3d67ff4c863c..000000000000 --- a/datafusion/ffi/src/physical_expr/sort.rs +++ /dev/null @@ -1,74 +0,0 @@ -// 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::expr::expr_properties::FFI_SortOptions; -use crate::physical_expr::FFI_PhysicalExpr; -use abi_stable::std_types::RVec; -use abi_stable::StableAbi; -use arrow_schema::SortOptions; -use datafusion_common::{exec_datafusion_err, DataFusionError}; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use std::sync::Arc; - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_PhysicalSortExpr { - pub expr: FFI_PhysicalExpr, - pub options: FFI_SortOptions, -} - -impl From<&PhysicalSortExpr> for FFI_PhysicalSortExpr { - fn from(value: &PhysicalSortExpr) -> Self { - let expr = FFI_PhysicalExpr::from(value.clone().expr); - let options = FFI_SortOptions::from(&value.options); - - Self { expr, options } - } -} - -impl From<&FFI_PhysicalSortExpr> for PhysicalSortExpr { - fn from(value: &FFI_PhysicalSortExpr) -> Self { - let expr = >::from(value.expr.clone()); - let options = SortOptions::from(&value.options); - - Self { expr, options } - } -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_LexOrdering { - pub expr: RVec, -} - -impl From<&LexOrdering> for FFI_LexOrdering { - fn from(value: &LexOrdering) -> Self { - let expr = value.iter().map(FFI_PhysicalSortExpr::from).collect(); - FFI_LexOrdering { expr } - } -} - -impl TryFrom<&FFI_LexOrdering> for LexOrdering { - type Error = DataFusionError; - fn try_from(value: &FFI_LexOrdering) -> Result { - LexOrdering::new(value.expr.iter().map(PhysicalSortExpr::from)) - .ok_or(exec_datafusion_err!("FFI_LexOrdering was empty")) - } -} diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 244fc5901c66..f85659066aea 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -18,17 +18,21 @@ use std::sync::Arc; use crate::arrow_wrappers::WrappedSchema; -use crate::physical_expr::sort::FFI_PhysicalSortExpr; -use crate::physical_expr::FFI_PhysicalExpr; use abi_stable::{ std_types::{RString, RVec}, StableAbi, }; use arrow::{datatypes::Schema, ffi::FFI_ArrowSchema}; use arrow_schema::FieldRef; +use prost::Message; use datafusion_common::error::DataFusionError; +use datafusion_common::exec_datafusion_err; use datafusion_expr::function::AccumulatorArgs; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; +use datafusion_proto::physical_plan::from_proto::{parse_physical_exprs, parse_physical_sort_exprs}; +use datafusion_proto::physical_plan::to_proto::{serialize_physical_exprs, serialize_physical_sort_exprs}; +use datafusion_proto::protobuf::PhysicalAggregateExprNode; /// A stable struct for sharing [`AccumulatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding field @@ -43,8 +47,7 @@ pub struct FFI_AccumulatorArgs { is_distinct: bool, is_reversed: bool, name: RString, - order_bys: RVec, - exprs: RVec, + physical_expr_def: RVec, } impl TryFrom> for FFI_AccumulatorArgs { @@ -55,18 +58,22 @@ impl TryFrom> for FFI_AccumulatorArgs { WrappedSchema(FFI_ArrowSchema::try_from(args.return_field.as_ref())?); let schema = WrappedSchema(FFI_ArrowSchema::try_from(args.schema)?); - let exprs = args - .exprs - .iter() - .map(Arc::clone) - .map(FFI_PhysicalExpr::from) - .collect(); + let codec = DefaultPhysicalExtensionCodec {}; + let ordering_req = + serialize_physical_sort_exprs(args.order_bys.to_owned(), &codec)?; - let order_bys = args - .order_bys - .iter() - .map(FFI_PhysicalSortExpr::from) - .collect(); + let expr = serialize_physical_exprs(args.exprs, &codec)?; + + let physical_expr_def = PhysicalAggregateExprNode { + expr, + ordering_req, + distinct: args.is_distinct, + ignore_nulls: args.ignore_nulls, + fun_definition: None, + aggregate_function: None, + human_display: args.name.to_string(), + }; + let physical_expr_def = physical_expr_def.encode_to_vec().into(); Ok(Self { return_field, @@ -75,8 +82,7 @@ impl TryFrom> for FFI_AccumulatorArgs { ignore_nulls: args.ignore_nulls, is_distinct: args.is_distinct, name: args.name.into(), - order_bys, - exprs, + physical_expr_def, }) } } @@ -101,13 +107,30 @@ impl TryFrom for ForeignAccumulatorArgs { type Error = DataFusionError; fn try_from(value: FFI_AccumulatorArgs) -> Result { - let exprs: Vec> = - value.exprs.into_iter().map(Into::into).collect(); + let proto_def = PhysicalAggregateExprNode::decode( + value.physical_expr_def.as_ref(), + ) + .map_err(|e| { + exec_datafusion_err!("Failed to decode PhysicalAggregateExprNode: {e}") + })?; let return_field = Arc::new((&value.return_field.0).try_into()?); let schema = Schema::try_from(&value.schema.0)?; - let order_bys = value.order_bys.iter().map(PhysicalSortExpr::from).collect(); + let task_ctx = default_ctx.task_ctx(); + let codex = DefaultPhysicalExtensionCodec {}; + + let order_bys = parse_physical_sort_exprs( + &proto_def.ordering_req, + &task_ctx, + &schema, + &codex, + )?; + + let exprs = parse_physical_exprs(&proto_def.expr, &task_ctx, &schema, &codex)?; + + let return_field = Arc::new((&value.return_field.0).try_into()?); + let schema = Schema::try_from(&value.schema.0)?; let expr_fields = exprs .iter() diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index 2030cf0c304c..13cba77c8081 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -18,7 +18,6 @@ use std::{collections::HashMap, sync::Arc}; use crate::arrow_wrappers::WrappedSchema; -use crate::physical_expr::FFI_PhysicalExpr; use abi_stable::{std_types::RVec, StableAbi}; use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef}, @@ -26,9 +25,15 @@ use arrow::{ ffi::FFI_ArrowSchema, }; use arrow_schema::FieldRef; +use prost::Message; use datafusion_common::error::{DataFusionError, Result}; +use datafusion_common::exec_datafusion_err; use datafusion_expr::function::PartitionEvaluatorArgs; use datafusion_physical_plan::{expressions::Column, PhysicalExpr}; +use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; +use datafusion_proto::physical_plan::from_proto::parse_physical_expr; +use datafusion_proto::physical_plan::to_proto::serialize_physical_exprs; +use datafusion_proto::protobuf::PhysicalExprNode; /// A stable struct for sharing [`PartitionEvaluatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding function @@ -37,7 +42,7 @@ use datafusion_physical_plan::{expressions::Column, PhysicalExpr}; #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] pub struct FFI_PartitionEvaluatorArgs { - input_exprs: RVec, + input_exprs: RVec>, input_fields: RVec, is_reversed: bool, ignore_nulls: bool, @@ -85,11 +90,10 @@ impl TryFrom> for FFI_PartitionEvaluatorArgs { let schema = Arc::new(Schema::new(fields)); - let input_exprs = args - .input_exprs() - .iter() - .map(Arc::clone) - .map(FFI_PhysicalExpr::from) + let codec = DefaultPhysicalExtensionCodec {}; + let input_exprs = serialize_physical_exprs(args.input_exprs(), &codec)? + .into_iter() + .map(|expr_node| expr_node.encode_to_vec().into()) .collect(); let input_fields = args @@ -126,13 +130,21 @@ impl TryFrom for ForeignPartitionEvaluatorArgs { type Error = DataFusionError; fn try_from(value: FFI_PartitionEvaluatorArgs) -> Result { + let codec = DefaultPhysicalExtensionCodec {}; + let schema: SchemaRef = value.schema.into(); let input_exprs = value .input_exprs .into_iter() - .map(>::from) - .collect::>(); + .map(|input_expr_bytes| PhysicalExprNode::decode(input_expr_bytes.as_ref())) + .collect::, prost::DecodeError>>() + .map_err(|e| exec_datafusion_err!("Failed to decode PhysicalExprNode: {e}"))? + .iter() + .map(|expr_node| { + parse_physical_expr(expr_node, &default_ctx.task_ctx(), &schema, &codec) + }) + .collect::>>()?; let input_fields = input_exprs .iter() From 2d568db628948b568b1ef0e4e8b3de527fcd1d2b Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 14:30:03 -0500 Subject: [PATCH 46/69] More FFI PhysicalExpr removal --- datafusion/ffi/src/plan_properties.rs | 92 +++++++++++++++++++++------ 1 file changed, 71 insertions(+), 21 deletions(-) diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index b51a75d717b7..217518c5a11e 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -18,11 +18,10 @@ use std::{ffi::c_void, sync::Arc}; use crate::arrow_wrappers::WrappedSchema; -use crate::physical_expr::partitioning::FFI_Partitioning; -use crate::physical_expr::sort::FFI_LexOrdering; -use abi_stable::std_types::ROption; +use abi_stable::std_types::{ROption, RResult, RString, RVec}; use abi_stable::StableAbi; use arrow::datatypes::SchemaRef; +use prost::Message; use datafusion_common::error::{DataFusionError, Result}; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr::LexOrdering; @@ -30,6 +29,11 @@ use datafusion_physical_plan::{ execution_plan::{Boundedness, EmissionType}, PlanProperties, }; +use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; +use datafusion_proto::physical_plan::from_proto::{parse_physical_sort_exprs, parse_protobuf_partitioning}; +use datafusion_proto::physical_plan::to_proto::{serialize_partitioning, serialize_physical_sort_exprs}; +use datafusion_proto::protobuf::{Partitioning, PhysicalSortExprNodeCollection}; +use crate::{df_result, rresult_return}; /// A stable struct for sharing [`PlanProperties`] across FFI boundaries. #[repr(C)] @@ -38,7 +42,7 @@ use datafusion_physical_plan::{ pub struct FFI_PlanProperties { /// The output partitioning is a [`Partitioning`] protobuf message serialized /// into bytes to pass across the FFI boundary. - pub output_partitioning: unsafe extern "C" fn(plan: &Self) -> FFI_Partitioning, + pub output_partitioning: unsafe extern "C" fn(plan: &Self) -> RResult, RString>, /// Return the emission type of the plan. pub emission_type: unsafe extern "C" fn(plan: &Self) -> FFI_EmissionType, @@ -47,7 +51,7 @@ pub struct FFI_PlanProperties { pub boundedness: unsafe extern "C" fn(plan: &Self) -> FFI_Boundedness, /// The output ordering of the plan. - pub output_ordering: unsafe extern "C" fn(plan: &Self) -> ROption, + pub output_ordering: unsafe extern "C" fn(plan: &Self) -> RResult, RString>, /// Return the schema of the plan. pub schema: unsafe extern "C" fn(plan: &Self) -> WrappedSchema, @@ -77,8 +81,16 @@ impl FFI_PlanProperties { unsafe extern "C" fn output_partitioning_fn_wrapper( properties: &FFI_PlanProperties, -) -> FFI_Partitioning { - (&properties.inner().partitioning).into() +) -> RResult, RString> { + let private_data = properties.private_data as *const PlanPropertiesPrivateData; + let props = &(*private_data).props; + + let codec = DefaultPhysicalExtensionCodec {}; + let partitioning_data = + rresult_return!(serialize_partitioning(props.output_partitioning(), &codec)); + let output_partitioning = partitioning_data.encode_to_vec(); + + RResult::ROk(output_partitioning.into()) } unsafe extern "C" fn emission_type_fn_wrapper( @@ -95,12 +107,25 @@ unsafe extern "C" fn boundedness_fn_wrapper( unsafe extern "C" fn output_ordering_fn_wrapper( properties: &FFI_PlanProperties, -) -> ROption { - properties - .inner() - .output_ordering() - .map(FFI_LexOrdering::from) - .into() +) -> RResult, RString> { + let private_data = properties.private_data as *const PlanPropertiesPrivateData; + let props = &(*private_data).props; + + let codec = DefaultPhysicalExtensionCodec {}; + let output_ordering = match props.output_ordering() { + Some(ordering) => { + let physical_sort_expr_nodes = rresult_return!( + serialize_physical_sort_exprs(ordering.to_owned(), &codec) + ); + let ordering_data = PhysicalSortExprNodeCollection { + physical_sort_expr_nodes, + }; + + ordering_data.encode_to_vec() + } + None => Vec::default(), + }; + RResult::ROk(output_ordering.into()) } unsafe extern "C" fn schema_fn_wrapper(properties: &FFI_PlanProperties) -> WrappedSchema { @@ -150,16 +175,41 @@ impl TryFrom for PlanProperties { let ffi_schema = unsafe { (ffi_props.schema)(&ffi_props) }; let schema = (&ffi_schema.0).try_into()?; - let ffi_orderings = unsafe { (ffi_props.output_ordering)(&ffi_props) }; + let task_context = default_ctx.task_ctx(); + let codex = DefaultPhysicalExtensionCodec {}; - let partitioning = unsafe { (ffi_props.output_partitioning)(&ffi_props) }.into(); + let ffi_orderings = unsafe { (ffi_props.output_ordering)(&ffi_props) }; - let eq_properties = match ffi_orderings { - ROption::RSome(lex_ordering) => { - let ordering = LexOrdering::try_from(&lex_ordering)?; - EquivalenceProperties::new_with_orderings(Arc::new(schema), [ordering]) - } - ROption::RNone => EquivalenceProperties::new(Arc::new(schema)), + let proto_output_ordering = + PhysicalSortExprNodeCollection::decode(df_result!(ffi_orderings)?.as_ref()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + let sort_exprs = parse_physical_sort_exprs( + &proto_output_ordering.physical_sort_expr_nodes, + &task_context, + &schema, + &codex, + )?; + + let partitioning_vec = + unsafe { df_result!((ffi_props.output_partitioning)(&ffi_props))? }; + let proto_output_partitioning = + Partitioning::decode(partitioning_vec.as_ref()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + let partitioning = parse_protobuf_partitioning( + Some(&proto_output_partitioning), + &task_context, + &schema, + &codex, + )? + .ok_or(DataFusionError::Plan( + "Unable to deserialize partitioning protobuf in FFI_PlanProperties" + .to_string(), + ))?; + + let eq_properties = if sort_exprs.is_empty() { + EquivalenceProperties::new(Arc::new(schema)) + } else { + EquivalenceProperties::new_with_orderings(Arc::new(schema), [sort_exprs]) }; let emission_type: EmissionType = From b1837a5e16c8d92a328c3a600d7c04228eba4ba3 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Sun, 9 Nov 2025 22:52:50 -0500 Subject: [PATCH 47/69] Plumbing through task ctx accessor --- .../ffi/ffi_example_table_provider/src/lib.rs | 4 +- .../ffi/ffi_module_interface/src/lib.rs | 3 +- datafusion/core/src/execution/context/mod.rs | 7 + datafusion/execution/src/lib.rs | 2 +- datafusion/execution/src/task.rs | 5 + datafusion/ffi/src/catalog_provider.rs | 14 +- datafusion/ffi/src/execution_plan.rs | 45 +++-- datafusion/ffi/src/plan_properties.rs | 21 ++- datafusion/ffi/src/schema_provider.rs | 19 ++- datafusion/ffi/src/session/mod.rs | 19 ++- .../src/session/{task.rs => task_context.rs} | 11 +- .../ffi/src/session/task_ctx_accessor.rs | 155 ++++++++++++++++++ datafusion/ffi/src/table_provider.rs | 30 ++-- datafusion/ffi/src/tests/async_provider.rs | 3 + datafusion/ffi/src/tests/catalog.rs | 4 +- datafusion/ffi/src/tests/mod.rs | 14 +- datafusion/ffi/src/tests/sync_provider.rs | 10 +- datafusion/ffi/src/tests/udf_udaf_udwf.rs | 8 +- datafusion/ffi/src/udtf.rs | 13 +- datafusion/ffi/src/udwf/mod.rs | 28 ++-- .../ffi/src/udwf/partition_evaluator_args.rs | 12 +- 21 files changed, 342 insertions(+), 85 deletions(-) rename datafusion/ffi/src/session/{task.rs => task_context.rs} (94%) create mode 100644 datafusion/ffi/src/session/task_ctx_accessor.rs diff --git a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs index c8362d1993c1..57b17e169029 100644 --- a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs @@ -22,6 +22,7 @@ use arrow::array::RecordBatch; use arrow::datatypes::{DataType, Field, Schema}; use datafusion::{common::record_batch, datasource::MemTable}; use datafusion_ffi::function_registry::FFI_WeakFunctionRegistry; +use datafusion_ffi::session::task_ctx_accessor::FFI_TaskContextAccessor; use datafusion_ffi::table_provider::FFI_TableProvider; use ffi_module_interface::{TableProviderModule, TableProviderModuleRef}; @@ -37,6 +38,7 @@ fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { /// We create an in-memory table and convert it to it's FFI counterpart. extern "C" fn construct_simple_table_provider( function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), @@ -53,7 +55,7 @@ extern "C" fn construct_simple_table_provider( let table_provider = MemTable::try_new(schema, vec![batches]).unwrap(); - FFI_TableProvider::new(Arc::new(table_provider), true, None, function_registry) + FFI_TableProvider::new(Arc::new(table_provider), true, None, function_registry, task_ctx_accessor) } #[export_root_module] diff --git a/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs b/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs index 4b277eb0ed5c..943a9eb1f7b0 100644 --- a/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs @@ -23,6 +23,7 @@ use abi_stable::{ StableAbi, }; use datafusion_ffi::function_registry::FFI_WeakFunctionRegistry; +use datafusion_ffi::session::task_ctx_accessor::FFI_TaskContextAccessor; use datafusion_ffi::table_provider::FFI_TableProvider; #[repr(C)] @@ -35,7 +36,7 @@ use datafusion_ffi::table_provider::FFI_TableProvider; /// how a user may wish to separate these concerns. pub struct TableProviderModule { /// Constructs the table provider - pub create_table: extern "C" fn(FFI_WeakFunctionRegistry) -> FFI_TableProvider, + pub create_table: extern "C" fn(FFI_WeakFunctionRegistry, FFI_TaskContextAccessor) -> FFI_TableProvider, } impl RootModule for TableProviderModuleRef { diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c732c2c92f64..be100709349d 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -94,6 +94,7 @@ use chrono::{DateTime, Utc}; use object_store::ObjectStore; use parking_lot::RwLock; use url::Url; +use datafusion_execution::TaskContextAccessor; mod csv; mod json; @@ -1794,6 +1795,12 @@ impl FunctionRegistry for SessionContext { } } +impl TaskContextAccessor for SessionContext { + fn get_task_context(&self) -> Arc { + self.task_ctx() + } +} + /// Create a new task context instance from SessionContext impl From<&SessionContext> for TaskContext { fn from(session: &SessionContext) -> Self { diff --git a/datafusion/execution/src/lib.rs b/datafusion/execution/src/lib.rs index 55243e301e0e..b0a4dd0afc37 100644 --- a/datafusion/execution/src/lib.rs +++ b/datafusion/execution/src/lib.rs @@ -46,4 +46,4 @@ pub mod registry { pub use disk_manager::DiskManager; pub use registry::FunctionRegistry; pub use stream::{RecordBatchStream, SendableRecordBatchStream}; -pub use task::TaskContext; +pub use task::{TaskContext, TaskContextAccessor}; diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index c8ee4f0823ec..3af03fd14702 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -211,6 +211,11 @@ impl FunctionRegistry for TaskContext { } } +pub trait TaskContextAccessor { + fn get_task_context(&self) -> Arc; +} + + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index 1d33aa5e13c0..e9752a1c4436 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -31,6 +31,7 @@ use crate::{ use crate::function_registry::FFI_WeakFunctionRegistry; use datafusion_common::error::Result; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`CatalogProvider`] across FFI boundaries. #[repr(C)] @@ -59,6 +60,7 @@ pub struct FFI_CatalogProvider { ) -> RResult, RString>, function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. @@ -117,6 +119,7 @@ unsafe extern "C" fn schema_fn_wrapper( schema, provider.runtime(), provider.function_registry.clone(), + provider.task_ctx_accessor.clone(), ) }) .into() @@ -134,7 +137,7 @@ unsafe extern "C" fn register_schema_fn_wrapper( .inner() .register_schema(name.as_str(), schema)) .map(|schema| { - FFI_SchemaProvider::new(schema, runtime, provider.function_registry.clone()) + FFI_SchemaProvider::new(schema, runtime, provider.function_registry.clone(), provider.task_ctx_accessor.clone()) }) .into(); @@ -158,6 +161,7 @@ unsafe extern "C" fn deregister_schema_fn_wrapper( schema, runtime, provider.function_registry.clone(), + provider.task_ctx_accessor.clone(), ) }) .into(), @@ -186,6 +190,7 @@ unsafe extern "C" fn clone_fn_wrapper( register_schema: register_schema_fn_wrapper, deregister_schema: deregister_schema_fn_wrapper, function_registry: provider.function_registry.clone(), + task_ctx_accessor: provider.task_ctx_accessor.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -206,6 +211,7 @@ impl FFI_CatalogProvider { provider: Arc, runtime: Option, function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -215,6 +221,7 @@ impl FFI_CatalogProvider { register_schema: register_schema_fn_wrapper, deregister_schema: deregister_schema_fn_wrapper, function_registry, + task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -289,6 +296,7 @@ impl CatalogProvider for ForeignCatalogProvider { schema, None, self.0.function_registry.clone(), + self.0.task_ctx_accessor.clone(), ), }; let returned_schema: Option = @@ -323,6 +331,7 @@ mod tests { use super::*; use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; use datafusion::prelude::SessionContext; + use datafusion_execution::TaskContextAccessor; use datafusion_expr::registry::FunctionRegistry; #[test] @@ -339,9 +348,10 @@ mod tests { let ctx = Arc::new(SessionContext::new()); let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let mut ffi_catalog = - FFI_CatalogProvider::new(catalog, None, function_registry.into()); + FFI_CatalogProvider::new(catalog, None, function_registry.into(), task_ctx_accessor.into()); ffi_catalog.library_marker_id = crate::mock_foreign_marker_id; let foreign_catalog: Arc = (&ffi_catalog).into(); diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index d4e8b9a795fc..9e345bfafa65 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -17,10 +17,7 @@ use std::{ffi::c_void, pin::Pin, sync::Arc}; -use crate::{ - df_result, plan_properties::FFI_PlanProperties, - record_batch_stream::FFI_RecordBatchStream, rresult, -}; +use crate::{df_result, plan_properties::FFI_PlanProperties, record_batch_stream::FFI_RecordBatchStream, rresult, rresult_return}; use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, @@ -30,6 +27,7 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_plan::DisplayFormatType; use datafusion_physical_plan::{DisplayAs, ExecutionPlan, PlanProperties}; use tokio::runtime::Handle; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing a [`ExecutionPlan`] across FFI boundaries. #[repr(C)] @@ -52,6 +50,8 @@ pub struct FFI_ExecutionPlan { partition: usize, ) -> RResult, + pub task_ctx_accessor: FFI_TaskContextAccessor, + /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -73,7 +73,6 @@ unsafe impl Sync for FFI_ExecutionPlan {} pub struct ExecutionPlanPrivateData { pub plan: Arc, - pub context: Arc, pub runtime: Option, } @@ -87,24 +86,22 @@ impl FFI_ExecutionPlan { unsafe extern "C" fn properties_fn_wrapper( plan: &FFI_ExecutionPlan, ) -> FFI_PlanProperties { - let plan = plan.inner(); - - plan.properties().into() + FFI_PlanProperties::new(plan.inner().properties(), plan.task_ctx_accessor.clone()) } unsafe extern "C" fn children_fn_wrapper( plan: &FFI_ExecutionPlan, ) -> RVec { + let ctx = &plan.task_ctx_accessor; let private_data = plan.private_data as *const ExecutionPlanPrivateData; let plan = &(*private_data).plan; - let ctx = &(*private_data).context; let runtime = &(*private_data).runtime; let children: Vec<_> = plan .children() .into_iter() .map(|child| { - FFI_ExecutionPlan::new(Arc::clone(child), Arc::clone(ctx), runtime.clone()) + FFI_ExecutionPlan::new(Arc::clone(child), ctx.clone(), runtime.clone()) }) .collect(); @@ -115,13 +112,13 @@ unsafe extern "C" fn execute_fn_wrapper( plan: &FFI_ExecutionPlan, partition: usize, ) -> RResult { + let ctx = rresult_return!(>::try_from(&plan.task_ctx_accessor)); let private_data = plan.private_data as *const ExecutionPlanPrivateData; let plan = &(*private_data).plan; - let ctx = &(*private_data).context; let runtime = (*private_data).runtime.clone(); rresult!(plan - .execute(partition, Arc::clone(ctx)) + .execute(partition, ctx) .map(|rbs| FFI_RecordBatchStream::new(rbs, runtime))) } @@ -135,12 +132,13 @@ unsafe extern "C" fn release_fn_wrapper(plan: &mut FFI_ExecutionPlan) { } unsafe extern "C" fn clone_fn_wrapper(plan: &FFI_ExecutionPlan) -> FFI_ExecutionPlan { + let ctx = plan.task_ctx_accessor.clone(); let private_data = plan.private_data as *const ExecutionPlanPrivateData; let plan_data = &(*private_data); FFI_ExecutionPlan::new( Arc::clone(&plan_data.plan), - Arc::clone(&plan_data.context), + ctx, plan_data.runtime.clone(), ) } @@ -155,12 +153,11 @@ impl FFI_ExecutionPlan { /// This function is called on the provider's side. pub fn new( plan: Arc, - context: Arc, + context: FFI_TaskContextAccessor, runtime: Option, ) -> Self { let private_data = Box::new(ExecutionPlanPrivateData { plan, - context, runtime, }); @@ -169,6 +166,7 @@ impl FFI_ExecutionPlan { children: children_fn_wrapper, name: name_fn_wrapper, execute: execute_fn_wrapper, + task_ctx_accessor: context, clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, @@ -308,7 +306,7 @@ mod tests { }, prelude::SessionContext, }; - + use datafusion_execution::TaskContextAccessor; use super::*; #[derive(Debug)] @@ -385,12 +383,12 @@ mod tests { fn test_round_trip_ffi_execution_plan() -> Result<()> { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, false)])); - let ctx = SessionContext::new(); + let ctx = Arc::new(SessionContext::new()) as Arc; let original_plan = Arc::new(EmptyExec::new(schema)); let original_name = original_plan.name().to_string(); - let mut local_plan = FFI_ExecutionPlan::new(original_plan, ctx.task_ctx(), None); + let mut local_plan = FFI_ExecutionPlan::new(original_plan, ctx.into(), None); // Force round trip to go through foreign provider local_plan.library_marker_id = crate::mock_foreign_marker_id; @@ -416,16 +414,17 @@ mod tests { fn test_ffi_execution_plan_children() -> Result<()> { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, false)])); - let ctx = SessionContext::new(); + let ctx = Arc::new(SessionContext::new()) as Arc; + let ctx = FFI_TaskContextAccessor::from(ctx); // Version 1: Adding child to the foreign plan let child_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); - let mut child_local = FFI_ExecutionPlan::new(child_plan, ctx.task_ctx(), None); + let mut child_local = FFI_ExecutionPlan::new(child_plan, ctx.clone(), None); child_local.library_marker_id = crate::mock_foreign_marker_id; let child_foreign = >::try_from(&child_local)?; let parent_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); - let parent_local = FFI_ExecutionPlan::new(parent_plan, ctx.task_ctx(), None); + let parent_local = FFI_ExecutionPlan::new(parent_plan, ctx.clone(), None); let parent_foreign = >::try_from(&parent_local)?; assert_eq!(parent_foreign.children().len(), 0); @@ -436,12 +435,12 @@ mod tests { // Version 2: Adding child to the local plan let child_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); - let child_local = FFI_ExecutionPlan::new(child_plan, ctx.task_ctx(), None); + let child_local = FFI_ExecutionPlan::new(child_plan, ctx.clone(), None); let child_foreign = >::try_from(&child_local)?; let parent_plan = Arc::new(EmptyExec::new(Arc::clone(&schema))); let parent_plan = parent_plan.with_new_children(vec![child_foreign])?; - let parent_local = FFI_ExecutionPlan::new(parent_plan, ctx.task_ctx(), None); + let parent_local = FFI_ExecutionPlan::new(parent_plan, ctx, None); let parent_foreign = >::try_from(&parent_local)?; assert_eq!(parent_foreign.children().len(), 1); diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index 217518c5a11e..0f189a1b42c2 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -23,6 +23,7 @@ use abi_stable::StableAbi; use arrow::datatypes::SchemaRef; use prost::Message; use datafusion_common::error::{DataFusionError, Result}; +use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::{ @@ -34,6 +35,7 @@ use datafusion_proto::physical_plan::from_proto::{parse_physical_sort_exprs, par use datafusion_proto::physical_plan::to_proto::{serialize_partitioning, serialize_physical_sort_exprs}; use datafusion_proto::protobuf::{Partitioning, PhysicalSortExprNodeCollection}; use crate::{df_result, rresult_return}; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`PlanProperties`] across FFI boundaries. #[repr(C)] @@ -56,6 +58,8 @@ pub struct FFI_PlanProperties { /// Return the schema of the plan. pub schema: unsafe extern "C" fn(plan: &Self) -> WrappedSchema, + pub task_ctx_accessor: FFI_TaskContextAccessor, + /// Release the memory of the private data when it is no longer being used. pub release: unsafe extern "C" fn(arg: &mut Self), @@ -145,8 +149,8 @@ impl Drop for FFI_PlanProperties { } } -impl From<&PlanProperties> for FFI_PlanProperties { - fn from(props: &PlanProperties) -> Self { +impl FFI_PlanProperties { + pub fn new(props: &PlanProperties, task_ctx_accessor: FFI_TaskContextAccessor) -> Self { let private_data = Box::new(PlanPropertiesPrivateData { props: props.clone(), }); @@ -157,6 +161,7 @@ impl From<&PlanProperties> for FFI_PlanProperties { boundedness: boundedness_fn_wrapper, output_ordering: output_ordering_fn_wrapper, schema: schema_fn_wrapper, + task_ctx_accessor, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, library_marker_id: crate::get_library_marker_id, @@ -175,7 +180,7 @@ impl TryFrom for PlanProperties { let ffi_schema = unsafe { (ffi_props.schema)(&ffi_props) }; let schema = (&ffi_schema.0).try_into()?; - let task_context = default_ctx.task_ctx(); + let task_ctx: Arc = (&ffi_props.task_ctx_accessor).try_into()?; let codex = DefaultPhysicalExtensionCodec {}; let ffi_orderings = unsafe { (ffi_props.output_ordering)(&ffi_props) }; @@ -185,7 +190,7 @@ impl TryFrom for PlanProperties { .map_err(|e| DataFusionError::External(Box::new(e)))?; let sort_exprs = parse_physical_sort_exprs( &proto_output_ordering.physical_sort_expr_nodes, - &task_context, + &task_ctx, &schema, &codex, )?; @@ -197,7 +202,7 @@ impl TryFrom for PlanProperties { .map_err(|e| DataFusionError::External(Box::new(e)))?; let partitioning = parse_protobuf_partitioning( Some(&proto_output_partitioning), - &task_context, + &task_ctx, &schema, &codex, )? @@ -295,7 +300,8 @@ impl From for EmissionType { #[cfg(test)] mod tests { use datafusion::{physical_expr::PhysicalSortExpr, physical_plan::Partitioning}; - + use datafusion::prelude::SessionContext; + use datafusion_execution::TaskContextAccessor; use super::*; #[test] @@ -314,8 +320,9 @@ mod tests { EmissionType::Incremental, Boundedness::Bounded, ); + let ctx = Arc::new(SessionContext::default()) as Arc; - let mut local_props_ptr = FFI_PlanProperties::from(&original_props); + let mut local_props_ptr = FFI_PlanProperties::new(&original_props, (&ctx).into()); local_props_ptr.library_marker_id = crate::mock_foreign_marker_id; let foreign_props: PlanProperties = local_props_ptr.try_into()?; diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index fa9bcfaf7749..0a5991e0820f 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -32,6 +32,7 @@ use crate::{ df_result, rresult_return, table_provider::{FFI_TableProvider, ForeignTableProvider}, }; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`SchemaProvider`] across FFI boundaries. #[repr(C)] @@ -65,6 +66,7 @@ pub struct FFI_SchemaProvider { pub table_exist: unsafe extern "C" fn(provider: &Self, name: RString) -> bool, pub function_registry: FFI_WeakFunctionRegistry, + pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. @@ -119,12 +121,13 @@ unsafe extern "C" fn table_fn_wrapper( name: RString, ) -> FfiFuture, RString>> { let function_registry = provider.function_registry.clone(); + let task_ctx_accessor = provider.task_ctx_accessor.clone(); let runtime = provider.runtime(); let provider = Arc::clone(provider.inner()); async move { let table = rresult_return!(provider.table(name.as_str()).await) - .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry)) + .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry, task_ctx_accessor)) .into(); RResult::ROk(table) @@ -139,12 +142,13 @@ unsafe extern "C" fn register_table_fn_wrapper( ) -> RResult, RString> { let runtime = provider.runtime(); let function_registry = provider.function_registry.clone(); + let task_ctx_accessor = provider.task_ctx_accessor.clone(); let provider = provider.inner(); let table = Arc::new(ForeignTableProvider(table)); let returned_table = rresult_return!(provider.register_table(name.into(), table)) - .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry)); + .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry, task_ctx_accessor)); RResult::ROk(returned_table.into()) } @@ -154,11 +158,12 @@ unsafe extern "C" fn deregister_table_fn_wrapper( name: RString, ) -> RResult, RString> { let function_registry = provider.function_registry.clone(); + let task_ctx_accessor = provider.task_ctx_accessor.clone(); let runtime = provider.runtime(); let provider = provider.inner(); let returned_table = rresult_return!(provider.deregister_table(name.as_str())) - .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry)); + .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry, task_ctx_accessor)); RResult::ROk(returned_table.into()) } @@ -198,6 +203,7 @@ unsafe extern "C" fn clone_fn_wrapper( deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, function_registry: provider.function_registry.clone(), + task_ctx_accessor: provider.task_ctx_accessor.clone(), library_marker_id: crate::get_library_marker_id, } } @@ -214,6 +220,7 @@ impl FFI_SchemaProvider { provider: Arc, runtime: Option, function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let owner_name = provider.owner_name().map(|s| s.into()).into(); let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -230,6 +237,7 @@ impl FFI_SchemaProvider { deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, function_registry, + task_ctx_accessor, library_marker_id: crate::get_library_marker_id, } } @@ -308,6 +316,7 @@ impl SchemaProvider for ForeignSchemaProvider { true, None, self.0.function_registry.clone(), + self.0.task_ctx_accessor.clone(), ), }; @@ -341,6 +350,7 @@ mod tests { use arrow::datatypes::Schema; use datafusion::prelude::SessionContext; use datafusion::{catalog::MemorySchemaProvider, datasource::empty::EmptyTable}; + use datafusion_execution::TaskContextAccessor; use datafusion_expr::registry::FunctionRegistry; fn empty_table() -> Arc { @@ -358,9 +368,10 @@ mod tests { let ctx = Arc::new(SessionContext::new()); let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let mut ffi_schema_provider = - FFI_SchemaProvider::new(schema_provider, None, function_registry.into()); + FFI_SchemaProvider::new(schema_provider, None, function_registry.into(), task_ctx_accessor.into()); ffi_schema_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_schema_provider: Arc = diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 99aeaa5de069..3d74c6db32b4 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -19,7 +19,7 @@ use crate::arrow_wrappers::WrappedSchema; use crate::execution_plan::FFI_ExecutionPlan; use crate::function_registry::FFI_WeakFunctionRegistry; use crate::session::config::FFI_SessionConfig; -use crate::session::task::FFI_TaskContext; +use crate::session::task_context::FFI_TaskContext; use crate::udaf::FFI_AggregateUDF; use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; @@ -60,9 +60,11 @@ use std::any::Any; use std::collections::HashMap; use std::{ffi::c_void, sync::Arc}; use tokio::runtime::Handle; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; pub mod config; -mod task; +pub mod task_context; +pub mod task_ctx_accessor; /// A stable struct for sharing [`Session`] across FFI boundaries. #[repr(C)] @@ -103,6 +105,7 @@ pub struct FFI_Session { pub task_ctx: unsafe extern "C" fn(&Self) -> FFI_TaskContext, pub function_registry: FFI_WeakFunctionRegistry, + pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the registry. This should /// only need to be called by the receiver of the plan. @@ -163,6 +166,7 @@ unsafe extern "C" fn create_physical_plan_fn_wrapper( session: &FFI_Session, logical_plan_serialized: RVec, ) -> FfiFuture> { + let task_ctx_accessor = session.task_ctx_accessor.clone(); let runtime = session.runtime().clone(); let session = session.clone(); async move { @@ -178,7 +182,7 @@ unsafe extern "C" fn create_physical_plan_fn_wrapper( rresult!(physical_plan.map(|plan| FFI_ExecutionPlan::new( plan, - task_ctx, + task_ctx_accessor, runtime ))) } @@ -239,11 +243,12 @@ unsafe extern "C" fn aggregate_functions_fn_wrapper( unsafe extern "C" fn window_functions_fn_wrapper( session: &FFI_Session, ) -> RHashMap { + let task_ctx_accessor = &session.task_ctx_accessor; let session = session.inner(); session .window_functions() .iter() - .map(|(name, udwf)| (name.clone().into(), FFI_WindowUDF::from(Arc::clone(udwf)))) + .map(|(name, udwf)| (name.clone().into(), FFI_WindowUDF::new(Arc::clone(udwf), task_ctx_accessor.clone()))) .collect() } @@ -273,8 +278,9 @@ unsafe extern "C" fn default_table_options_fn_wrapper( } unsafe extern "C" fn task_ctx_fn_wrapper(session: &FFI_Session) -> FFI_TaskContext { + let task_ctx_accessor = session.task_ctx_accessor.clone(); let session = session.inner(); - FFI_TaskContext::from(session.task_ctx()) + FFI_TaskContext::new(session.task_ctx(), task_ctx_accessor) } unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_Session) { @@ -302,6 +308,7 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_Session) -> FFI_Session { default_table_options: default_table_options_fn_wrapper, task_ctx: task_ctx_fn_wrapper, function_registry: provider.function_registry.clone(), + task_ctx_accessor: provider.task_ctx_accessor.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, @@ -322,6 +329,7 @@ impl FFI_Session { pub fn new( session: &(dyn Session + Send + Sync), function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, runtime: Option, ) -> Self { let private_data = Box::new(SessionPrivateData { session, runtime }); @@ -338,6 +346,7 @@ impl FFI_Session { default_table_options: default_table_options_fn_wrapper, task_ctx: task_ctx_fn_wrapper, function_registry, + task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, diff --git a/datafusion/ffi/src/session/task.rs b/datafusion/ffi/src/session/task_context.rs similarity index 94% rename from datafusion/ffi/src/session/task.rs rename to datafusion/ffi/src/session/task_context.rs index e61879da5a13..285e4c44074b 100644 --- a/datafusion/ffi/src/session/task.rs +++ b/datafusion/ffi/src/session/task_context.rs @@ -29,6 +29,7 @@ use datafusion_expr::{ AggregateUDF, AggregateUDFImpl, ScalarUDF, ScalarUDFImpl, WindowUDF, WindowUDFImpl, }; use std::{ffi::c_void, sync::Arc}; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`TaskContext`] across FFI boundaries. #[repr(C)] @@ -48,6 +49,8 @@ pub struct FFI_TaskContext { pub window_functions: unsafe extern "C" fn(&Self) -> RHashMap, + pub task_ctx_accessor: FFI_TaskContextAccessor, + /// Release the memory of the private data when it is no longer being used. pub release: unsafe extern "C" fn(arg: &mut Self), @@ -111,10 +114,11 @@ unsafe extern "C" fn aggregate_functions_fn_wrapper( unsafe extern "C" fn window_functions_fn_wrapper( ctx: &FFI_TaskContext, ) -> RHashMap { + let task_ctx_accessor = &ctx.task_ctx_accessor; let ctx = ctx.inner(); ctx.window_functions() .iter() - .map(|(name, udf)| (name.to_owned().into(), udf.into())) + .map(|(name, udf)| (name.to_owned().into(), FFI_WindowUDF::new(Arc::clone(udf), task_ctx_accessor.clone()))) .collect() } @@ -129,8 +133,8 @@ impl Drop for FFI_TaskContext { } } -impl From> for FFI_TaskContext { - fn from(ctx: Arc) -> Self { +impl FFI_TaskContext { + pub fn new(ctx: Arc, task_ctx_accessor: FFI_TaskContextAccessor) -> Self { let private_data = Box::new(TaskContextPrivateData { ctx }); FFI_TaskContext { @@ -140,6 +144,7 @@ impl From> for FFI_TaskContext { scalar_functions: scalar_functions_fn_wrapper, aggregate_functions: aggregate_functions_fn_wrapper, window_functions: window_functions_fn_wrapper, + task_ctx_accessor, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, library_marker_id: crate::get_library_marker_id, diff --git a/datafusion/ffi/src/session/task_ctx_accessor.rs b/datafusion/ffi/src/session/task_ctx_accessor.rs new file mode 100644 index 000000000000..83c476ba7ced --- /dev/null +++ b/datafusion/ffi/src/session/task_ctx_accessor.rs @@ -0,0 +1,155 @@ +// 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::execution_plan::{ExecutionPlanPrivateData, FFI_ExecutionPlan}; +use crate::session::config::FFI_SessionConfig; +use crate::session::task_context::FFI_TaskContext; +use crate::udaf::FFI_AggregateUDF; +use crate::udf::FFI_ScalarUDF; +use crate::udwf::FFI_WindowUDF; +use crate::{df_result, rresult}; +use abi_stable::pmr::ROption; +use abi_stable::std_types::{RHashMap, RResult}; +use abi_stable::{std_types::RString, StableAbi}; +use arrow_schema::ArrowError; +use datafusion_common::{exec_datafusion_err, DataFusionError}; +use datafusion_execution::config::SessionConfig; +use datafusion_execution::runtime_env::RuntimeEnv; +use datafusion_execution::{TaskContext, TaskContextAccessor}; +use std::sync::Weak; +use std::{ffi::c_void, sync::Arc}; + +#[repr(C)] +#[derive(Debug, StableAbi)] +#[allow(non_camel_case_types)] +pub struct FFI_TaskContextAccessor { + pub get_task_context: + unsafe extern "C" fn(&Self) -> RResult, + + /// Used to create a clone on the task context accessor. This should + /// only need to be called by the receiver of the plan. + pub clone: unsafe extern "C" fn(plan: &Self) -> Self, + + /// Release the memory of the private data when it is no longer being used. + pub release: unsafe extern "C" fn(arg: &mut Self), + + /// Internal data. This is only to be accessed by the provider of the plan. + /// The foreign library should never attempt to access this data. + pub private_data: *mut c_void, + + /// Utility to identify when FFI objects are accessed locally through + /// the foreign interface. + pub library_marker_id: extern "C" fn() -> u64, +} + +unsafe impl Send for FFI_TaskContextAccessor {} +unsafe impl Sync for FFI_TaskContextAccessor {} + +struct TaskContextAccessorPrivateData { + ctx: Weak, +} + +impl FFI_TaskContextAccessor { + unsafe fn inner(&self) -> Option> { + let private_data = self.private_data as *const TaskContextAccessorPrivateData; + (*private_data) + .ctx + .upgrade() + .map(|ctx| ctx.get_task_context()) + } +} + +unsafe extern "C" fn get_task_context_fn_wrapper( + ctx: &FFI_TaskContextAccessor, +) -> RResult { + rresult!(ctx.inner().map(FFI_TaskContext::from).ok_or_else(|| { + exec_datafusion_err!("TaskContextAccessor went out of scope over FFI boundary.") + })) +} + +unsafe extern "C" fn clone_fn_wrapper( + accessor: &FFI_TaskContextAccessor, +) -> FFI_TaskContextAccessor { + let private_data = accessor.private_data as *const TaskContextAccessorPrivateData; + let ctx = Weak::clone(&(*private_data).ctx); + + let private_data = Box::new(TaskContextAccessorPrivateData { ctx }); + + FFI_TaskContextAccessor { + get_task_context: get_task_context_fn_wrapper, + release: release_fn_wrapper, + clone: clone_fn_wrapper, + private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, + } +} +unsafe extern "C" fn release_fn_wrapper(ctx: &mut FFI_TaskContextAccessor) { + let private_data = + Box::from_raw(ctx.private_data as *mut TaskContextAccessorPrivateData); + drop(private_data); +} +impl Drop for FFI_TaskContextAccessor { + fn drop(&mut self) { + unsafe { (self.release)(self) } + } +} + +impl Clone for FFI_TaskContextAccessor { + fn clone(&self) -> Self { + unsafe { (self.clone)(self) } + } +} + +impl From> for FFI_TaskContextAccessor { + fn from(ctx: Arc) -> Self { + (&ctx).into() + } +} + +impl From<&Arc> for FFI_TaskContextAccessor { + fn from(ctx: &Arc) -> Self { + let ctx = Arc::downgrade(ctx); + let private_data = Box::new(TaskContextAccessorPrivateData { ctx }); + + FFI_TaskContextAccessor { + get_task_context: get_task_context_fn_wrapper, + clone: clone_fn_wrapper, + release: release_fn_wrapper, + private_data: Box::into_raw(private_data) as *mut c_void, + library_marker_id: crate::get_library_marker_id, + } + } +} + +impl TryFrom<&FFI_TaskContextAccessor> for Arc { + type Error = DataFusionError; + fn try_from(ffi_ctx: &FFI_TaskContextAccessor) -> Result { + unsafe { + if (ffi_ctx.library_marker_id)() == crate::get_library_marker_id() { + return ffi_ctx.inner().ok_or_else(|| { + exec_datafusion_err!( + "TaskContextAccessor went out of scope over FFI boundary." + ) + }); + } + + df_result!((ffi_ctx.get_task_context)(&ffi_ctx)) + .map(Into::into) + .map(Arc::new) + } + } +} diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index e21355649176..d6ae8bd1ddd8 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -50,6 +50,7 @@ use datafusion_expr::dml::InsertOp; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType}; use datafusion_physical_plan::ExecutionPlan; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`TableProvider`] across FFI boundaries. /// @@ -140,6 +141,8 @@ pub struct FFI_TableProvider { pub function_registry: FFI_WeakFunctionRegistry, + pub task_ctx_accessor: FFI_TaskContextAccessor, + /// Used to create a clone on the provider of the execution plan. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -240,6 +243,7 @@ unsafe extern "C" fn scan_fn_wrapper( filters_serialized: RVec, limit: ROption, ) -> FfiFuture> { + let task_ctx_accessor = provider.task_ctx_accessor.clone(); let function_registry = >::try_from(&provider.function_registry) .expect(""); @@ -277,7 +281,7 @@ unsafe extern "C" fn scan_fn_wrapper( RResult::ROk(FFI_ExecutionPlan::new( plan, - session.task_ctx(), + task_ctx_accessor, runtime.clone(), )) } @@ -290,6 +294,7 @@ unsafe extern "C" fn insert_into_fn_wrapper( input: &FFI_ExecutionPlan, insert_op: FFI_InsertOp, ) -> FfiFuture> { + let task_ctx_accessor = provider.task_ctx_accessor.clone(); let internal_provider = Arc::clone(provider.inner()); let session = session.clone(); let input = input.clone(); @@ -312,7 +317,7 @@ unsafe extern "C" fn insert_into_fn_wrapper( RResult::ROk(FFI_ExecutionPlan::new( plan, - session.task_ctx(), + task_ctx_accessor, runtime.clone(), )) } @@ -340,6 +345,7 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_TableProvider) -> FFI_Table supports_filters_pushdown: provider.supports_filters_pushdown, insert_into: provider.insert_into, function_registry: provider.function_registry.clone(), + task_ctx_accessor: provider.task_ctx_accessor.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -361,6 +367,7 @@ impl FFI_TableProvider { can_support_pushdown_filters: bool, runtime: Option, function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -374,6 +381,7 @@ impl FFI_TableProvider { }, insert_into: insert_into_fn_wrapper, function_registry, + task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -432,7 +440,7 @@ impl TableProvider for ForeignTableProvider { limit: Option, ) -> Result> { // let session_config: FFI_SessionConfig = session.config().into(); - let session = FFI_Session::new(session, self.0.function_registry.clone(), None); + let session = FFI_Session::new(session, self.0.function_registry.clone(), self.0.task_ctx_accessor.clone(), None); let projections: Option> = projection.map(|p| p.iter().map(|v| v.to_owned()).collect()); @@ -495,13 +503,11 @@ impl TableProvider for ForeignTableProvider { input: Arc, insert_op: InsertOp, ) -> Result> { - // let session_config: FFI_SessionConfig = session.config().into(); - let task_ctx = session.task_ctx(); let rc = Handle::try_current().ok(); let session = - FFI_Session::new(session, self.0.function_registry.clone(), rc.clone()); + FFI_Session::new(session, self.0.function_registry.clone(), self.0.task_ctx_accessor.clone(), rc.clone()); - let input = FFI_ExecutionPlan::new(input, task_ctx, rc); + let input = FFI_ExecutionPlan::new(input, self.0.task_ctx_accessor.clone(), rc); let insert_op: FFI_InsertOp = insert_op.into(); let plan = unsafe { @@ -520,6 +526,7 @@ mod tests { use super::*; use arrow::datatypes::Schema; use datafusion::prelude::{col, lit, SessionContext}; + use datafusion_execution::TaskContextAccessor; use datafusion_expr::registry::FunctionRegistry; #[tokio::test] @@ -546,12 +553,13 @@ mod tests { let ctx = Arc::new(SessionContext::new()); let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); let mut ffi_provider = - FFI_TableProvider::new(provider, true, None, function_registry.into()); + FFI_TableProvider::new(provider, true, None, function_registry.into(), task_ctx_accessor.into()); ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); @@ -592,12 +600,13 @@ mod tests { let ctx = Arc::new(SessionContext::new()); let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); let mut ffi_provider = - FFI_TableProvider::new(provider, true, None, function_registry.into()); + FFI_TableProvider::new(provider, true, None, function_registry.into(), task_ctx_accessor.into()); ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); @@ -643,11 +652,12 @@ mod tests { let ctx = Arc::new(SessionContext::new()); let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1]])?); let mut ffi_provider = - FFI_TableProvider::new(provider, true, None, function_registry.into()); + FFI_TableProvider::new(provider, true, None, function_registry.into(), task_ctx_accessor.into()); ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); diff --git a/datafusion/ffi/src/tests/async_provider.rs b/datafusion/ffi/src/tests/async_provider.rs index 102d6feb03f2..8d177d5a69f5 100644 --- a/datafusion/ffi/src/tests/async_provider.rs +++ b/datafusion/ffi/src/tests/async_provider.rs @@ -44,6 +44,7 @@ use tokio::{ runtime::Handle, sync::{broadcast, mpsc}, }; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; #[derive(Debug)] pub struct AsyncTableProvider { @@ -276,6 +277,7 @@ impl Stream for AsyncTestRecordBatchStream { pub(crate) fn create_async_table_provider( function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { let (table_provider, tokio_rt) = start_async_provider(); FFI_TableProvider::new( @@ -283,5 +285,6 @@ pub(crate) fn create_async_table_provider( true, Some(tokio_rt), function_registry, + task_ctx_accessor, ) } diff --git a/datafusion/ffi/src/tests/catalog.rs b/datafusion/ffi/src/tests/catalog.rs index 3c75a50f40de..b45a3fec9411 100644 --- a/datafusion/ffi/src/tests/catalog.rs +++ b/datafusion/ffi/src/tests/catalog.rs @@ -37,6 +37,7 @@ use datafusion_catalog::{ }; use datafusion_common::error::{DataFusionError, Result}; use datafusion_common::exec_err; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// This schema provider is intended only for unit tests. It prepopulates with one /// table and only allows for tables named sales and purchases. @@ -177,7 +178,8 @@ impl CatalogProvider for FixedCatalogProvider { pub(crate) extern "C" fn create_catalog_provider( function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_CatalogProvider { let catalog_provider = Arc::new(FixedCatalogProvider::default()); - FFI_CatalogProvider::new(catalog_provider, None, function_registry) + FFI_CatalogProvider::new(catalog_provider, None, function_registry, task_ctx_accessor) } diff --git a/datafusion/ffi/src/tests/mod.rs b/datafusion/ffi/src/tests/mod.rs index 8ba910575547..cd22533951c8 100644 --- a/datafusion/ffi/src/tests/mod.rs +++ b/datafusion/ffi/src/tests/mod.rs @@ -44,6 +44,7 @@ use udf_udaf_udwf::{ create_ffi_abs_func, create_ffi_random_func, create_ffi_rank_func, create_ffi_stddev_func, create_ffi_sum_func, create_ffi_table_func, }; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; mod async_provider; pub mod catalog; @@ -60,12 +61,13 @@ pub mod utils; pub struct ForeignLibraryModule { /// Construct an opinionated catalog provider pub create_catalog: - extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_CatalogProvider, + extern "C" fn(function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_CatalogProvider, /// Constructs the table provider pub create_table: extern "C" fn( synchronous: bool, function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider, /// Create a scalar UDF @@ -74,7 +76,8 @@ pub struct ForeignLibraryModule { pub create_nullary_udf: extern "C" fn() -> FFI_ScalarUDF, pub create_table_function: - extern "C" fn(function_registry: FFI_WeakFunctionRegistry) -> FFI_TableFunction, + extern "C" fn(function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_TableFunction, /// Create an aggregate UDAF using sum pub create_sum_udaf: extern "C" fn() -> FFI_AggregateUDF, @@ -82,7 +85,7 @@ pub struct ForeignLibraryModule { /// Create grouping UDAF using stddev pub create_stddev_udaf: extern "C" fn() -> FFI_AggregateUDF, - pub create_rank_udwf: extern "C" fn() -> FFI_WindowUDF, + pub create_rank_udwf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_WindowUDF, pub version: extern "C" fn() -> u64, } @@ -118,10 +121,11 @@ pub fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { extern "C" fn construct_table_provider( synchronous: bool, function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { match synchronous { - true => create_sync_table_provider(function_registry), - false => create_async_table_provider(function_registry), + true => create_sync_table_provider(function_registry, task_ctx_accessor), + false => create_async_table_provider(function_registry, task_ctx_accessor), } } diff --git a/datafusion/ffi/src/tests/sync_provider.rs b/datafusion/ffi/src/tests/sync_provider.rs index 16c2e7cc1e61..cd23c4b1d0b0 100644 --- a/datafusion/ffi/src/tests/sync_provider.rs +++ b/datafusion/ffi/src/tests/sync_provider.rs @@ -17,12 +17,14 @@ use super::{create_record_batch, create_test_schema}; use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::table_provider::FFI_TableProvider; use datafusion_catalog::MemTable; use std::sync::Arc; pub(crate) fn create_sync_table_provider( function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { let schema = create_test_schema(); @@ -36,5 +38,11 @@ pub(crate) fn create_sync_table_provider( let table_provider = MemTable::try_new(schema, vec![batches]).unwrap(); - FFI_TableProvider::new(Arc::new(table_provider), true, None, function_registry) + FFI_TableProvider::new( + Arc::new(table_provider), + true, + None, + function_registry, + task_ctx_accessor, + ) } diff --git a/datafusion/ffi/src/tests/udf_udaf_udwf.rs b/datafusion/ffi/src/tests/udf_udaf_udwf.rs index 4e0a60547a16..c777d12bb367 100644 --- a/datafusion/ffi/src/tests/udf_udaf_udwf.rs +++ b/datafusion/ffi/src/tests/udf_udaf_udwf.rs @@ -30,6 +30,7 @@ use datafusion_functions_aggregate::sum::Sum; use datafusion_functions_table::generate_series::RangeFunc; use datafusion_functions_window::rank::Rank; use std::sync::Arc; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; pub(crate) extern "C" fn create_ffi_abs_func() -> FFI_ScalarUDF { let udf: Arc = Arc::new(AbsFunc::new().into()); @@ -45,10 +46,11 @@ pub(crate) extern "C" fn create_ffi_random_func() -> FFI_ScalarUDF { pub(crate) extern "C" fn create_ffi_table_func( function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableFunction { let udtf: Arc = Arc::new(RangeFunc {}); - FFI_TableFunction::new(udtf, None, function_registry) + FFI_TableFunction::new(udtf, None, function_registry, task_ctx_accessor) } pub(crate) extern "C" fn create_ffi_sum_func() -> FFI_AggregateUDF { @@ -63,7 +65,7 @@ pub(crate) extern "C" fn create_ffi_stddev_func() -> FFI_AggregateUDF { udaf.into() } -pub(crate) extern "C" fn create_ffi_rank_func() -> FFI_WindowUDF { +pub(crate) extern "C" fn create_ffi_rank_func(task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_WindowUDF { let udwf: Arc = Arc::new( Rank::new( "rank_demo".to_string(), @@ -72,5 +74,5 @@ pub(crate) extern "C" fn create_ffi_rank_func() -> FFI_WindowUDF { .into(), ); - udwf.into() + FFI_WindowUDF::new(udwf, task_ctx_accessor) } diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index 24366d5ae944..56c8612cdee4 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -36,6 +36,7 @@ use datafusion_proto::{ }; use prost::Message; use tokio::runtime::Handle; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing a [`TableFunctionImpl`] across FFI boundaries. #[repr(C)] @@ -51,6 +52,8 @@ pub struct FFI_TableFunction { pub function_registry: FFI_WeakFunctionRegistry, + pub task_ctx_accessor: FFI_TaskContextAccessor, + /// Used to create a clone on the provider of the udtf. This should /// only need to be called by the receiver of the udtf. pub clone: unsafe extern "C" fn(udtf: &Self) -> Self, @@ -92,6 +95,7 @@ unsafe extern "C" fn call_fn_wrapper( args: RVec, ) -> RResult { let function_registry = udtf.function_registry.clone(); + let task_ctx_accessor = udtf.task_ctx_accessor.clone(); let foreign_registry = rresult_return!( >::try_from(&udtf.function_registry) ); @@ -115,6 +119,7 @@ unsafe extern "C" fn call_fn_wrapper( false, runtime, function_registry, + task_ctx_accessor, )) } @@ -125,10 +130,11 @@ unsafe extern "C" fn release_fn_wrapper(udtf: &mut FFI_TableFunction) { unsafe extern "C" fn clone_fn_wrapper(udtf: &FFI_TableFunction) -> FFI_TableFunction { let function_registry = udtf.function_registry.clone(); + let task_ctx_accessor = udtf.task_ctx_accessor.clone(); let runtime = udtf.runtime(); let udtf = udtf.inner(); - FFI_TableFunction::new(Arc::clone(udtf), runtime, function_registry) + FFI_TableFunction::new(Arc::clone(udtf), runtime, function_registry, task_ctx_accessor) } impl Clone for FFI_TableFunction { @@ -142,12 +148,14 @@ impl FFI_TableFunction { udtf: Arc, runtime: Option, function_registry: FFI_WeakFunctionRegistry, + task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let private_data = Box::new(TableFunctionPrivateData { udtf, runtime }); Self { call: call_fn_wrapper, function_registry, + task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, @@ -213,6 +221,7 @@ mod tests { use datafusion::{ catalog::MemTable, common::exec_err, prelude::lit, scalar::ScalarValue, }; + use datafusion_execution::TaskContextAccessor; use datafusion_expr::registry::FunctionRegistry; #[derive(Debug)] @@ -300,10 +309,12 @@ mod tests { let ctx = Arc::new(SessionContext::default()); let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let mut local_udtf: FFI_TableFunction = FFI_TableFunction::new( Arc::clone(&original_udtf), None, function_registry.into(), + task_ctx_accessor.into(), ); // Add unit test coverage to check for memory leaks on clone diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index 6b107e66a856..a8874a01d18a 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -51,6 +51,7 @@ use crate::{ util::{rvec_wrapped_to_vec_datatype, vec_datatype_to_rvec_wrapped}, volatility::FFI_Volatility, }; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing a [`WindowUDF`] across FFI boundaries. #[repr(C)] @@ -89,6 +90,8 @@ pub struct FFI_WindowUDF { pub sort_options: ROption, + task_ctx_accessor: FFI_TaskContextAccessor, + /// Used to create a clone on the provider of the udf. This should /// only need to be called by the receiver of the udf. pub clone: unsafe extern "C" fn(udf: &Self) -> Self, @@ -197,6 +200,7 @@ unsafe extern "C" fn clone_fn_wrapper(udwf: &FFI_WindowUDF) -> FFI_WindowUDF { coerce_types: coerce_types_fn_wrapper, field: field_fn_wrapper, clone: clone_fn_wrapper, + task_ctx_accessor: udwf.task_ctx_accessor.clone(), release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, library_marker_id: crate::get_library_marker_id, @@ -209,14 +213,8 @@ impl Clone for FFI_WindowUDF { } } -impl From<&Arc> for FFI_WindowUDF { - fn from(udwf: &Arc) -> Self { - Arc::clone(udwf).into() - } -} - -impl From> for FFI_WindowUDF { - fn from(udf: Arc) -> Self { +impl FFI_WindowUDF { + pub fn new(udf: Arc, task_ctx_accessor: FFI_TaskContextAccessor) -> Self { let name = udf.name().into(); let aliases = udf.aliases().iter().map(|a| a.to_owned().into()).collect(); let volatility = udf.signature().volatility.into(); @@ -233,6 +231,7 @@ impl From> for FFI_WindowUDF { coerce_types: coerce_types_fn_wrapper, field: field_fn_wrapper, clone: clone_fn_wrapper, + task_ctx_accessor, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, library_marker_id: crate::get_library_marker_id, @@ -328,7 +327,7 @@ impl WindowUDFImpl for ForeignWindowUDF { args: datafusion_expr::function::PartitionEvaluatorArgs, ) -> Result> { let evaluator = unsafe { - let args = FFI_PartitionEvaluatorArgs::try_from(args)?; + let args = FFI_PartitionEvaluatorArgs::try_new(args, self.udf.task_ctx_accessor.clone())?; (self.udf.partition_evaluator)(&self.udf, args) }; @@ -401,13 +400,15 @@ mod tests { use datafusion::logical_expr::{col, ExprFunctionExt, WindowUDF, WindowUDFImpl}; use datafusion::prelude::SessionContext; use std::sync::Arc; + use datafusion_execution::TaskContextAccessor; fn create_test_foreign_udwf( original_udwf: impl WindowUDFImpl + 'static, + ctx: Arc, ) -> datafusion::common::Result { let original_udwf = Arc::new(WindowUDF::from(original_udwf)); - let mut local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); + let mut local_udwf = FFI_WindowUDF::new(Arc::clone(&original_udwf), ctx.into()); local_udwf.library_marker_id = crate::mock_foreign_marker_id; let foreign_udwf: Arc = (&local_udwf).try_into()?; @@ -418,9 +419,10 @@ mod tests { fn test_round_trip_udwf() -> datafusion::common::Result<()> { let original_udwf = lag_udwf(); let original_name = original_udwf.name().to_owned(); + let task_ctx_accessor = Arc::new(SessionContext::default()) as Arc; // Convert to FFI format - let mut local_udwf: FFI_WindowUDF = Arc::clone(&original_udwf).into(); + let mut local_udwf = FFI_WindowUDF::new(Arc::clone(&original_udwf).into(), task_ctx_accessor.into()); local_udwf.library_marker_id = crate::mock_foreign_marker_id; // Convert back to native format @@ -433,9 +435,9 @@ mod tests { #[tokio::test] async fn test_lag_udwf() -> datafusion::common::Result<()> { - let udwf = create_test_foreign_udwf(WindowShift::lag())?; + let ctx = Arc::new(SessionContext::default()); + let udwf = create_test_foreign_udwf(WindowShift::lag(), Arc::clone(&ctx) as Arc)?; - let ctx = SessionContext::default(); let df = ctx.read_batch(create_record_batch(-5, 5))?; let df = df.select(vec![ diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index 13cba77c8081..d3493d015291 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -28,12 +28,14 @@ use arrow_schema::FieldRef; use prost::Message; use datafusion_common::error::{DataFusionError, Result}; use datafusion_common::exec_datafusion_err; +use datafusion_execution::{TaskContext, TaskContextAccessor}; use datafusion_expr::function::PartitionEvaluatorArgs; use datafusion_physical_plan::{expressions::Column, PhysicalExpr}; use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; use datafusion_proto::physical_plan::from_proto::parse_physical_expr; use datafusion_proto::physical_plan::to_proto::serialize_physical_exprs; use datafusion_proto::protobuf::PhysicalExprNode; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`PartitionEvaluatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding function @@ -47,11 +49,11 @@ pub struct FFI_PartitionEvaluatorArgs { is_reversed: bool, ignore_nulls: bool, schema: WrappedSchema, + task_ctx_accessor: FFI_TaskContextAccessor, } -impl TryFrom> for FFI_PartitionEvaluatorArgs { - type Error = DataFusionError; - fn try_from(args: PartitionEvaluatorArgs) -> Result { +impl FFI_PartitionEvaluatorArgs { + pub fn try_new(args: PartitionEvaluatorArgs, task_ctx_accessor: FFI_TaskContextAccessor) -> Result { // This is a bit of a hack. Since PartitionEvaluatorArgs does not carry a schema // around, and instead passes the data types directly we are unable to decode the // protobuf PhysicalExpr correctly. In evaluating the code the only place these @@ -111,6 +113,7 @@ impl TryFrom> for FFI_PartitionEvaluatorArgs { schema, is_reversed: args.is_reversed(), ignore_nulls: args.ignore_nulls(), + task_ctx_accessor, }) } } @@ -133,6 +136,7 @@ impl TryFrom for ForeignPartitionEvaluatorArgs { let codec = DefaultPhysicalExtensionCodec {}; let schema: SchemaRef = value.schema.into(); + let task_ctx: Arc = (&value.task_ctx_accessor).try_into()?; let input_exprs = value .input_exprs @@ -142,7 +146,7 @@ impl TryFrom for ForeignPartitionEvaluatorArgs { .map_err(|e| exec_datafusion_err!("Failed to decode PhysicalExprNode: {e}"))? .iter() .map(|expr_node| { - parse_physical_expr(expr_node, &default_ctx.task_ctx(), &schema, &codec) + parse_physical_expr(expr_node, &task_ctx, &schema, &codec) }) .collect::>>()?; From 3e3f6c8c6363994a186078130508a2c4180f0fbf Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 09:34:06 -0500 Subject: [PATCH 48/69] Intermediate work on moving to task ctx accessor throughout ffi crate --- .../ffi/ffi_example_table_provider/src/lib.rs | 4 +- .../ffi/ffi_module_interface/src/lib.rs | 3 +- .../ffi/ffi_module_loader/src/main.rs | 6 +- datafusion/core/src/execution/context/mod.rs | 2 +- datafusion/execution/src/task.rs | 1 - datafusion/ffi/src/catalog_provider.rs | 17 +---- datafusion/ffi/src/execution_plan.rs | 20 +++--- datafusion/ffi/src/function_registry.rs | 24 +++++-- datafusion/ffi/src/plan_properties.rs | 37 +++++----- datafusion/ffi/src/schema_provider.rs | 22 ++---- datafusion/ffi/src/session/mod.rs | 29 ++++---- datafusion/ffi/src/session/task_context.rs | 22 ++++-- .../ffi/src/session/task_ctx_accessor.rs | 26 +++---- datafusion/ffi/src/table_provider.rs | 56 ++++++---------- datafusion/ffi/src/tests/async_provider.rs | 5 +- datafusion/ffi/src/tests/catalog.rs | 6 +- datafusion/ffi/src/tests/mod.rs | 18 ++--- datafusion/ffi/src/tests/sync_provider.rs | 10 +-- datafusion/ffi/src/tests/udf_udaf_udwf.rs | 22 +++--- datafusion/ffi/src/udaf/accumulator_args.rs | 41 +++++++----- datafusion/ffi/src/udaf/mod.rs | 67 ++++++++++++------- datafusion/ffi/src/udtf.rs | 25 ++----- datafusion/ffi/src/udwf/mod.rs | 20 ++++-- .../ffi/src/udwf/partition_evaluator_args.rs | 17 ++--- datafusion/ffi/tests/ffi_integration.rs | 12 ++-- datafusion/ffi/tests/ffi_udaf.rs | 11 +-- datafusion/ffi/tests/ffi_udtf.rs | 7 +- datafusion/ffi/tests/ffi_udwf.rs | 6 +- 28 files changed, 264 insertions(+), 272 deletions(-) diff --git a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs index 57b17e169029..4a0c218328a2 100644 --- a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs @@ -21,7 +21,6 @@ use abi_stable::{export_root_module, prefix_type::PrefixTypeTrait}; use arrow::array::RecordBatch; use arrow::datatypes::{DataType, Field, Schema}; use datafusion::{common::record_batch, datasource::MemTable}; -use datafusion_ffi::function_registry::FFI_WeakFunctionRegistry; use datafusion_ffi::session::task_ctx_accessor::FFI_TaskContextAccessor; use datafusion_ffi::table_provider::FFI_TableProvider; use ffi_module_interface::{TableProviderModule, TableProviderModuleRef}; @@ -37,7 +36,6 @@ fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { /// Here we only wish to create a simple table provider as an example. /// We create an in-memory table and convert it to it's FFI counterpart. extern "C" fn construct_simple_table_provider( - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { let schema = Arc::new(Schema::new(vec![ @@ -55,7 +53,7 @@ extern "C" fn construct_simple_table_provider( let table_provider = MemTable::try_new(schema, vec![batches]).unwrap(); - FFI_TableProvider::new(Arc::new(table_provider), true, None, function_registry, task_ctx_accessor) + FFI_TableProvider::new(Arc::new(table_provider), true, None, task_ctx_accessor) } #[export_root_module] diff --git a/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs b/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs index 943a9eb1f7b0..3d222b99d723 100644 --- a/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_module_interface/src/lib.rs @@ -22,7 +22,6 @@ use abi_stable::{ sabi_types::VersionStrings, StableAbi, }; -use datafusion_ffi::function_registry::FFI_WeakFunctionRegistry; use datafusion_ffi::session::task_ctx_accessor::FFI_TaskContextAccessor; use datafusion_ffi::table_provider::FFI_TableProvider; @@ -36,7 +35,7 @@ use datafusion_ffi::table_provider::FFI_TableProvider; /// how a user may wish to separate these concerns. pub struct TableProviderModule { /// Constructs the table provider - pub create_table: extern "C" fn(FFI_WeakFunctionRegistry, FFI_TaskContextAccessor) -> FFI_TableProvider, + pub create_table: extern "C" fn(FFI_TaskContextAccessor) -> FFI_TableProvider, } impl RootModule for TableProviderModuleRef { diff --git a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs index 6f9a6ee4c6c7..6918608551f1 100644 --- a/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs +++ b/datafusion-examples/examples/ffi/ffi_module_loader/src/main.rs @@ -24,7 +24,7 @@ use datafusion::{ use abi_stable::library::{development_utils::compute_library_path, RootModule}; use datafusion::catalog::TableProvider; -use datafusion::execution::FunctionRegistry; +use datafusion::execution::TaskContextAccessor; use ffi_module_interface::TableProviderModuleRef; #[tokio::main] @@ -41,7 +41,7 @@ async fn main() -> Result<()> { .map_err(|e| DataFusionError::External(Box::new(e)))?; let ctx = Arc::new(SessionContext::new()); - let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; // By calling the code below, the table provided will be created within // the module's code. @@ -50,7 +50,7 @@ async fn main() -> Result<()> { .create_table() .ok_or(DataFusionError::NotImplemented( "External table provider failed to implement create_table".to_string(), - ))?(function_registry.into()); + ))?(task_ctx_accessor.into()); // In order to access the table provider within this executable, we need to // turn it into a `ForeignTableProvider`. diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index be100709349d..46fa5633bea5 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -91,10 +91,10 @@ use datafusion_session::SessionStore; use async_trait::async_trait; use chrono::{DateTime, Utc}; +use datafusion_execution::TaskContextAccessor; use object_store::ObjectStore; use parking_lot::RwLock; use url::Url; -use datafusion_execution::TaskContextAccessor; mod csv; mod json; diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 3af03fd14702..cda130ac23a3 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -215,7 +215,6 @@ pub trait TaskContextAccessor { fn get_task_context(&self) -> Arc; } - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index e9752a1c4436..d3db8d43fe7d 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -29,9 +29,8 @@ use crate::{ schema_provider::{FFI_SchemaProvider, ForeignSchemaProvider}, }; -use crate::function_registry::FFI_WeakFunctionRegistry; -use datafusion_common::error::Result; use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; +use datafusion_common::error::Result; /// A stable struct for sharing [`CatalogProvider`] across FFI boundaries. #[repr(C)] @@ -59,7 +58,6 @@ pub struct FFI_CatalogProvider { cascade: bool, ) -> RResult, RString>, - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should @@ -118,7 +116,6 @@ unsafe extern "C" fn schema_fn_wrapper( FFI_SchemaProvider::new( schema, provider.runtime(), - provider.function_registry.clone(), provider.task_ctx_accessor.clone(), ) }) @@ -137,7 +134,7 @@ unsafe extern "C" fn register_schema_fn_wrapper( .inner() .register_schema(name.as_str(), schema)) .map(|schema| { - FFI_SchemaProvider::new(schema, runtime, provider.function_registry.clone(), provider.task_ctx_accessor.clone()) + FFI_SchemaProvider::new(schema, runtime, provider.task_ctx_accessor.clone()) }) .into(); @@ -160,7 +157,6 @@ unsafe extern "C" fn deregister_schema_fn_wrapper( FFI_SchemaProvider::new( schema, runtime, - provider.function_registry.clone(), provider.task_ctx_accessor.clone(), ) }) @@ -189,7 +185,6 @@ unsafe extern "C" fn clone_fn_wrapper( schema: schema_fn_wrapper, register_schema: register_schema_fn_wrapper, deregister_schema: deregister_schema_fn_wrapper, - function_registry: provider.function_registry.clone(), task_ctx_accessor: provider.task_ctx_accessor.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, @@ -210,7 +205,6 @@ impl FFI_CatalogProvider { pub fn new( provider: Arc, runtime: Option, - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -220,7 +214,6 @@ impl FFI_CatalogProvider { schema: schema_fn_wrapper, register_schema: register_schema_fn_wrapper, deregister_schema: deregister_schema_fn_wrapper, - function_registry, task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, @@ -295,7 +288,6 @@ impl CatalogProvider for ForeignCatalogProvider { None => &FFI_SchemaProvider::new( schema, None, - self.0.function_registry.clone(), self.0.task_ctx_accessor.clone(), ), }; @@ -332,7 +324,6 @@ mod tests { use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; use datafusion::prelude::SessionContext; use datafusion_execution::TaskContextAccessor; - use datafusion_expr::registry::FunctionRegistry; #[test] fn test_round_trip_ffi_catalog_provider() { @@ -346,12 +337,10 @@ mod tests { .is_none()); let ctx = Arc::new(SessionContext::new()); - let function_registry = - Arc::clone(&ctx) as Arc; let task_ctx_accessor = Arc::clone(&ctx) as Arc; let mut ffi_catalog = - FFI_CatalogProvider::new(catalog, None, function_registry.into(), task_ctx_accessor.into()); + FFI_CatalogProvider::new(catalog, None, task_ctx_accessor.into()); ffi_catalog.library_marker_id = crate::mock_foreign_marker_id; let foreign_catalog: Arc = (&ffi_catalog).into(); diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index 9e345bfafa65..65f20751354d 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -17,7 +17,11 @@ use std::{ffi::c_void, pin::Pin, sync::Arc}; -use crate::{df_result, plan_properties::FFI_PlanProperties, record_batch_stream::FFI_RecordBatchStream, rresult, rresult_return}; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; +use crate::{ + df_result, plan_properties::FFI_PlanProperties, + record_batch_stream::FFI_RecordBatchStream, rresult, rresult_return, +}; use abi_stable::{ std_types::{RResult, RString, RVec}, StableAbi, @@ -27,7 +31,6 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_plan::DisplayFormatType; use datafusion_physical_plan::{DisplayAs, ExecutionPlan, PlanProperties}; use tokio::runtime::Handle; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing a [`ExecutionPlan`] across FFI boundaries. #[repr(C)] @@ -136,11 +139,7 @@ unsafe extern "C" fn clone_fn_wrapper(plan: &FFI_ExecutionPlan) -> FFI_Execution let private_data = plan.private_data as *const ExecutionPlanPrivateData; let plan_data = &(*private_data); - FFI_ExecutionPlan::new( - Arc::clone(&plan_data.plan), - ctx, - plan_data.runtime.clone(), - ) + FFI_ExecutionPlan::new(Arc::clone(&plan_data.plan), ctx, plan_data.runtime.clone()) } impl Clone for FFI_ExecutionPlan { @@ -156,10 +155,7 @@ impl FFI_ExecutionPlan { context: FFI_TaskContextAccessor, runtime: Option, ) -> Self { - let private_data = Box::new(ExecutionPlanPrivateData { - plan, - runtime, - }); + let private_data = Box::new(ExecutionPlanPrivateData { plan, runtime }); Self { properties: properties_fn_wrapper, @@ -298,6 +294,7 @@ impl ExecutionPlan for ForeignExecutionPlan { #[cfg(test)] mod tests { + use super::*; use arrow::datatypes::{DataType, Field, Schema}; use datafusion::{ physical_plan::{ @@ -307,7 +304,6 @@ mod tests { prelude::SessionContext, }; use datafusion_execution::TaskContextAccessor; - use super::*; #[derive(Debug)] pub struct EmptyExec { diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs index 395ac894c719..8e98a2cc4c8d 100644 --- a/datafusion/ffi/src/function_registry.rs +++ b/datafusion/ffi/src/function_registry.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::udaf::FFI_AggregateUDF; use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; @@ -51,6 +52,8 @@ pub struct FFI_WeakFunctionRegistry { pub udwf: unsafe extern "C" fn(&Self, name: RString) -> RResult, + pub task_ctx_accessor: FFI_TaskContextAccessor, + /// Used to create a clone on the provider of the registry. This should /// only need to be called by the receiver of the plan. pub clone: unsafe extern "C" fn(plan: &Self) -> Self, @@ -125,7 +128,10 @@ unsafe extern "C" fn udaf_fn_wrapper( ) -> RResult { let inner = rresult_return!(registry.inner()); let udaf = rresult_return!(inner.udaf(name.as_str())); - RResult::ROk(FFI_AggregateUDF::from(udaf)) + RResult::ROk(FFI_AggregateUDF::new( + udaf, + registry.task_ctx_accessor.clone(), + )) } unsafe extern "C" fn udwf_fn_wrapper( registry: &FFI_WeakFunctionRegistry, @@ -133,7 +139,7 @@ unsafe extern "C" fn udwf_fn_wrapper( ) -> RResult { let inner = rresult_return!(registry.inner()); let udwf = rresult_return!(inner.udwf(name.as_str())); - RResult::ROk(FFI_WindowUDF::from(udwf)) + RResult::ROk(FFI_WindowUDF::new(udwf, registry.task_ctx_accessor.clone())) } unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_WeakFunctionRegistry) { @@ -162,6 +168,7 @@ unsafe extern "C" fn clone_fn_wrapper( clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, + task_ctx_accessor: provider.task_ctx_accessor.clone(), private_data, library_marker_id: crate::get_library_marker_id, } @@ -173,9 +180,12 @@ impl Drop for FFI_WeakFunctionRegistry { } } -impl From> for FFI_WeakFunctionRegistry { +impl FFI_WeakFunctionRegistry { /// Creates a new [`FFI_WeakFunctionRegistry`]. - fn from(registry: Arc) -> Self { + pub fn new( + registry: Arc, + task_ctx_accessor: FFI_TaskContextAccessor, + ) -> Self { let registry = Arc::downgrade(®istry); let private_data = Box::new(RegistryPrivateData { registry }); @@ -188,6 +198,7 @@ impl From> for FFI_WeakFunctionRegistry udaf: udaf_fn_wrapper, udwf: udwf_fn_wrapper, + task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, version: super::version, @@ -345,6 +356,7 @@ impl FunctionRegistry for ForeignWeakFunctionRegistry { mod tests { use super::*; use datafusion::prelude::SessionContext; + use datafusion_execution::TaskContextAccessor; use datafusion_expr::registry::FunctionRegistry; #[tokio::test] @@ -352,8 +364,10 @@ mod tests { let ctx = Arc::new(SessionContext::new()); let function_registry = Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; - let mut ffi_registry = FFI_WeakFunctionRegistry::from(function_registry); + let mut ffi_registry = + FFI_WeakFunctionRegistry::new(function_registry, task_ctx_accessor.into()); ffi_registry.library_marker_id = crate::mock_foreign_marker_id; let foreign_registry: Arc = diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index 0f189a1b42c2..05c9d4c02c0a 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -18,24 +18,27 @@ use std::{ffi::c_void, sync::Arc}; use crate::arrow_wrappers::WrappedSchema; -use abi_stable::std_types::{ROption, RResult, RString, RVec}; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; +use crate::{df_result, rresult_return}; +use abi_stable::std_types::{RResult, RString, RVec}; use abi_stable::StableAbi; use arrow::datatypes::SchemaRef; -use prost::Message; use datafusion_common::error::{DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; -use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::{ execution_plan::{Boundedness, EmissionType}, PlanProperties, }; +use datafusion_proto::physical_plan::from_proto::{ + parse_physical_sort_exprs, parse_protobuf_partitioning, +}; +use datafusion_proto::physical_plan::to_proto::{ + serialize_partitioning, serialize_physical_sort_exprs, +}; use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; -use datafusion_proto::physical_plan::from_proto::{parse_physical_sort_exprs, parse_protobuf_partitioning}; -use datafusion_proto::physical_plan::to_proto::{serialize_partitioning, serialize_physical_sort_exprs}; use datafusion_proto::protobuf::{Partitioning, PhysicalSortExprNodeCollection}; -use crate::{df_result, rresult_return}; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; +use prost::Message; /// A stable struct for sharing [`PlanProperties`] across FFI boundaries. #[repr(C)] @@ -44,7 +47,8 @@ use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; pub struct FFI_PlanProperties { /// The output partitioning is a [`Partitioning`] protobuf message serialized /// into bytes to pass across the FFI boundary. - pub output_partitioning: unsafe extern "C" fn(plan: &Self) -> RResult, RString>, + pub output_partitioning: + unsafe extern "C" fn(plan: &Self) -> RResult, RString>, /// Return the emission type of the plan. pub emission_type: unsafe extern "C" fn(plan: &Self) -> FFI_EmissionType, @@ -150,7 +154,10 @@ impl Drop for FFI_PlanProperties { } impl FFI_PlanProperties { - pub fn new(props: &PlanProperties, task_ctx_accessor: FFI_TaskContextAccessor) -> Self { + pub fn new( + props: &PlanProperties, + task_ctx_accessor: FFI_TaskContextAccessor, + ) -> Self { let private_data = Box::new(PlanPropertiesPrivateData { props: props.clone(), }); @@ -206,10 +213,10 @@ impl TryFrom for PlanProperties { &schema, &codex, )? - .ok_or(DataFusionError::Plan( - "Unable to deserialize partitioning protobuf in FFI_PlanProperties" - .to_string(), - ))?; + .ok_or(DataFusionError::Plan( + "Unable to deserialize partitioning protobuf in FFI_PlanProperties" + .to_string(), + ))?; let eq_properties = if sort_exprs.is_empty() { EquivalenceProperties::new(Arc::new(schema)) @@ -299,10 +306,10 @@ impl From for EmissionType { #[cfg(test)] mod tests { - use datafusion::{physical_expr::PhysicalSortExpr, physical_plan::Partitioning}; + use super::*; use datafusion::prelude::SessionContext; + use datafusion::{physical_expr::PhysicalSortExpr, physical_plan::Partitioning}; use datafusion_execution::TaskContextAccessor; - use super::*; #[test] fn test_round_trip_ffi_plan_properties() -> Result<()> { diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index 0a5991e0820f..b615aaf7925d 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -27,12 +27,11 @@ use datafusion_catalog::{SchemaProvider, TableProvider}; use datafusion_common::error::{DataFusionError, Result}; use tokio::runtime::Handle; -use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::{ df_result, rresult_return, table_provider::{FFI_TableProvider, ForeignTableProvider}, }; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`SchemaProvider`] across FFI boundaries. #[repr(C)] @@ -65,7 +64,6 @@ pub struct FFI_SchemaProvider { pub table_exist: unsafe extern "C" fn(provider: &Self, name: RString) -> bool, - pub function_registry: FFI_WeakFunctionRegistry, pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should @@ -120,14 +118,13 @@ unsafe extern "C" fn table_fn_wrapper( provider: &FFI_SchemaProvider, name: RString, ) -> FfiFuture, RString>> { - let function_registry = provider.function_registry.clone(); let task_ctx_accessor = provider.task_ctx_accessor.clone(); let runtime = provider.runtime(); let provider = Arc::clone(provider.inner()); async move { let table = rresult_return!(provider.table(name.as_str()).await) - .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry, task_ctx_accessor)) + .map(|t| FFI_TableProvider::new(t, true, runtime, task_ctx_accessor)) .into(); RResult::ROk(table) @@ -141,14 +138,13 @@ unsafe extern "C" fn register_table_fn_wrapper( table: FFI_TableProvider, ) -> RResult, RString> { let runtime = provider.runtime(); - let function_registry = provider.function_registry.clone(); let task_ctx_accessor = provider.task_ctx_accessor.clone(); let provider = provider.inner(); let table = Arc::new(ForeignTableProvider(table)); let returned_table = rresult_return!(provider.register_table(name.into(), table)) - .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry, task_ctx_accessor)); + .map(|t| FFI_TableProvider::new(t, true, runtime, task_ctx_accessor)); RResult::ROk(returned_table.into()) } @@ -157,13 +153,12 @@ unsafe extern "C" fn deregister_table_fn_wrapper( provider: &FFI_SchemaProvider, name: RString, ) -> RResult, RString> { - let function_registry = provider.function_registry.clone(); let task_ctx_accessor = provider.task_ctx_accessor.clone(); let runtime = provider.runtime(); let provider = provider.inner(); let returned_table = rresult_return!(provider.deregister_table(name.as_str())) - .map(|t| FFI_TableProvider::new(t, true, runtime, function_registry, task_ctx_accessor)); + .map(|t| FFI_TableProvider::new(t, true, runtime, task_ctx_accessor)); RResult::ROk(returned_table.into()) } @@ -202,7 +197,6 @@ unsafe extern "C" fn clone_fn_wrapper( register_table: register_table_fn_wrapper, deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, - function_registry: provider.function_registry.clone(), task_ctx_accessor: provider.task_ctx_accessor.clone(), library_marker_id: crate::get_library_marker_id, } @@ -219,7 +213,6 @@ impl FFI_SchemaProvider { pub fn new( provider: Arc, runtime: Option, - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let owner_name = provider.owner_name().map(|s| s.into()).into(); @@ -236,7 +229,6 @@ impl FFI_SchemaProvider { register_table: register_table_fn_wrapper, deregister_table: deregister_table_fn_wrapper, table_exist: table_exist_fn_wrapper, - function_registry, task_ctx_accessor, library_marker_id: crate::get_library_marker_id, } @@ -315,7 +307,6 @@ impl SchemaProvider for ForeignSchemaProvider { table, true, None, - self.0.function_registry.clone(), self.0.task_ctx_accessor.clone(), ), }; @@ -351,7 +342,6 @@ mod tests { use datafusion::prelude::SessionContext; use datafusion::{catalog::MemorySchemaProvider, datasource::empty::EmptyTable}; use datafusion_execution::TaskContextAccessor; - use datafusion_expr::registry::FunctionRegistry; fn empty_table() -> Arc { Arc::new(EmptyTable::new(Arc::new(Schema::empty()))) @@ -366,12 +356,10 @@ mod tests { .unwrap() .is_none()); let ctx = Arc::new(SessionContext::new()); - let function_registry = - Arc::clone(&ctx) as Arc; let task_ctx_accessor = Arc::clone(&ctx) as Arc; let mut ffi_schema_provider = - FFI_SchemaProvider::new(schema_provider, None, function_registry.into(), task_ctx_accessor.into()); + FFI_SchemaProvider::new(schema_provider, None, task_ctx_accessor.into()); ffi_schema_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_schema_provider: Arc = diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 3d74c6db32b4..c5a45fffca15 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -17,9 +17,9 @@ use crate::arrow_wrappers::WrappedSchema; use crate::execution_plan::FFI_ExecutionPlan; -use crate::function_registry::FFI_WeakFunctionRegistry; use crate::session::config::FFI_SessionConfig; use crate::session::task_context::FFI_TaskContext; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::udaf::FFI_AggregateUDF; use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; @@ -40,7 +40,6 @@ use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; -use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{ AggregateUDF, AggregateUDFImpl, Expr, LogicalPlan, ScalarUDF, ScalarUDFImpl, WindowUDF, WindowUDFImpl, @@ -60,7 +59,6 @@ use std::any::Any; use std::collections::HashMap; use std::{ffi::c_void, sync::Arc}; use tokio::runtime::Handle; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; pub mod config; pub mod task_context; @@ -104,7 +102,6 @@ pub struct FFI_Session { pub task_ctx: unsafe extern "C" fn(&Self) -> FFI_TaskContext, - pub function_registry: FFI_WeakFunctionRegistry, pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the registry. This should @@ -140,10 +137,8 @@ impl FFI_Session { unsafe { (*private_data).session } } - fn function_registry( - &self, - ) -> Result, DataFusionError> { - (&self.function_registry).try_into() + fn task_ctx(&self) -> Result, DataFusionError> { + (&self.task_ctx_accessor).try_into() } unsafe fn runtime(&self) -> &Option { @@ -194,13 +189,12 @@ unsafe extern "C" fn create_physical_expr_fn_wrapper( expr_serialized: RVec, schema: WrappedSchema, ) -> RResult, RString> { - let function_registry = rresult_return!(session.function_registry()); + let task_ctx = rresult_return!(session.task_ctx()); let session = session.inner(); let codec = DefaultLogicalExtensionCodec {}; let logical_expr = LogicalExprNode::decode(expr_serialized.as_slice()).unwrap(); - let logical_expr = - parse_expr(&logical_expr, function_registry.as_ref(), &codec).unwrap(); + let logical_expr = parse_expr(&logical_expr, task_ctx.as_ref(), &codec).unwrap(); let schema: SchemaRef = schema.into(); let schema: DFSchema = rresult_return!(schema.try_into()); @@ -227,6 +221,7 @@ unsafe extern "C" fn scalar_functions_fn_wrapper( unsafe extern "C" fn aggregate_functions_fn_wrapper( session: &FFI_Session, ) -> RHashMap { + let task_ctx_accessor = &session.task_ctx_accessor; let session = session.inner(); session .aggregate_functions() @@ -234,7 +229,7 @@ unsafe extern "C" fn aggregate_functions_fn_wrapper( .map(|(name, udaf)| { ( name.clone().into(), - FFI_AggregateUDF::from(Arc::clone(udaf)), + FFI_AggregateUDF::new(Arc::clone(udaf), task_ctx_accessor.clone()), ) }) .collect() @@ -248,7 +243,12 @@ unsafe extern "C" fn window_functions_fn_wrapper( session .window_functions() .iter() - .map(|(name, udwf)| (name.clone().into(), FFI_WindowUDF::new(Arc::clone(udwf), task_ctx_accessor.clone()))) + .map(|(name, udwf)| { + ( + name.clone().into(), + FFI_WindowUDF::new(Arc::clone(udwf), task_ctx_accessor.clone()), + ) + }) .collect() } @@ -307,7 +307,6 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_Session) -> FFI_Session { table_options: table_options_fn_wrapper, default_table_options: default_table_options_fn_wrapper, task_ctx: task_ctx_fn_wrapper, - function_registry: provider.function_registry.clone(), task_ctx_accessor: provider.task_ctx_accessor.clone(), clone: clone_fn_wrapper, @@ -328,7 +327,6 @@ impl FFI_Session { /// Creates a new [`FFI_Session`]. pub fn new( session: &(dyn Session + Send + Sync), - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, runtime: Option, ) -> Self { @@ -345,7 +343,6 @@ impl FFI_Session { table_options: table_options_fn_wrapper, default_table_options: default_table_options_fn_wrapper, task_ctx: task_ctx_fn_wrapper, - function_registry, task_ctx_accessor, clone: clone_fn_wrapper, diff --git a/datafusion/ffi/src/session/task_context.rs b/datafusion/ffi/src/session/task_context.rs index 285e4c44074b..057f8d98c072 100644 --- a/datafusion/ffi/src/session/task_context.rs +++ b/datafusion/ffi/src/session/task_context.rs @@ -16,6 +16,7 @@ // under the License. use crate::session::config::FFI_SessionConfig; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::udaf::FFI_AggregateUDF; use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; @@ -29,7 +30,6 @@ use datafusion_expr::{ AggregateUDF, AggregateUDFImpl, ScalarUDF, ScalarUDFImpl, WindowUDF, WindowUDFImpl, }; use std::{ffi::c_void, sync::Arc}; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`TaskContext`] across FFI boundaries. #[repr(C)] @@ -104,10 +104,16 @@ unsafe extern "C" fn scalar_functions_fn_wrapper( unsafe extern "C" fn aggregate_functions_fn_wrapper( ctx: &FFI_TaskContext, ) -> RHashMap { + let task_ctx_accessor = &ctx.task_ctx_accessor; let ctx = ctx.inner(); ctx.aggregate_functions() .iter() - .map(|(name, udf)| (name.to_owned().into(), udf.into())) + .map(|(name, udaf)| { + ( + name.to_owned().into(), + FFI_AggregateUDF::new(Arc::clone(udaf), task_ctx_accessor.clone()), + ) + }) .collect() } @@ -118,7 +124,12 @@ unsafe extern "C" fn window_functions_fn_wrapper( let ctx = ctx.inner(); ctx.window_functions() .iter() - .map(|(name, udf)| (name.to_owned().into(), FFI_WindowUDF::new(Arc::clone(udf), task_ctx_accessor.clone()))) + .map(|(name, udf)| { + ( + name.to_owned().into(), + FFI_WindowUDF::new(Arc::clone(udf), task_ctx_accessor.clone()), + ) + }) .collect() } @@ -134,7 +145,10 @@ impl Drop for FFI_TaskContext { } impl FFI_TaskContext { - pub fn new(ctx: Arc, task_ctx_accessor: FFI_TaskContextAccessor) -> Self { + pub fn new( + ctx: Arc, + task_ctx_accessor: FFI_TaskContextAccessor, + ) -> Self { let private_data = Box::new(TaskContextPrivateData { ctx }); FFI_TaskContext { diff --git a/datafusion/ffi/src/session/task_ctx_accessor.rs b/datafusion/ffi/src/session/task_ctx_accessor.rs index 83c476ba7ced..610fa3e053b6 100644 --- a/datafusion/ffi/src/session/task_ctx_accessor.rs +++ b/datafusion/ffi/src/session/task_ctx_accessor.rs @@ -15,20 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::execution_plan::{ExecutionPlanPrivateData, FFI_ExecutionPlan}; -use crate::session::config::FFI_SessionConfig; use crate::session::task_context::FFI_TaskContext; -use crate::udaf::FFI_AggregateUDF; -use crate::udf::FFI_ScalarUDF; -use crate::udwf::FFI_WindowUDF; use crate::{df_result, rresult}; -use abi_stable::pmr::ROption; -use abi_stable::std_types::{RHashMap, RResult}; +use abi_stable::std_types::RResult; use abi_stable::{std_types::RString, StableAbi}; -use arrow_schema::ArrowError; use datafusion_common::{exec_datafusion_err, DataFusionError}; -use datafusion_execution::config::SessionConfig; -use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::{TaskContext, TaskContextAccessor}; use std::sync::Weak; use std::{ffi::c_void, sync::Arc}; @@ -74,11 +65,16 @@ impl FFI_TaskContextAccessor { } unsafe extern "C" fn get_task_context_fn_wrapper( - ctx: &FFI_TaskContextAccessor, + ctx_accessor: &FFI_TaskContextAccessor, ) -> RResult { - rresult!(ctx.inner().map(FFI_TaskContext::from).ok_or_else(|| { - exec_datafusion_err!("TaskContextAccessor went out of scope over FFI boundary.") - })) + rresult!(ctx_accessor + .inner() + .map(|ctx| FFI_TaskContext::new(ctx, ctx_accessor.clone())) + .ok_or_else(|| { + exec_datafusion_err!( + "TaskContextAccessor went out of scope over FFI boundary." + ) + })) } unsafe extern "C" fn clone_fn_wrapper( @@ -147,7 +143,7 @@ impl TryFrom<&FFI_TaskContextAccessor> for Arc { }); } - df_result!((ffi_ctx.get_task_context)(&ffi_ctx)) + df_result!((ffi_ctx.get_task_context)(ffi_ctx)) .map(Into::into) .map(Arc::new) } diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index d6ae8bd1ddd8..d919652645c5 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -35,7 +35,7 @@ use datafusion_proto::{ use prost::Message; use tokio::runtime::Handle; -use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::session::{FFI_Session, ForeignSession}; use crate::{ arrow_wrappers::WrappedSchema, @@ -46,11 +46,10 @@ use crate::{ table_source::{FFI_TableProviderFilterPushDown, FFI_TableType}, }; use datafusion_common::{DataFusionError, Result}; +use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; -use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType}; use datafusion_physical_plan::ExecutionPlan; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`TableProvider`] across FFI boundaries. /// @@ -139,8 +138,6 @@ pub struct FFI_TableProvider { insert_op: FFI_InsertOp, ) -> FfiFuture>, - pub function_registry: FFI_WeakFunctionRegistry, - pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should @@ -195,7 +192,7 @@ unsafe extern "C" fn table_type_fn_wrapper( fn supports_filters_pushdown_internal( provider: &Arc, filters_serialized: &[u8], - function_registry: &dyn FunctionRegistry, + task_ctx: &Arc, ) -> Result> { let codec = DefaultLogicalExtensionCodec {}; @@ -205,7 +202,7 @@ fn supports_filters_pushdown_internal( let proto_filters = LogicalExprList::decode(filters_serialized) .map_err(|e| DataFusionError::Plan(e.to_string()))?; - parse_exprs(proto_filters.expr.iter(), function_registry, &codec)? + parse_exprs(proto_filters.expr.iter(), task_ctx.as_ref(), &codec)? } }; let filters_borrowed: Vec<&Expr> = filters.iter().collect(); @@ -223,17 +220,12 @@ unsafe extern "C" fn supports_filters_pushdown_fn_wrapper( provider: &FFI_TableProvider, filters_serialized: RVec, ) -> RResult, RString> { - let function_registry = rresult_return!( - >::try_from(&provider.function_registry) - ); - - supports_filters_pushdown_internal( - provider.inner(), - &filters_serialized, - function_registry.as_ref(), - ) - .map_err(|e| e.to_string().into()) - .into() + let task_ctx = + rresult_return!(>::try_from(&provider.task_ctx_accessor)); + + supports_filters_pushdown_internal(provider.inner(), &filters_serialized, &task_ctx) + .map_err(|e| e.to_string().into()) + .into() } unsafe extern "C" fn scan_fn_wrapper( @@ -243,16 +235,16 @@ unsafe extern "C" fn scan_fn_wrapper( filters_serialized: RVec, limit: ROption, ) -> FfiFuture> { + let task_ctx: Result, DataFusionError> = + (&provider.task_ctx_accessor).try_into(); let task_ctx_accessor = provider.task_ctx_accessor.clone(); - let function_registry = - >::try_from(&provider.function_registry) - .expect(""); let session = ForeignSession::try_from(session); let internal_provider = Arc::clone(provider.inner()); let runtime = provider.runtime().clone(); async move { let session = rresult_return!(session); + let task_ctx = rresult_return!(task_ctx); let filters = match filters_serialized.is_empty() { true => vec![], @@ -265,7 +257,7 @@ unsafe extern "C" fn scan_fn_wrapper( rresult_return!(parse_exprs( proto_filters.expr.iter(), - function_registry.as_ref(), + task_ctx.as_ref(), &codec )) } @@ -344,7 +336,6 @@ unsafe extern "C" fn clone_fn_wrapper(provider: &FFI_TableProvider) -> FFI_Table table_type: table_type_fn_wrapper, supports_filters_pushdown: provider.supports_filters_pushdown, insert_into: provider.insert_into, - function_registry: provider.function_registry.clone(), task_ctx_accessor: provider.task_ctx_accessor.clone(), clone: clone_fn_wrapper, release: release_fn_wrapper, @@ -366,7 +357,6 @@ impl FFI_TableProvider { provider: Arc, can_support_pushdown_filters: bool, runtime: Option, - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let private_data = Box::new(ProviderPrivateData { provider, runtime }); @@ -380,7 +370,6 @@ impl FFI_TableProvider { false => None, }, insert_into: insert_into_fn_wrapper, - function_registry, task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, @@ -440,7 +429,7 @@ impl TableProvider for ForeignTableProvider { limit: Option, ) -> Result> { // let session_config: FFI_SessionConfig = session.config().into(); - let session = FFI_Session::new(session, self.0.function_registry.clone(), self.0.task_ctx_accessor.clone(), None); + let session = FFI_Session::new(session, self.0.task_ctx_accessor.clone(), None); let projections: Option> = projection.map(|p| p.iter().map(|v| v.to_owned()).collect()); @@ -505,7 +494,7 @@ impl TableProvider for ForeignTableProvider { ) -> Result> { let rc = Handle::try_current().ok(); let session = - FFI_Session::new(session, self.0.function_registry.clone(), self.0.task_ctx_accessor.clone(), rc.clone()); + FFI_Session::new(session, self.0.task_ctx_accessor.clone(), rc.clone()); let input = FFI_ExecutionPlan::new(input, self.0.task_ctx_accessor.clone(), rc); let insert_op: FFI_InsertOp = insert_op.into(); @@ -527,7 +516,6 @@ mod tests { use arrow::datatypes::Schema; use datafusion::prelude::{col, lit, SessionContext}; use datafusion_execution::TaskContextAccessor; - use datafusion_expr::registry::FunctionRegistry; #[tokio::test] async fn test_round_trip_ffi_table_provider_scan() -> Result<()> { @@ -551,15 +539,13 @@ mod tests { )?; let ctx = Arc::new(SessionContext::new()); - let function_registry = - Arc::clone(&ctx) as Arc; let task_ctx_accessor = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); let mut ffi_provider = - FFI_TableProvider::new(provider, true, None, function_registry.into(), task_ctx_accessor.into()); + FFI_TableProvider::new(provider, true, None, task_ctx_accessor.into()); ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); @@ -598,15 +584,13 @@ mod tests { )?; let ctx = Arc::new(SessionContext::new()); - let function_registry = - Arc::clone(&ctx) as Arc; let task_ctx_accessor = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1], vec![batch2]])?); let mut ffi_provider = - FFI_TableProvider::new(provider, true, None, function_registry.into(), task_ctx_accessor.into()); + FFI_TableProvider::new(provider, true, None, task_ctx_accessor.into()); ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); @@ -650,14 +634,12 @@ mod tests { )?; let ctx = Arc::new(SessionContext::new()); - let function_registry = - Arc::clone(&ctx) as Arc; let task_ctx_accessor = Arc::clone(&ctx) as Arc; let provider = Arc::new(MemTable::try_new(schema, vec![vec![batch1]])?); let mut ffi_provider = - FFI_TableProvider::new(provider, true, None, function_registry.into(), task_ctx_accessor.into()); + FFI_TableProvider::new(provider, true, None, task_ctx_accessor.into()); ffi_provider.library_marker_id = crate::mock_foreign_marker_id; let foreign_table_provider: Arc = (&ffi_provider).into(); diff --git a/datafusion/ffi/src/tests/async_provider.rs b/datafusion/ffi/src/tests/async_provider.rs index 8d177d5a69f5..5a0e217f453f 100644 --- a/datafusion/ffi/src/tests/async_provider.rs +++ b/datafusion/ffi/src/tests/async_provider.rs @@ -28,7 +28,7 @@ use std::{any::Any, fmt::Debug, sync::Arc}; use super::create_record_batch; -use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::table_provider::FFI_TableProvider; use arrow::array::RecordBatch; use arrow::datatypes::Schema; @@ -44,7 +44,6 @@ use tokio::{ runtime::Handle, sync::{broadcast, mpsc}, }; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; #[derive(Debug)] pub struct AsyncTableProvider { @@ -276,7 +275,6 @@ impl Stream for AsyncTestRecordBatchStream { } pub(crate) fn create_async_table_provider( - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { let (table_provider, tokio_rt) = start_async_provider(); @@ -284,7 +282,6 @@ pub(crate) fn create_async_table_provider( Arc::new(table_provider), true, Some(tokio_rt), - function_registry, task_ctx_accessor, ) } diff --git a/datafusion/ffi/src/tests/catalog.rs b/datafusion/ffi/src/tests/catalog.rs index b45a3fec9411..6630adad2be1 100644 --- a/datafusion/ffi/src/tests/catalog.rs +++ b/datafusion/ffi/src/tests/catalog.rs @@ -28,7 +28,7 @@ use std::{any::Any, fmt::Debug, sync::Arc}; use crate::catalog_provider::FFI_CatalogProvider; -use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use arrow::datatypes::Schema; use async_trait::async_trait; use datafusion_catalog::{ @@ -37,7 +37,6 @@ use datafusion_catalog::{ }; use datafusion_common::error::{DataFusionError, Result}; use datafusion_common::exec_err; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// This schema provider is intended only for unit tests. It prepopulates with one /// table and only allows for tables named sales and purchases. @@ -177,9 +176,8 @@ impl CatalogProvider for FixedCatalogProvider { } pub(crate) extern "C" fn create_catalog_provider( - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_CatalogProvider { let catalog_provider = Arc::new(FixedCatalogProvider::default()); - FFI_CatalogProvider::new(catalog_provider, None, function_registry, task_ctx_accessor) + FFI_CatalogProvider::new(catalog_provider, None, task_ctx_accessor) } diff --git a/datafusion/ffi/src/tests/mod.rs b/datafusion/ffi/src/tests/mod.rs index cd22533951c8..4a002bb68e1d 100644 --- a/datafusion/ffi/src/tests/mod.rs +++ b/datafusion/ffi/src/tests/mod.rs @@ -34,7 +34,7 @@ use crate::udaf::FFI_AggregateUDF; use crate::udwf::FFI_WindowUDF; use super::{table_provider::FFI_TableProvider, udf::FFI_ScalarUDF}; -use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use arrow::array::RecordBatch; use arrow::datatypes::{DataType, Field, Schema}; use async_provider::create_async_table_provider; @@ -44,7 +44,6 @@ use udf_udaf_udwf::{ create_ffi_abs_func, create_ffi_random_func, create_ffi_rank_func, create_ffi_stddev_func, create_ffi_sum_func, create_ffi_table_func, }; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; mod async_provider; pub mod catalog; @@ -61,12 +60,11 @@ pub mod utils; pub struct ForeignLibraryModule { /// Construct an opinionated catalog provider pub create_catalog: - extern "C" fn(function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_CatalogProvider, + extern "C" fn(task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_CatalogProvider, /// Constructs the table provider pub create_table: extern "C" fn( synchronous: bool, - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider, @@ -76,14 +74,13 @@ pub struct ForeignLibraryModule { pub create_nullary_udf: extern "C" fn() -> FFI_ScalarUDF, pub create_table_function: - extern "C" fn(function_registry: FFI_WeakFunctionRegistry, - task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_TableFunction, + extern "C" fn(task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_TableFunction, /// Create an aggregate UDAF using sum - pub create_sum_udaf: extern "C" fn() -> FFI_AggregateUDF, + pub create_sum_udaf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_AggregateUDF, /// Create grouping UDAF using stddev - pub create_stddev_udaf: extern "C" fn() -> FFI_AggregateUDF, + pub create_stddev_udaf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_AggregateUDF, pub create_rank_udwf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_WindowUDF, @@ -120,12 +117,11 @@ pub fn create_record_batch(start_value: i32, num_values: usize) -> RecordBatch { /// We create an in-memory table and convert it to it's FFI counterpart. extern "C" fn construct_table_provider( synchronous: bool, - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { match synchronous { - true => create_sync_table_provider(function_registry, task_ctx_accessor), - false => create_async_table_provider(function_registry, task_ctx_accessor), + true => create_sync_table_provider(task_ctx_accessor), + false => create_async_table_provider(task_ctx_accessor), } } diff --git a/datafusion/ffi/src/tests/sync_provider.rs b/datafusion/ffi/src/tests/sync_provider.rs index cd23c4b1d0b0..bde8939afe91 100644 --- a/datafusion/ffi/src/tests/sync_provider.rs +++ b/datafusion/ffi/src/tests/sync_provider.rs @@ -16,14 +16,12 @@ // under the License. use super::{create_record_batch, create_test_schema}; -use crate::function_registry::FFI_WeakFunctionRegistry; use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::table_provider::FFI_TableProvider; use datafusion_catalog::MemTable; use std::sync::Arc; pub(crate) fn create_sync_table_provider( - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableProvider { let schema = create_test_schema(); @@ -38,11 +36,5 @@ pub(crate) fn create_sync_table_provider( let table_provider = MemTable::try_new(schema, vec![batches]).unwrap(); - FFI_TableProvider::new( - Arc::new(table_provider), - true, - None, - function_registry, - task_ctx_accessor, - ) + FFI_TableProvider::new(Arc::new(table_provider), true, None, task_ctx_accessor) } diff --git a/datafusion/ffi/src/tests/udf_udaf_udwf.rs b/datafusion/ffi/src/tests/udf_udaf_udwf.rs index c777d12bb367..f6a6f09c305c 100644 --- a/datafusion/ffi/src/tests/udf_udaf_udwf.rs +++ b/datafusion/ffi/src/tests/udf_udaf_udwf.rs @@ -20,7 +20,7 @@ use crate::{ udwf::FFI_WindowUDF, }; -use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use datafusion_catalog::TableFunctionImpl; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use datafusion_functions::math::abs::AbsFunc; @@ -30,7 +30,6 @@ use datafusion_functions_aggregate::sum::Sum; use datafusion_functions_table::generate_series::RangeFunc; use datafusion_functions_window::rank::Rank; use std::sync::Arc; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; pub(crate) extern "C" fn create_ffi_abs_func() -> FFI_ScalarUDF { let udf: Arc = Arc::new(AbsFunc::new().into()); @@ -45,27 +44,32 @@ pub(crate) extern "C" fn create_ffi_random_func() -> FFI_ScalarUDF { } pub(crate) extern "C" fn create_ffi_table_func( - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> FFI_TableFunction { let udtf: Arc = Arc::new(RangeFunc {}); - FFI_TableFunction::new(udtf, None, function_registry, task_ctx_accessor) + FFI_TableFunction::new(udtf, None, task_ctx_accessor) } -pub(crate) extern "C" fn create_ffi_sum_func() -> FFI_AggregateUDF { +pub(crate) extern "C" fn create_ffi_sum_func( + task_ctx_accessor: FFI_TaskContextAccessor, +) -> FFI_AggregateUDF { let udaf: Arc = Arc::new(Sum::new().into()); - udaf.into() + FFI_AggregateUDF::new(udaf, task_ctx_accessor) } -pub(crate) extern "C" fn create_ffi_stddev_func() -> FFI_AggregateUDF { +pub(crate) extern "C" fn create_ffi_stddev_func( + task_ctx_accessor: FFI_TaskContextAccessor, +) -> FFI_AggregateUDF { let udaf: Arc = Arc::new(Stddev::new().into()); - udaf.into() + FFI_AggregateUDF::new(udaf, task_ctx_accessor) } -pub(crate) extern "C" fn create_ffi_rank_func(task_ctx_accessor: FFI_TaskContextAccessor) -> FFI_WindowUDF { +pub(crate) extern "C" fn create_ffi_rank_func( + task_ctx_accessor: FFI_TaskContextAccessor, +) -> FFI_WindowUDF { let udwf: Arc = Arc::new( Rank::new( "rank_demo".to_string(), diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index f85659066aea..3dcb69e62706 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -18,21 +18,27 @@ use std::sync::Arc; use crate::arrow_wrappers::WrappedSchema; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use abi_stable::{ std_types::{RString, RVec}, StableAbi, }; use arrow::{datatypes::Schema, ffi::FFI_ArrowSchema}; use arrow_schema::FieldRef; -use prost::Message; use datafusion_common::error::DataFusionError; use datafusion_common::exec_datafusion_err; +use datafusion_execution::TaskContext; use datafusion_expr::function::AccumulatorArgs; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_proto::physical_plan::from_proto::{ + parse_physical_exprs, parse_physical_sort_exprs, +}; +use datafusion_proto::physical_plan::to_proto::{ + serialize_physical_exprs, serialize_physical_sort_exprs, +}; use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; -use datafusion_proto::physical_plan::from_proto::{parse_physical_exprs, parse_physical_sort_exprs}; -use datafusion_proto::physical_plan::to_proto::{serialize_physical_exprs, serialize_physical_sort_exprs}; use datafusion_proto::protobuf::PhysicalAggregateExprNode; +use prost::Message; /// A stable struct for sharing [`AccumulatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding field @@ -48,12 +54,14 @@ pub struct FFI_AccumulatorArgs { is_reversed: bool, name: RString, physical_expr_def: RVec, + task_ctx_accessor: FFI_TaskContextAccessor, } -impl TryFrom> for FFI_AccumulatorArgs { - type Error = DataFusionError; - - fn try_from(args: AccumulatorArgs) -> Result { +impl FFI_AccumulatorArgs { + pub fn try_new( + args: AccumulatorArgs, + task_ctx_accessor: FFI_TaskContextAccessor, + ) -> Result { let return_field = WrappedSchema(FFI_ArrowSchema::try_from(args.return_field.as_ref())?); let schema = WrappedSchema(FFI_ArrowSchema::try_from(args.schema)?); @@ -83,6 +91,7 @@ impl TryFrom> for FFI_AccumulatorArgs { is_distinct: args.is_distinct, name: args.name.into(), physical_expr_def, + task_ctx_accessor, }) } } @@ -110,14 +119,14 @@ impl TryFrom for ForeignAccumulatorArgs { let proto_def = PhysicalAggregateExprNode::decode( value.physical_expr_def.as_ref(), ) - .map_err(|e| { - exec_datafusion_err!("Failed to decode PhysicalAggregateExprNode: {e}") - })?; + .map_err(|e| { + exec_datafusion_err!("Failed to decode PhysicalAggregateExprNode: {e}") + })?; let return_field = Arc::new((&value.return_field.0).try_into()?); let schema = Schema::try_from(&value.schema.0)?; - let task_ctx = default_ctx.task_ctx(); + let task_ctx: Arc = (&value.task_ctx_accessor).try_into()?; let codex = DefaultPhysicalExtensionCodec {}; let order_bys = parse_physical_sort_exprs( @@ -129,9 +138,6 @@ impl TryFrom for ForeignAccumulatorArgs { let exprs = parse_physical_exprs(&proto_def.expr, &task_ctx, &schema, &codex)?; - let return_field = Arc::new((&value.return_field.0).try_into()?); - let schema = Schema::try_from(&value.schema.0)?; - let expr_fields = exprs .iter() .map(|e| e.return_field(&schema)) @@ -171,10 +177,13 @@ impl<'a> From<&'a ForeignAccumulatorArgs> for AccumulatorArgs<'a> { mod tests { use super::{FFI_AccumulatorArgs, ForeignAccumulatorArgs}; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion::prelude::SessionContext; use datafusion::{ error::Result, logical_expr::function::AccumulatorArgs, physical_expr::PhysicalSortExpr, physical_plan::expressions::col, }; + use datafusion_execution::TaskContextAccessor; + use std::sync::Arc; #[test] fn test_round_trip_accumulator_args() -> Result<()> { @@ -191,8 +200,10 @@ mod tests { exprs: &[col("a", &schema)?], }; let orig_str = format!("{orig_args:?}"); + let task_ctx_accessor = + Arc::new(SessionContext::new()) as Arc; - let ffi_args: FFI_AccumulatorArgs = orig_args.try_into()?; + let ffi_args = FFI_AccumulatorArgs::try_new(orig_args, task_ctx_accessor.into())?; let foreign_args: ForeignAccumulatorArgs = ffi_args.try_into()?; let round_trip_args: AccumulatorArgs = (&foreign_args).into(); diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index 49bae72825af..a99299135bad 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -40,6 +40,7 @@ use groups_accumulator::FFI_GroupsAccumulator; use std::hash::{Hash, Hasher}; use std::{ffi::c_void, sync::Arc}; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::util::{rvec_wrapped_to_vec_fieldref, vec_fieldref_to_rvec_wrapped}; use crate::{ arrow_wrappers::WrappedSchema, @@ -132,6 +133,8 @@ pub struct FFI_AggregateUDF { arg_types: RVec, ) -> RResult, RString>, + task_ctx_accessor: FFI_TaskContextAccessor, + /// Used to create a clone on the provider of the udaf. This should /// only need to be called by the receiver of the udaf. pub clone: unsafe extern "C" fn(udaf: &Self) -> Self, @@ -237,6 +240,7 @@ unsafe extern "C" fn with_beneficial_ordering_fn_wrapper( udaf: &FFI_AggregateUDF, beneficial_ordering: bool, ) -> RResult, RString> { + let task_ctx_accessor = udaf.task_ctx_accessor.clone(); let udaf = udaf.inner().as_ref().clone(); let result = rresult_return!(udaf.with_beneficial_ordering(beneficial_ordering)); @@ -244,7 +248,7 @@ unsafe extern "C" fn with_beneficial_ordering_fn_wrapper( .map(|func| func.with_beneficial_ordering(beneficial_ordering)) .transpose()) .flatten() - .map(|func| FFI_AggregateUDF::from(Arc::new(func))); + .map(|func| FFI_AggregateUDF::new(Arc::new(func), task_ctx_accessor)); RResult::ROk(result.into()) } @@ -327,7 +331,7 @@ unsafe extern "C" fn release_fn_wrapper(udaf: &mut FFI_AggregateUDF) { } unsafe extern "C" fn clone_fn_wrapper(udaf: &FFI_AggregateUDF) -> FFI_AggregateUDF { - Arc::clone(udaf.inner()).into() + FFI_AggregateUDF::new(Arc::clone(udaf.inner()), udaf.task_ctx_accessor.clone()) } impl Clone for FFI_AggregateUDF { @@ -336,14 +340,11 @@ impl Clone for FFI_AggregateUDF { } } -impl From<&Arc> for FFI_AggregateUDF { - fn from(udf: &Arc) -> Self { - Arc::clone(udf).into() - } -} - -impl From> for FFI_AggregateUDF { - fn from(udaf: Arc) -> Self { +impl FFI_AggregateUDF { + pub fn new( + udaf: Arc, + task_ctx_accessor: FFI_TaskContextAccessor, + ) -> Self { let name = udaf.name().into(); let aliases = udaf.aliases().iter().map(|a| a.to_owned().into()).collect(); let is_nullable = udaf.is_nullable(); @@ -365,6 +366,7 @@ impl From> for FFI_AggregateUDF { state_fields: state_fields_fn_wrapper, order_sensitivity: order_sensitivity_fn_wrapper, coerce_types: coerce_types_fn_wrapper, + task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, @@ -463,7 +465,8 @@ impl AggregateUDFImpl for ForeignAggregateUDF { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let args = acc_args.try_into()?; + let args = + FFI_AccumulatorArgs::try_new(acc_args, self.udaf.task_ctx_accessor.clone())?; unsafe { df_result!((self.udaf.accumulator)(&self.udaf, args)) .map(>::from) @@ -510,13 +513,15 @@ impl AggregateUDFImpl for ForeignAggregateUDF { } fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { - let args = match FFI_AccumulatorArgs::try_from(args) { - Ok(v) => v, - Err(e) => { - log::warn!("Attempting to convert accumulator arguments: {e}"); - return false; - } - }; + let args = + match FFI_AccumulatorArgs::try_new(args, self.udaf.task_ctx_accessor.clone()) + { + Ok(v) => v, + Err(e) => { + log::warn!("Attempting to convert accumulator arguments: {e}"); + return false; + } + }; unsafe { (self.udaf.groups_accumulator_supported)(&self.udaf, args) } } @@ -525,7 +530,8 @@ impl AggregateUDFImpl for ForeignAggregateUDF { &self, args: AccumulatorArgs, ) -> Result> { - let args = FFI_AccumulatorArgs::try_from(args)?; + let args = + FFI_AccumulatorArgs::try_new(args, self.udaf.task_ctx_accessor.clone())?; unsafe { df_result!((self.udaf.create_groups_accumulator)(&self.udaf, args)) @@ -541,7 +547,8 @@ impl AggregateUDFImpl for ForeignAggregateUDF { &self, args: AccumulatorArgs, ) -> Result> { - let args = args.try_into()?; + let args = + FFI_AccumulatorArgs::try_new(args, self.udaf.task_ctx_accessor.clone())?; unsafe { df_result!((self.udaf.create_sliding_accumulator)(&self.udaf, args)) .map(>::from) @@ -617,17 +624,18 @@ impl From for FFI_AggregateOrderSensitivity { #[cfg(test)] mod tests { + use super::*; use arrow::datatypes::Schema; + use datafusion::prelude::SessionContext; use datafusion::{ common::create_array, functions_aggregate::sum::Sum, physical_expr::PhysicalSortExpr, physical_plan::expressions::col, scalar::ScalarValue, }; + use datafusion_execution::TaskContextAccessor; use std::any::Any; use std::collections::HashMap; - use super::*; - #[derive(Default, Debug, Hash, Eq, PartialEq)] struct SumWithCopiedMetadata { inner: Sum, @@ -663,9 +671,12 @@ mod tests { fn create_test_foreign_udaf( original_udaf: impl AggregateUDFImpl + 'static, ) -> Result { + let task_ctx_accessor = + Arc::new(SessionContext::new()) as Arc; let original_udaf = Arc::new(AggregateUDF::from(original_udaf)); - let mut local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + let mut local_udaf = + FFI_AggregateUDF::new(Arc::clone(&original_udaf), task_ctx_accessor.into()); local_udaf.library_marker_id = crate::mock_foreign_marker_id; let foreign_udaf: Arc = (&local_udaf).try_into()?; @@ -677,9 +688,12 @@ mod tests { let original_udaf = Sum::new(); let original_name = original_udaf.name().to_owned(); let original_udaf = Arc::new(AggregateUDF::from(original_udaf)); + let task_ctx_accessor = + Arc::new(SessionContext::new()) as Arc; // Convert to FFI format - let mut local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + let mut local_udaf = + FFI_AggregateUDF::new(Arc::clone(&original_udaf), task_ctx_accessor.into()); local_udaf.library_marker_id = crate::mock_foreign_marker_id; // Convert back to native format @@ -732,9 +746,12 @@ mod tests { fn test_round_trip_udaf_metadata() -> Result<()> { let original_udaf = SumWithCopiedMetadata::default(); let original_udaf = Arc::new(AggregateUDF::from(original_udaf)); + let task_ctx_accessor = + Arc::new(SessionContext::new()) as Arc; // Convert to FFI format - let mut local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into(); + let mut local_udaf = + FFI_AggregateUDF::new(Arc::clone(&original_udaf), task_ctx_accessor.into()); local_udaf.library_marker_id = crate::mock_foreign_marker_id; // Convert back to native format diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index 56c8612cdee4..bf23010a07ce 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -22,11 +22,11 @@ use abi_stable::{ StableAbi, }; -use crate::function_registry::FFI_WeakFunctionRegistry; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::{df_result, rresult_return, table_provider::FFI_TableProvider}; use datafusion_catalog::{TableFunctionImpl, TableProvider}; use datafusion_common::error::Result; -use datafusion_expr::registry::FunctionRegistry; +use datafusion_execution::TaskContext; use datafusion_expr::Expr; use datafusion_proto::{ logical_plan::{ @@ -36,7 +36,6 @@ use datafusion_proto::{ }; use prost::Message; use tokio::runtime::Handle; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing a [`TableFunctionImpl`] across FFI boundaries. #[repr(C)] @@ -50,8 +49,6 @@ pub struct FFI_TableFunction { args: RVec, ) -> RResult, - pub function_registry: FFI_WeakFunctionRegistry, - pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the udtf. This should @@ -94,11 +91,9 @@ unsafe extern "C" fn call_fn_wrapper( udtf: &FFI_TableFunction, args: RVec, ) -> RResult { - let function_registry = udtf.function_registry.clone(); let task_ctx_accessor = udtf.task_ctx_accessor.clone(); - let foreign_registry = rresult_return!( - >::try_from(&udtf.function_registry) - ); + let task_ctx: Arc = + rresult_return!((&udtf.task_ctx_accessor).try_into()); let runtime = udtf.runtime(); let udtf = udtf.inner(); @@ -109,7 +104,7 @@ unsafe extern "C" fn call_fn_wrapper( let args = rresult_return!(parse_exprs( proto_filters.expr.iter(), - foreign_registry.as_ref(), + task_ctx.as_ref(), &codec )); @@ -118,7 +113,6 @@ unsafe extern "C" fn call_fn_wrapper( table_provider, false, runtime, - function_registry, task_ctx_accessor, )) } @@ -129,12 +123,11 @@ unsafe extern "C" fn release_fn_wrapper(udtf: &mut FFI_TableFunction) { } unsafe extern "C" fn clone_fn_wrapper(udtf: &FFI_TableFunction) -> FFI_TableFunction { - let function_registry = udtf.function_registry.clone(); let task_ctx_accessor = udtf.task_ctx_accessor.clone(); let runtime = udtf.runtime(); let udtf = udtf.inner(); - FFI_TableFunction::new(Arc::clone(udtf), runtime, function_registry, task_ctx_accessor) + FFI_TableFunction::new(Arc::clone(udtf), runtime, task_ctx_accessor) } impl Clone for FFI_TableFunction { @@ -147,14 +140,12 @@ impl FFI_TableFunction { pub fn new( udtf: Arc, runtime: Option, - function_registry: FFI_WeakFunctionRegistry, task_ctx_accessor: FFI_TaskContextAccessor, ) -> Self { let private_data = Box::new(TableFunctionPrivateData { udtf, runtime }); Self { call: call_fn_wrapper, - function_registry, task_ctx_accessor, clone: clone_fn_wrapper, release: release_fn_wrapper, @@ -222,7 +213,6 @@ mod tests { catalog::MemTable, common::exec_err, prelude::lit, scalar::ScalarValue, }; use datafusion_execution::TaskContextAccessor; - use datafusion_expr::registry::FunctionRegistry; #[derive(Debug)] struct TestUDTF {} @@ -307,13 +297,10 @@ mod tests { let original_udtf = Arc::new(TestUDTF {}) as Arc; let ctx = Arc::new(SessionContext::default()); - let function_registry = - Arc::clone(&ctx) as Arc; let task_ctx_accessor = Arc::clone(&ctx) as Arc; let mut local_udtf: FFI_TableFunction = FFI_TableFunction::new( Arc::clone(&original_udtf), None, - function_registry.into(), task_ctx_accessor.into(), ); diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index a8874a01d18a..0fd5ddf2e3aa 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -44,6 +44,7 @@ mod partition_evaluator; mod partition_evaluator_args; mod range; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::util::{rvec_wrapped_to_vec_fieldref, vec_fieldref_to_rvec_wrapped}; use crate::{ arrow_wrappers::WrappedSchema, @@ -51,7 +52,6 @@ use crate::{ util::{rvec_wrapped_to_vec_datatype, vec_datatype_to_rvec_wrapped}, volatility::FFI_Volatility, }; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing a [`WindowUDF`] across FFI boundaries. #[repr(C)] @@ -327,7 +327,10 @@ impl WindowUDFImpl for ForeignWindowUDF { args: datafusion_expr::function::PartitionEvaluatorArgs, ) -> Result> { let evaluator = unsafe { - let args = FFI_PartitionEvaluatorArgs::try_new(args, self.udf.task_ctx_accessor.clone())?; + let args = FFI_PartitionEvaluatorArgs::try_new( + args, + self.udf.task_ctx_accessor.clone(), + )?; (self.udf.partition_evaluator)(&self.udf, args) }; @@ -399,8 +402,8 @@ mod tests { use datafusion::logical_expr::expr::Sort; use datafusion::logical_expr::{col, ExprFunctionExt, WindowUDF, WindowUDFImpl}; use datafusion::prelude::SessionContext; - use std::sync::Arc; use datafusion_execution::TaskContextAccessor; + use std::sync::Arc; fn create_test_foreign_udwf( original_udwf: impl WindowUDFImpl + 'static, @@ -419,10 +422,12 @@ mod tests { fn test_round_trip_udwf() -> datafusion::common::Result<()> { let original_udwf = lag_udwf(); let original_name = original_udwf.name().to_owned(); - let task_ctx_accessor = Arc::new(SessionContext::default()) as Arc; + let task_ctx_accessor = + Arc::new(SessionContext::default()) as Arc; // Convert to FFI format - let mut local_udwf = FFI_WindowUDF::new(Arc::clone(&original_udwf).into(), task_ctx_accessor.into()); + let mut local_udwf = + FFI_WindowUDF::new(Arc::clone(&original_udwf), task_ctx_accessor.into()); local_udwf.library_marker_id = crate::mock_foreign_marker_id; // Convert back to native format @@ -436,7 +441,10 @@ mod tests { #[tokio::test] async fn test_lag_udwf() -> datafusion::common::Result<()> { let ctx = Arc::new(SessionContext::default()); - let udwf = create_test_foreign_udwf(WindowShift::lag(), Arc::clone(&ctx) as Arc)?; + let udwf = create_test_foreign_udwf( + WindowShift::lag(), + Arc::clone(&ctx) as Arc, + )?; let df = ctx.read_batch(create_record_batch(-5, 5))?; diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index d3493d015291..46319668f233 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -18,6 +18,7 @@ use std::{collections::HashMap, sync::Arc}; use crate::arrow_wrappers::WrappedSchema; +use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use abi_stable::{std_types::RVec, StableAbi}; use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef}, @@ -25,17 +26,16 @@ use arrow::{ ffi::FFI_ArrowSchema, }; use arrow_schema::FieldRef; -use prost::Message; use datafusion_common::error::{DataFusionError, Result}; use datafusion_common::exec_datafusion_err; -use datafusion_execution::{TaskContext, TaskContextAccessor}; +use datafusion_execution::TaskContext; use datafusion_expr::function::PartitionEvaluatorArgs; use datafusion_physical_plan::{expressions::Column, PhysicalExpr}; -use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; use datafusion_proto::physical_plan::from_proto::parse_physical_expr; use datafusion_proto::physical_plan::to_proto::serialize_physical_exprs; +use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; use datafusion_proto::protobuf::PhysicalExprNode; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; +use prost::Message; /// A stable struct for sharing [`PartitionEvaluatorArgs`] across FFI boundaries. /// For an explanation of each field, see the corresponding function @@ -53,7 +53,10 @@ pub struct FFI_PartitionEvaluatorArgs { } impl FFI_PartitionEvaluatorArgs { - pub fn try_new(args: PartitionEvaluatorArgs, task_ctx_accessor: FFI_TaskContextAccessor) -> Result { + pub fn try_new( + args: PartitionEvaluatorArgs, + task_ctx_accessor: FFI_TaskContextAccessor, + ) -> Result { // This is a bit of a hack. Since PartitionEvaluatorArgs does not carry a schema // around, and instead passes the data types directly we are unable to decode the // protobuf PhysicalExpr correctly. In evaluating the code the only place these @@ -145,9 +148,7 @@ impl TryFrom for ForeignPartitionEvaluatorArgs { .collect::, prost::DecodeError>>() .map_err(|e| exec_datafusion_err!("Failed to decode PhysicalExprNode: {e}"))? .iter() - .map(|expr_node| { - parse_physical_expr(expr_node, &task_ctx, &schema, &codec) - }) + .map(|expr_node| parse_physical_expr(expr_node, &task_ctx, &schema, &codec)) .collect::>>()?; let input_fields = input_exprs diff --git a/datafusion/ffi/tests/ffi_integration.rs b/datafusion/ffi/tests/ffi_integration.rs index 92f88790f6c9..54fbe7b7b793 100644 --- a/datafusion/ffi/tests/ffi_integration.rs +++ b/datafusion/ffi/tests/ffi_integration.rs @@ -22,7 +22,7 @@ mod tests { use datafusion::error::{DataFusionError, Result}; use datafusion::prelude::SessionContext; use datafusion_catalog::{CatalogProvider, TableProvider}; - use datafusion_expr::registry::FunctionRegistry; + use datafusion_execution::TaskContextAccessor; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; use std::sync::Arc; @@ -34,8 +34,7 @@ mod tests { let table_provider_module = get_module()?; let ctx = Arc::new(SessionContext::new()); - let function_registry = - Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; // By calling the code below, the table provided will be created within // the module's code. @@ -43,7 +42,7 @@ mod tests { DataFusionError::NotImplemented( "External table provider failed to implement create_table".to_string(), ), - )?(synchronous, function_registry.into()); + )?(synchronous, task_ctx_accessor.into()); // In order to access the table provider within this executable, we need to // turn it into a `ForeignTableProvider`. @@ -76,8 +75,7 @@ mod tests { async fn test_catalog() -> Result<()> { let module = get_module()?; let ctx = Arc::new(SessionContext::default()); - let function_registry = - Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let ffi_catalog = module @@ -85,7 +83,7 @@ mod tests { .ok_or(DataFusionError::NotImplemented( "External catalog provider failed to implement create_catalog" .to_string(), - ))?(function_registry.into()); + ))?(task_ctx_accessor.into()); let foreign_catalog: Arc = (&ffi_catalog).into(); let _ = ctx.register_catalog("fruit", foreign_catalog); diff --git a/datafusion/ffi/tests/ffi_udaf.rs b/datafusion/ffi/tests/ffi_udaf.rs index 17db7a64561a..bb4893802cde 100644 --- a/datafusion/ffi/tests/ffi_udaf.rs +++ b/datafusion/ffi/tests/ffi_udaf.rs @@ -24,6 +24,7 @@ mod tests { use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::AggregateUDF; use datafusion::prelude::{col, SessionContext}; + use datafusion_execution::TaskContextAccessor; use datafusion_expr::AggregateUDFImpl; use datafusion_ffi::tests::utils::get_module; use std::sync::Arc; @@ -31,18 +32,19 @@ mod tests { #[tokio::test] async fn test_ffi_udaf() -> Result<()> { let module = get_module()?; + let ctx = Arc::new(SessionContext::default()); + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let ffi_sum_func = module .create_sum_udaf() .ok_or(DataFusionError::NotImplemented( "External table provider failed to implement create_udaf".to_string(), - ))?(); + ))?(task_ctx_accessor.into()); let foreign_sum_func: Arc = (&ffi_sum_func).try_into()?; let udaf = AggregateUDF::new_from_shared_impl(foreign_sum_func); - let ctx = SessionContext::default(); let record_batch = record_batch!( ("a", Int32, vec![1, 2, 2, 4, 4, 4, 4]), ("b", Float64, vec![1.0, 2.0, 2.0, 4.0, 4.0, 4.0, 4.0]) @@ -73,19 +75,20 @@ mod tests { #[tokio::test] async fn test_ffi_grouping_udaf() -> Result<()> { let module = get_module()?; + let ctx = Arc::new(SessionContext::default()); + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let ffi_stddev_func = module .create_stddev_udaf() .ok_or(DataFusionError::NotImplemented( "External table provider failed to implement create_udaf".to_string(), - ))?(); + ))?(task_ctx_accessor.into()); let foreign_stddev_func: Arc = (&ffi_stddev_func).try_into()?; let udaf = AggregateUDF::new_from_shared_impl(foreign_stddev_func); - let ctx = SessionContext::default(); let record_batch = record_batch!( ("a", Int32, vec![1, 2, 2, 4, 4, 4, 4]), ( diff --git a/datafusion/ffi/tests/ffi_udtf.rs b/datafusion/ffi/tests/ffi_udtf.rs index 205934650aa4..aa60fff366c5 100644 --- a/datafusion/ffi/tests/ffi_udtf.rs +++ b/datafusion/ffi/tests/ffi_udtf.rs @@ -26,7 +26,7 @@ mod tests { use datafusion::error::{DataFusionError, Result}; use datafusion::prelude::SessionContext; use datafusion_catalog::TableFunctionImpl; - use datafusion_expr::registry::FunctionRegistry; + use datafusion_execution::TaskContextAccessor; use datafusion_ffi::tests::utils::get_module; /// This test validates that we can load an external module and use a scalar @@ -37,15 +37,14 @@ mod tests { let module = get_module()?; let ctx = Arc::new(SessionContext::default()); - let function_registry = - Arc::clone(&ctx) as Arc; + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let ffi_table_func = module .create_table_function() .ok_or(DataFusionError::NotImplemented( "External table function provider failed to implement create_table_function" .to_string(), - ))?(function_registry.into()); + ))?(task_ctx_accessor.into()); let foreign_table_func: Arc = ffi_table_func.into(); ctx.register_udtf("my_range", foreign_table_func); diff --git a/datafusion/ffi/tests/ffi_udwf.rs b/datafusion/ffi/tests/ffi_udwf.rs index d663a17982ee..073d8afa5abb 100644 --- a/datafusion/ffi/tests/ffi_udwf.rs +++ b/datafusion/ffi/tests/ffi_udwf.rs @@ -24,6 +24,7 @@ mod tests { use datafusion::logical_expr::expr::Sort; use datafusion::logical_expr::{col, ExprFunctionExt, WindowUDF}; use datafusion::prelude::SessionContext; + use datafusion_execution::TaskContextAccessor; use datafusion_expr::WindowUDFImpl; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; @@ -32,6 +33,8 @@ mod tests { #[tokio::test] async fn test_rank_udwf() -> Result<()> { let module = get_module()?; + let ctx = Arc::new(SessionContext::default()); + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let ffi_rank_func = module @@ -39,12 +42,11 @@ mod tests { .ok_or(DataFusionError::NotImplemented( "External table provider failed to implement create_scalar_udf" .to_string(), - ))?(); + ))?(task_ctx_accessor.into()); let foreign_rank_func: Arc = (&ffi_rank_func).try_into()?; let udwf = WindowUDF::new_from_shared_impl(foreign_rank_func); - let ctx = SessionContext::default(); let df = ctx.read_batch(create_record_batch(-5, 5))?; let df = df.select(vec![ From c243a6d5af333f9f99c15ba91189bfa76de506e5 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 14:47:58 -0500 Subject: [PATCH 49/69] Set scope for task context accessor in tests --- datafusion/ffi/src/udaf/accumulator_args.rs | 4 ++-- datafusion/ffi/src/udaf/mod.rs | 15 ++++++++++----- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 3dcb69e62706..5c6caf613c19 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -200,8 +200,8 @@ mod tests { exprs: &[col("a", &schema)?], }; let orig_str = format!("{orig_args:?}"); - let task_ctx_accessor = - Arc::new(SessionContext::new()) as Arc; + let ctx = Arc::new(SessionContext::new()); + let task_ctx_accessor = Arc::clone(&ctx) as Arc; let ffi_args = FFI_AccumulatorArgs::try_new(orig_args, task_ctx_accessor.into())?; let foreign_args: ForeignAccumulatorArgs = ffi_args.try_into()?; diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index a99299135bad..59e4d928aac2 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -670,9 +670,9 @@ mod tests { fn create_test_foreign_udaf( original_udaf: impl AggregateUDFImpl + 'static, + ctx: &Arc, ) -> Result { - let task_ctx_accessor = - Arc::new(SessionContext::new()) as Arc; + let task_ctx_accessor = Arc::clone(ctx) as Arc; let original_udaf = Arc::new(AggregateUDF::from(original_udaf)); let mut local_udaf = @@ -706,8 +706,9 @@ mod tests { #[test] fn test_foreign_udaf_aliases() -> Result<()> { + let ctx = Arc::new(SessionContext::new()); let foreign_udaf = - create_test_foreign_udaf(Sum::new())?.with_aliases(["my_function"]); + create_test_foreign_udaf(Sum::new(), &ctx)?.with_aliases(["my_function"]); let return_field = foreign_udaf @@ -719,7 +720,8 @@ mod tests { #[test] fn test_foreign_udaf_accumulator() -> Result<()> { - let foreign_udaf = create_test_foreign_udaf(Sum::new())?; + let ctx = Arc::new(SessionContext::new()); + let foreign_udaf = create_test_foreign_udaf(Sum::new(), &ctx)?; let schema = Schema::new(vec![Field::new("a", DataType::Float64, true)]); let acc_args = AccumulatorArgs { @@ -773,8 +775,10 @@ mod tests { #[test] fn test_beneficial_ordering() -> Result<()> { + let ctx = Arc::new(SessionContext::new()); let foreign_udaf = create_test_foreign_udaf( datafusion::functions_aggregate::first_last::FirstValue::new(), + &ctx, )?; let foreign_udaf = foreign_udaf.with_beneficial_ordering(true)?.unwrap(); @@ -800,7 +804,8 @@ mod tests { #[test] fn test_sliding_accumulator() -> Result<()> { - let foreign_udaf = create_test_foreign_udaf(Sum::new())?; + let ctx = Arc::new(SessionContext::new()); + let foreign_udaf = create_test_foreign_udaf(Sum::new(), &ctx)?; let schema = Schema::new(vec![Field::new("a", DataType::Float64, true)]); // Note: sum distinct is only support Int64 until now From e39a6c3930ecd5112e4aec9c36d3078f945b6336 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 14:53:15 -0500 Subject: [PATCH 50/69] Remove unused expr code --- datafusion/ffi/src/expr/columnar_value.rs | 57 ------ datafusion/ffi/src/expr/distribution.rs | 213 --------------------- datafusion/ffi/src/expr/expr_properties.rs | 113 ----------- datafusion/ffi/src/expr/interval.rs | 63 ------ datafusion/ffi/src/expr/mod.rs | 22 --- datafusion/ffi/src/expr/util.rs | 34 ---- datafusion/ffi/src/lib.rs | 1 - 7 files changed, 503 deletions(-) delete mode 100644 datafusion/ffi/src/expr/columnar_value.rs delete mode 100644 datafusion/ffi/src/expr/distribution.rs delete mode 100644 datafusion/ffi/src/expr/expr_properties.rs delete mode 100644 datafusion/ffi/src/expr/interval.rs delete mode 100644 datafusion/ffi/src/expr/mod.rs delete mode 100644 datafusion/ffi/src/expr/util.rs diff --git a/datafusion/ffi/src/expr/columnar_value.rs b/datafusion/ffi/src/expr/columnar_value.rs deleted file mode 100644 index ac98d4d09a37..000000000000 --- a/datafusion/ffi/src/expr/columnar_value.rs +++ /dev/null @@ -1,57 +0,0 @@ -// 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::arrow_wrappers::WrappedArray; -use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; -use abi_stable::std_types::RVec; -use abi_stable::StableAbi; -use datafusion_common::DataFusionError; -use datafusion_expr::ColumnarValue; - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub enum FFI_ColumnarValue { - Array(WrappedArray), - Scalar(RVec), -} - -impl TryFrom for FFI_ColumnarValue { - type Error = DataFusionError; - fn try_from(value: ColumnarValue) -> Result { - Ok(match value { - ColumnarValue::Array(v) => { - FFI_ColumnarValue::Array(WrappedArray::try_from(&v)?) - } - ColumnarValue::Scalar(v) => { - FFI_ColumnarValue::Scalar(scalar_value_to_rvec_u8(&v)?) - } - }) - } -} - -impl TryFrom for ColumnarValue { - type Error = DataFusionError; - fn try_from(value: FFI_ColumnarValue) -> Result { - Ok(match value { - FFI_ColumnarValue::Array(v) => ColumnarValue::Array(v.try_into()?), - FFI_ColumnarValue::Scalar(v) => { - ColumnarValue::Scalar(rvec_u8_to_scalar_value(&v)?) - } - }) - } -} diff --git a/datafusion/ffi/src/expr/distribution.rs b/datafusion/ffi/src/expr/distribution.rs deleted file mode 100644 index c17768c590a0..000000000000 --- a/datafusion/ffi/src/expr/distribution.rs +++ /dev/null @@ -1,213 +0,0 @@ -// 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::expr::interval::FFI_Interval; -use crate::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; -use abi_stable::std_types::RVec; -use abi_stable::StableAbi; -use datafusion_common::DataFusionError; -use datafusion_expr::statistics::{ - BernoulliDistribution, Distribution, ExponentialDistribution, GaussianDistribution, - GenericDistribution, UniformDistribution, -}; - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub enum FFI_Distribution { - Uniform(FFI_UniformDistribution), - Exponential(FFI_ExponentialDistribution), - Gaussian(FFI_GaussianDistribution), - Bernoulli(FFI_BernoulliDistribution), - Generic(FFI_GenericDistribution), -} - -impl TryFrom<&Distribution> for FFI_Distribution { - type Error = DataFusionError; - fn try_from(value: &Distribution) -> Result { - match value { - Distribution::Uniform(d) => Ok(FFI_Distribution::Uniform(d.try_into()?)), - Distribution::Exponential(d) => { - Ok(FFI_Distribution::Exponential(d.try_into()?)) - } - Distribution::Gaussian(d) => Ok(FFI_Distribution::Gaussian(d.try_into()?)), - Distribution::Bernoulli(d) => Ok(FFI_Distribution::Bernoulli(d.try_into()?)), - Distribution::Generic(d) => Ok(FFI_Distribution::Generic(d.try_into()?)), - } - } -} - -impl TryFrom<&FFI_Distribution> for Distribution { - type Error = DataFusionError; - fn try_from(value: &FFI_Distribution) -> Result { - match value { - FFI_Distribution::Uniform(d) => d.try_into(), - FFI_Distribution::Exponential(d) => d.try_into(), - FFI_Distribution::Gaussian(d) => d.try_into(), - FFI_Distribution::Bernoulli(d) => d.try_into(), - FFI_Distribution::Generic(d) => d.try_into(), - } - } -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_UniformDistribution { - interval: FFI_Interval, -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_ExponentialDistribution { - rate: RVec, - offset: RVec, - positive_tail: bool, -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_GaussianDistribution { - mean: RVec, - variance: RVec, -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_BernoulliDistribution { - p: RVec, -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_GenericDistribution { - mean: RVec, - median: RVec, - variance: RVec, - range: FFI_Interval, -} - -impl TryFrom<&UniformDistribution> for FFI_UniformDistribution { - type Error = DataFusionError; - fn try_from(value: &UniformDistribution) -> Result { - Ok(Self { - interval: value.range().try_into()?, - }) - } -} - -impl TryFrom<&ExponentialDistribution> for FFI_ExponentialDistribution { - type Error = DataFusionError; - fn try_from(value: &ExponentialDistribution) -> Result { - let rate = scalar_value_to_rvec_u8(value.rate())?; - let offset = scalar_value_to_rvec_u8(value.offset())?; - - Ok(Self { - rate, - offset, - positive_tail: value.positive_tail(), - }) - } -} - -impl TryFrom<&GaussianDistribution> for FFI_GaussianDistribution { - type Error = DataFusionError; - fn try_from(value: &GaussianDistribution) -> Result { - let mean = scalar_value_to_rvec_u8(value.mean())?; - let variance = scalar_value_to_rvec_u8(value.variance())?; - - Ok(Self { mean, variance }) - } -} - -impl TryFrom<&BernoulliDistribution> for FFI_BernoulliDistribution { - type Error = DataFusionError; - fn try_from(value: &BernoulliDistribution) -> Result { - let p = scalar_value_to_rvec_u8(value.p_value())?; - - Ok(Self { p }) - } -} - -impl TryFrom<&GenericDistribution> for FFI_GenericDistribution { - type Error = DataFusionError; - fn try_from(value: &GenericDistribution) -> Result { - let mean = scalar_value_to_rvec_u8(value.mean())?; - let median = scalar_value_to_rvec_u8(value.median())?; - let variance = scalar_value_to_rvec_u8(value.variance())?; - - Ok(Self { - mean, - median, - variance, - range: value.range().try_into()?, - }) - } -} - -impl TryFrom<&FFI_UniformDistribution> for Distribution { - type Error = DataFusionError; - fn try_from(value: &FFI_UniformDistribution) -> Result { - let interval = (&value.interval).try_into()?; - Distribution::new_uniform(interval) - } -} - -impl TryFrom<&FFI_ExponentialDistribution> for Distribution { - type Error = DataFusionError; - fn try_from(value: &FFI_ExponentialDistribution) -> Result { - let rate = rvec_u8_to_scalar_value(&value.rate)?; - let offset = rvec_u8_to_scalar_value(&value.offset)?; - - Distribution::new_exponential(rate, offset, value.positive_tail) - } -} - -impl TryFrom<&FFI_GaussianDistribution> for Distribution { - type Error = DataFusionError; - fn try_from(value: &FFI_GaussianDistribution) -> Result { - let mean = rvec_u8_to_scalar_value(&value.mean)?; - let variance = rvec_u8_to_scalar_value(&value.variance)?; - - Distribution::new_gaussian(mean, variance) - } -} - -impl TryFrom<&FFI_BernoulliDistribution> for Distribution { - type Error = DataFusionError; - fn try_from(value: &FFI_BernoulliDistribution) -> Result { - let p = rvec_u8_to_scalar_value(&value.p)?; - - Distribution::new_bernoulli(p) - } -} - -impl TryFrom<&FFI_GenericDistribution> for Distribution { - type Error = DataFusionError; - fn try_from(value: &FFI_GenericDistribution) -> Result { - let mean = rvec_u8_to_scalar_value(&value.mean)?; - let median = rvec_u8_to_scalar_value(&value.median)?; - let variance = rvec_u8_to_scalar_value(&value.variance)?; - let range = (&value.range).try_into()?; - - Distribution::new_generic(mean, median, variance, range) - } -} diff --git a/datafusion/ffi/src/expr/expr_properties.rs b/datafusion/ffi/src/expr/expr_properties.rs deleted file mode 100644 index b7bfabc22c2f..000000000000 --- a/datafusion/ffi/src/expr/expr_properties.rs +++ /dev/null @@ -1,113 +0,0 @@ -// 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::expr::interval::FFI_Interval; -use abi_stable::StableAbi; -use arrow_schema::SortOptions; -use datafusion_common::DataFusionError; -use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_ExprProperties { - pub sort_properties: FFI_SortProperties, - pub range: FFI_Interval, - pub preserves_lex_ordering: bool, -} - -impl TryFrom<&ExprProperties> for FFI_ExprProperties { - type Error = DataFusionError; - fn try_from(value: &ExprProperties) -> Result { - let sort_properties = (&value.sort_properties).into(); - let range = (&value.range).try_into()?; - - Ok(FFI_ExprProperties { - sort_properties, - range, - preserves_lex_ordering: value.preserves_lex_ordering, - }) - } -} - -impl TryFrom<&FFI_ExprProperties> for ExprProperties { - type Error = DataFusionError; - fn try_from(value: &FFI_ExprProperties) -> Result { - let sort_properties = (&value.sort_properties).into(); - let range = (&value.range).try_into()?; - Ok(ExprProperties { - sort_properties, - range, - preserves_lex_ordering: value.preserves_lex_ordering, - }) - } -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub enum FFI_SortProperties { - Ordered(FFI_SortOptions), - Unordered, - Singleton, -} - -impl From<&SortProperties> for FFI_SortProperties { - fn from(value: &SortProperties) -> Self { - match value { - SortProperties::Unordered => FFI_SortProperties::Unordered, - SortProperties::Singleton => FFI_SortProperties::Singleton, - SortProperties::Ordered(o) => FFI_SortProperties::Ordered(o.into()), - } - } -} - -impl From<&FFI_SortProperties> for SortProperties { - fn from(value: &FFI_SortProperties) -> Self { - match value { - FFI_SortProperties::Unordered => SortProperties::Unordered, - FFI_SortProperties::Singleton => SortProperties::Singleton, - FFI_SortProperties::Ordered(o) => SortProperties::Ordered(o.into()), - } - } -} - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_SortOptions { - pub descending: bool, - pub nulls_first: bool, -} - -impl From<&SortOptions> for FFI_SortOptions { - fn from(value: &SortOptions) -> Self { - Self { - descending: value.descending, - nulls_first: value.nulls_first, - } - } -} - -impl From<&FFI_SortOptions> for SortOptions { - fn from(value: &FFI_SortOptions) -> Self { - Self { - descending: value.descending, - nulls_first: value.nulls_first, - } - } -} diff --git a/datafusion/ffi/src/expr/interval.rs b/datafusion/ffi/src/expr/interval.rs deleted file mode 100644 index 59f9a7ecc3ed..000000000000 --- a/datafusion/ffi/src/expr/interval.rs +++ /dev/null @@ -1,63 +0,0 @@ -// 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::expr::util::{rvec_u8_to_scalar_value, scalar_value_to_rvec_u8}; -use abi_stable::std_types::RVec; -use abi_stable::StableAbi; -use datafusion_common::DataFusionError; -use datafusion_expr::interval_arithmetic::Interval; - -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_Interval { - lower: RVec, - upper: RVec, -} - -impl TryFrom<&Interval> for FFI_Interval { - type Error = DataFusionError; - fn try_from(value: &Interval) -> Result { - let upper = scalar_value_to_rvec_u8(value.upper())?; - let lower = scalar_value_to_rvec_u8(value.lower())?; - - Ok(FFI_Interval { upper, lower }) - } -} -impl TryFrom for FFI_Interval { - type Error = DataFusionError; - fn try_from(value: Interval) -> Result { - FFI_Interval::try_from(&value) - } -} - -impl TryFrom<&FFI_Interval> for Interval { - type Error = DataFusionError; - fn try_from(value: &FFI_Interval) -> Result { - let upper = rvec_u8_to_scalar_value(&value.upper)?; - let lower = rvec_u8_to_scalar_value(&value.lower)?; - - Interval::try_new(lower, upper) - } -} - -impl TryFrom for Interval { - type Error = DataFusionError; - fn try_from(value: FFI_Interval) -> Result { - Interval::try_from(&value) - } -} diff --git a/datafusion/ffi/src/expr/mod.rs b/datafusion/ffi/src/expr/mod.rs deleted file mode 100644 index 717b7b4a2a16..000000000000 --- a/datafusion/ffi/src/expr/mod.rs +++ /dev/null @@ -1,22 +0,0 @@ -// 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. - -pub mod columnar_value; -pub mod distribution; -pub mod expr_properties; -pub mod interval; -pub(crate) mod util; diff --git a/datafusion/ffi/src/expr/util.rs b/datafusion/ffi/src/expr/util.rs deleted file mode 100644 index f8ac973261a2..000000000000 --- a/datafusion/ffi/src/expr/util.rs +++ /dev/null @@ -1,34 +0,0 @@ -// 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 abi_stable::std_types::RVec; -use datafusion_common::{exec_datafusion_err, Result, ScalarValue}; -use prost::Message; - -pub fn scalar_value_to_rvec_u8(value: &ScalarValue) -> Result> { - let value: datafusion_proto_common::ScalarValue = value.try_into()?; - Ok(value.encode_to_vec().into()) -} - -pub fn rvec_u8_to_scalar_value(value: &RVec) -> Result { - let value = datafusion_proto_common::ScalarValue::decode(value.as_ref()) - .map_err(|err| exec_datafusion_err!("{err}"))?; - - (&value) - .try_into() - .map_err(|err| exec_datafusion_err!("{err}")) -} diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index fe7ded25e123..dd7a3cfdf97c 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -27,7 +27,6 @@ pub mod arrow_wrappers; pub mod catalog_provider; pub mod execution_plan; -pub mod expr; pub mod function_registry; pub mod insert_op; pub mod plan_properties; From f96549268661136039e21e9d383c79cc6a7ff9e9 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 14:54:13 -0500 Subject: [PATCH 51/69] Remove function registry code --- datafusion/ffi/src/function_registry.rs | 409 ------------------------ datafusion/ffi/src/lib.rs | 1 - 2 files changed, 410 deletions(-) delete mode 100644 datafusion/ffi/src/function_registry.rs diff --git a/datafusion/ffi/src/function_registry.rs b/datafusion/ffi/src/function_registry.rs deleted file mode 100644 index 8e98a2cc4c8d..000000000000 --- a/datafusion/ffi/src/function_registry.rs +++ /dev/null @@ -1,409 +0,0 @@ -// 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::session::task_ctx_accessor::FFI_TaskContextAccessor; -use crate::udaf::FFI_AggregateUDF; -use crate::udf::FFI_ScalarUDF; -use crate::udwf::FFI_WindowUDF; -use crate::{df_result, rresult_return}; -use abi_stable::{ - std_types::{RResult, RString, RVec}, - StableAbi, -}; -use datafusion_common::{exec_datafusion_err, not_impl_err, DataFusionError}; -use datafusion_expr::expr_rewriter::FunctionRewrite; -use datafusion_expr::planner::ExprPlanner; -use datafusion_expr::registry::FunctionRegistry; -use datafusion_expr::{ - AggregateUDF, AggregateUDFImpl, ScalarUDF, ScalarUDFImpl, WindowUDF, WindowUDFImpl, -}; -use log::warn; -use std::collections::HashSet; -use std::sync::Weak; -use std::{ffi::c_void, sync::Arc}; - -/// A stable struct for sharing [`FunctionRegistry`] across FFI boundaries. -#[repr(C)] -#[derive(Debug, StableAbi)] -#[allow(non_camel_case_types)] -pub struct FFI_WeakFunctionRegistry { - pub udfs: unsafe extern "C" fn(&Self) -> RResult, RString>, - pub udafs: unsafe extern "C" fn(&Self) -> RResult, RString>, - pub udwfs: unsafe extern "C" fn(&Self) -> RResult, RString>, - - pub udf: - unsafe extern "C" fn(&Self, name: RString) -> RResult, - pub udaf: - unsafe extern "C" fn(&Self, name: RString) -> RResult, - pub udwf: - unsafe extern "C" fn(&Self, name: RString) -> RResult, - - pub task_ctx_accessor: FFI_TaskContextAccessor, - - /// Used to create a clone on the provider of the registry. This should - /// only need to be called by the receiver of the plan. - pub clone: unsafe extern "C" fn(plan: &Self) -> Self, - - /// Release the memory of the private data when it is no longer being used. - pub release: unsafe extern "C" fn(arg: &mut Self), - - /// Return the major DataFusion version number of this registry. - pub version: unsafe extern "C" fn() -> u64, - - /// Internal data. This is only to be accessed by the provider of the plan. - /// A [`ForeignWeakFunctionRegistry`] should never attempt to access this data. - pub private_data: *mut c_void, - - /// Utility to identify when FFI objects are accessed locally through - /// the foreign interface. - pub library_marker_id: extern "C" fn() -> u64, -} - -unsafe impl Send for FFI_WeakFunctionRegistry {} -unsafe impl Sync for FFI_WeakFunctionRegistry {} - -struct RegistryPrivateData { - registry: Weak, -} - -impl FFI_WeakFunctionRegistry { - fn inner(&self) -> Result, DataFusionError> { - let private_data = self.private_data as *const RegistryPrivateData; - unsafe { - (*private_data) - .registry - .upgrade() - .ok_or_else(|| exec_datafusion_err!("Unable to access FunctionRegistry via FFI. Ensure owning object has not gone out of scope.")) - } - } -} - -unsafe extern "C" fn udfs_fn_wrapper( - registry: &FFI_WeakFunctionRegistry, -) -> RResult, RString> { - let inner = rresult_return!(registry.inner()); - let udfs = inner.udfs().into_iter().map(|s| s.into()).collect(); - RResult::ROk(udfs) -} -unsafe extern "C" fn udafs_fn_wrapper( - registry: &FFI_WeakFunctionRegistry, -) -> RResult, RString> { - let inner = rresult_return!(registry.inner()); - let udafs = inner.udafs().into_iter().map(|s| s.into()).collect(); - RResult::ROk(udafs) -} -unsafe extern "C" fn udwfs_fn_wrapper( - registry: &FFI_WeakFunctionRegistry, -) -> RResult, RString> { - let inner = rresult_return!(registry.inner()); - let udwfs = inner.udwfs().into_iter().map(|s| s.into()).collect(); - RResult::ROk(udwfs) -} - -unsafe extern "C" fn udf_fn_wrapper( - registry: &FFI_WeakFunctionRegistry, - name: RString, -) -> RResult { - let inner = rresult_return!(registry.inner()); - let udf = rresult_return!(inner.udf(name.as_str())); - RResult::ROk(FFI_ScalarUDF::from(udf)) -} -unsafe extern "C" fn udaf_fn_wrapper( - registry: &FFI_WeakFunctionRegistry, - name: RString, -) -> RResult { - let inner = rresult_return!(registry.inner()); - let udaf = rresult_return!(inner.udaf(name.as_str())); - RResult::ROk(FFI_AggregateUDF::new( - udaf, - registry.task_ctx_accessor.clone(), - )) -} -unsafe extern "C" fn udwf_fn_wrapper( - registry: &FFI_WeakFunctionRegistry, - name: RString, -) -> RResult { - let inner = rresult_return!(registry.inner()); - let udwf = rresult_return!(inner.udwf(name.as_str())); - RResult::ROk(FFI_WindowUDF::new(udwf, registry.task_ctx_accessor.clone())) -} - -unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_WeakFunctionRegistry) { - let private_data = Box::from_raw(provider.private_data as *mut RegistryPrivateData); - drop(private_data); -} - -unsafe extern "C" fn clone_fn_wrapper( - provider: &FFI_WeakFunctionRegistry, -) -> FFI_WeakFunctionRegistry { - let old_private_data = provider.private_data as *const RegistryPrivateData; - - let private_data = Box::into_raw(Box::new(RegistryPrivateData { - registry: Weak::clone(&(*old_private_data).registry), - })) as *mut c_void; - - FFI_WeakFunctionRegistry { - udfs: udfs_fn_wrapper, - udafs: udafs_fn_wrapper, - udwfs: udwfs_fn_wrapper, - - udf: udf_fn_wrapper, - udaf: udaf_fn_wrapper, - udwf: udwf_fn_wrapper, - - clone: clone_fn_wrapper, - release: release_fn_wrapper, - version: super::version, - task_ctx_accessor: provider.task_ctx_accessor.clone(), - private_data, - library_marker_id: crate::get_library_marker_id, - } -} - -impl Drop for FFI_WeakFunctionRegistry { - fn drop(&mut self) { - unsafe { (self.release)(self) } - } -} - -impl FFI_WeakFunctionRegistry { - /// Creates a new [`FFI_WeakFunctionRegistry`]. - pub fn new( - registry: Arc, - task_ctx_accessor: FFI_TaskContextAccessor, - ) -> Self { - let registry = Arc::downgrade(®istry); - let private_data = Box::new(RegistryPrivateData { registry }); - - Self { - udfs: udfs_fn_wrapper, - udafs: udafs_fn_wrapper, - udwfs: udwfs_fn_wrapper, - - udf: udf_fn_wrapper, - udaf: udaf_fn_wrapper, - udwf: udwf_fn_wrapper, - - task_ctx_accessor, - clone: clone_fn_wrapper, - release: release_fn_wrapper, - version: super::version, - private_data: Box::into_raw(private_data) as *mut c_void, - library_marker_id: crate::get_library_marker_id, - } - } -} - -/// This wrapper struct exists on the receiver side of the FFI interface, so it has -/// no guarantees about being able to access the data in `private_data`. Any functions -/// defined on this struct must only use the stable functions provided in -/// FFI_FunctionRegistry to interact with the foreign table provider. -#[derive(Debug)] -pub struct ForeignWeakFunctionRegistry(FFI_WeakFunctionRegistry); - -unsafe impl Send for ForeignWeakFunctionRegistry {} -unsafe impl Sync for ForeignWeakFunctionRegistry {} - -impl TryFrom<&FFI_WeakFunctionRegistry> for Arc { - type Error = DataFusionError; - fn try_from(value: &FFI_WeakFunctionRegistry) -> Result { - if (value.library_marker_id)() == crate::get_library_marker_id() { - return value.inner(); - } - - Ok(Arc::new(ForeignWeakFunctionRegistry(value.clone()))) - } -} - -impl Clone for FFI_WeakFunctionRegistry { - fn clone(&self) -> Self { - unsafe { (self.clone)(self) } - } -} - -impl FunctionRegistry for ForeignWeakFunctionRegistry { - fn udfs(&self) -> HashSet { - let udfs = unsafe { (self.0.udfs)(&self.0) }; - match udfs { - RResult::ROk(udfs) => udfs.into_iter().map(String::from).collect(), - RResult::RErr(err) => { - warn!("{err}"); - HashSet::with_capacity(0) - } - } - } - - fn udafs(&self) -> HashSet { - let udafs = unsafe { (self.0.udafs)(&self.0) }; - match udafs { - RResult::ROk(udafs) => udafs.into_iter().map(String::from).collect(), - RResult::RErr(err) => { - warn!("{err}"); - HashSet::with_capacity(0) - } - } - } - - fn udwfs(&self) -> HashSet { - let udwfs = unsafe { (self.0.udwfs)(&self.0) }; - match udwfs { - RResult::ROk(udwfs) => udwfs.into_iter().map(String::from).collect(), - RResult::RErr(err) => { - warn!("{err}"); - HashSet::with_capacity(0) - } - } - } - - fn udf(&self, name: &str) -> datafusion_common::Result> { - let udf = df_result!(unsafe { (self.0.udf)(&self.0, name.into()) })?; - - let udf = >::try_from(&udf)?; - Ok(Arc::new(ScalarUDF::new_from_shared_impl(udf))) - } - - fn udaf(&self, name: &str) -> datafusion_common::Result> { - let udaf = df_result!(unsafe { (self.0.udaf)(&self.0, name.into()) })?; - - let udaf = >::try_from(&udaf)?; - Ok(Arc::new(AggregateUDF::new_from_shared_impl(udaf))) - } - - fn udwf(&self, name: &str) -> datafusion_common::Result> { - let udwf = df_result!(unsafe { (self.0.udwf)(&self.0, name.into()) })?; - - let udwf = >::try_from(&udwf)?; - Ok(Arc::new(WindowUDF::new_from_shared_impl(udwf))) - } - - fn register_udf( - &mut self, - _udf: Arc, - ) -> datafusion_common::Result>> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } - - fn register_udaf( - &mut self, - _udaf: Arc, - ) -> datafusion_common::Result>> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } - - fn register_udwf( - &mut self, - _udwf: Arc, - ) -> datafusion_common::Result>> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } - - fn deregister_udf( - &mut self, - _name: &str, - ) -> datafusion_common::Result>> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } - - fn deregister_udaf( - &mut self, - _name: &str, - ) -> datafusion_common::Result>> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } - - fn deregister_udwf( - &mut self, - _name: &str, - ) -> datafusion_common::Result>> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } - - fn register_function_rewrite( - &mut self, - _rewrite: Arc, - ) -> datafusion_common::Result<()> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } - - fn expr_planners(&self) -> Vec> { - warn!("FFI Function Registry does not support expression planners."); - vec![] - } - - fn register_expr_planner( - &mut self, - _expr_planner: Arc, - ) -> datafusion_common::Result<()> { - not_impl_err!("Function Registry does not allow mutation via FFI") - } -} - -#[cfg(test)] -mod tests { - use super::*; - use datafusion::prelude::SessionContext; - use datafusion_execution::TaskContextAccessor; - use datafusion_expr::registry::FunctionRegistry; - - #[tokio::test] - async fn test_round_trip_ffi_function_registry() -> Result<(), DataFusionError> { - let ctx = Arc::new(SessionContext::new()); - let function_registry = - Arc::clone(&ctx) as Arc; - let task_ctx_accessor = Arc::clone(&ctx) as Arc; - - let mut ffi_registry = - FFI_WeakFunctionRegistry::new(function_registry, task_ctx_accessor.into()); - ffi_registry.library_marker_id = crate::mock_foreign_marker_id; - - let foreign_registry: Arc = - (&ffi_registry).try_into()?; - - let udf_names = foreign_registry.udfs(); - assert!(!udf_names.is_empty()); - let udf = foreign_registry.udf(udf_names.iter().next().unwrap())?; - - let udaf_names = foreign_registry.udafs(); - assert!(!udaf_names.is_empty()); - let udaf = foreign_registry.udaf(udaf_names.iter().next().unwrap())?; - - let udwf_names = foreign_registry.udwfs(); - assert!(!udwf_names.is_empty()); - let udwf = foreign_registry.udwf(udwf_names.iter().next().unwrap())?; - - // The following tests exist to ensure that if we do add support - // for mutable function registry in the future that we have - // added test coverage. - - // Manually create foreign registry so we can make it mutable - let mut foreign_registry = ForeignWeakFunctionRegistry(ffi_registry); - - fn expect_not_implemented(input: Result) { - let Err(DataFusionError::NotImplemented(_)) = input else { - panic!("Expected not implemented feature"); - }; - } - expect_not_implemented(foreign_registry.register_udf(udf)); - expect_not_implemented(foreign_registry.register_udaf(udaf)); - expect_not_implemented(foreign_registry.register_udwf(udwf)); - expect_not_implemented(foreign_registry.deregister_udf("a")); - expect_not_implemented(foreign_registry.deregister_udaf("a")); - expect_not_implemented(foreign_registry.deregister_udwf("a")); - - Ok(()) - } -} diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index dd7a3cfdf97c..24e5fa632377 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -27,7 +27,6 @@ pub mod arrow_wrappers; pub mod catalog_provider; pub mod execution_plan; -pub mod function_registry; pub mod insert_op; pub mod plan_properties; pub mod record_batch_stream; From 5f09b549eecb6c4cbb02baf9d5a5d61543c3ce12 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 15:19:29 -0500 Subject: [PATCH 52/69] Add unit tests to improve coverage --- datafusion/ffi/src/tests/mod.rs | 10 ++++ datafusion/ffi/src/tests/udf_udaf_udwf.rs | 18 +++++++ datafusion/ffi/tests/ffi_udwf.rs | 65 ++++++++++++++++++----- 3 files changed, 80 insertions(+), 13 deletions(-) diff --git a/datafusion/ffi/src/tests/mod.rs b/datafusion/ffi/src/tests/mod.rs index 4a002bb68e1d..979531d47e99 100644 --- a/datafusion/ffi/src/tests/mod.rs +++ b/datafusion/ffi/src/tests/mod.rs @@ -35,6 +35,7 @@ use crate::udwf::FFI_WindowUDF; use super::{table_provider::FFI_TableProvider, udf::FFI_ScalarUDF}; use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; +use crate::tests::udf_udaf_udwf::{create_ffi_cumedist_func, create_ffi_ntile_func}; use arrow::array::RecordBatch; use arrow::datatypes::{DataType, Field, Schema}; use async_provider::create_async_table_provider; @@ -82,8 +83,15 @@ pub struct ForeignLibraryModule { /// Create grouping UDAF using stddev pub create_stddev_udaf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_AggregateUDF, + /// Rank will test `evaluate` pub create_rank_udwf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_WindowUDF, + /// NTile will test `evaluate_all` + pub create_ntile_udwf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_WindowUDF, + + /// NTile will test `evaluate_all_with_rank` + pub create_cumedist_udwf: extern "C" fn(FFI_TaskContextAccessor) -> FFI_WindowUDF, + pub version: extern "C" fn() -> u64, } @@ -137,6 +145,8 @@ pub fn get_foreign_library_module() -> ForeignLibraryModuleRef { create_sum_udaf: create_ffi_sum_func, create_stddev_udaf: create_ffi_stddev_func, create_rank_udwf: create_ffi_rank_func, + create_ntile_udwf: create_ffi_ntile_func, + create_cumedist_udwf: create_ffi_cumedist_func, version: super::version, } .leak_into_prefix() diff --git a/datafusion/ffi/src/tests/udf_udaf_udwf.rs b/datafusion/ffi/src/tests/udf_udaf_udwf.rs index f6a6f09c305c..6ff632efacf7 100644 --- a/datafusion/ffi/src/tests/udf_udaf_udwf.rs +++ b/datafusion/ffi/src/tests/udf_udaf_udwf.rs @@ -28,6 +28,8 @@ use datafusion_functions::math::random::RandomFunc; use datafusion_functions_aggregate::stddev::Stddev; use datafusion_functions_aggregate::sum::Sum; use datafusion_functions_table::generate_series::RangeFunc; +use datafusion_functions_window::cume_dist::CumeDist; +use datafusion_functions_window::ntile::Ntile; use datafusion_functions_window::rank::Rank; use std::sync::Arc; @@ -80,3 +82,19 @@ pub(crate) extern "C" fn create_ffi_rank_func( FFI_WindowUDF::new(udwf, task_ctx_accessor) } + +pub(crate) extern "C" fn create_ffi_ntile_func( + task_ctx_accessor: FFI_TaskContextAccessor, +) -> FFI_WindowUDF { + let udwf: Arc = Arc::new(Ntile::new().into()); + + FFI_WindowUDF::new(udwf, task_ctx_accessor) +} + +pub(crate) extern "C" fn create_ffi_cumedist_func( + task_ctx_accessor: FFI_TaskContextAccessor, +) -> FFI_WindowUDF { + let udwf: Arc = Arc::new(CumeDist::new().into()); + + FFI_WindowUDF::new(udwf, task_ctx_accessor) +} diff --git a/datafusion/ffi/tests/ffi_udwf.rs b/datafusion/ffi/tests/ffi_udwf.rs index 073d8afa5abb..aeb617cf9ca7 100644 --- a/datafusion/ffi/tests/ffi_udwf.rs +++ b/datafusion/ffi/tests/ffi_udwf.rs @@ -25,24 +25,22 @@ mod tests { use datafusion::logical_expr::{col, ExprFunctionExt, WindowUDF}; use datafusion::prelude::SessionContext; use datafusion_execution::TaskContextAccessor; - use datafusion_expr::WindowUDFImpl; + use datafusion_expr::{lit, Expr, WindowUDFImpl}; + use datafusion_ffi::session::task_ctx_accessor::FFI_TaskContextAccessor; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; + use datafusion_ffi::udwf::FFI_WindowUDF; use std::sync::Arc; - #[tokio::test] - async fn test_rank_udwf() -> Result<()> { - let module = get_module()?; + async fn test_window_function( + function: extern "C" fn(FFI_TaskContextAccessor) -> FFI_WindowUDF, + arguments: Vec, + expected: ArrayRef, + ) -> Result<()> { let ctx = Arc::new(SessionContext::default()); let task_ctx_accessor = Arc::clone(&ctx) as Arc; - let ffi_rank_func = - module - .create_rank_udwf() - .ok_or(DataFusionError::NotImplemented( - "External table provider failed to implement create_scalar_udf" - .to_string(), - ))?(task_ctx_accessor.into()); + let ffi_rank_func = function(task_ctx_accessor.into()); let foreign_rank_func: Arc = (&ffi_rank_func).try_into()?; let udwf = WindowUDF::new_from_shared_impl(foreign_rank_func); @@ -51,7 +49,7 @@ mod tests { let df = df.select(vec![ col("a"), - udwf.call(vec![]) + udwf.call(arguments) .order_by(vec![Sort::new(col("a"), true, true)]) .build() .unwrap() @@ -61,11 +59,52 @@ mod tests { df.clone().show().await?; let result = df.collect().await?; - let expected = create_array!(UInt64, [1, 2, 3, 4, 5]) as ArrayRef; assert_eq!(result.len(), 1); assert_eq!(result[0].column(1), &expected); Ok(()) } + + #[tokio::test] + async fn test_rank_udwf() -> Result<()> { + let module = get_module()?; + let expected = create_array!(UInt64, [1, 2, 3, 4, 5]) as ArrayRef; + let function = + module + .create_rank_udwf() + .ok_or(DataFusionError::NotImplemented( + "External table provider failed to implement window function" + .to_string(), + ))?; + test_window_function(function, vec![], expected).await + } + + #[tokio::test] + async fn test_ntile_udwf() -> Result<()> { + let module = get_module()?; + let expected = create_array!(UInt64, [1, 1, 2, 2, 3]) as ArrayRef; + let function = + module + .create_ntile_udwf() + .ok_or(DataFusionError::NotImplemented( + "External table provider failed to implement window function" + .to_string(), + ))?; + test_window_function(function, vec![lit(3)], expected).await + } + + #[tokio::test] + async fn test_cumedist_udwf() -> Result<()> { + let module = get_module()?; + let expected = create_array!(Float64, [0.2, 0.4, 0.6, 0.8, 1.0]) as ArrayRef; + let function = + module + .create_cumedist_udwf() + .ok_or(DataFusionError::NotImplemented( + "External table provider failed to implement window function" + .to_string(), + ))?; + test_window_function(function, vec![], expected).await + } } From dfbae2cd1bd3d18ae9053332a1ccf876428a8da6 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 16:03:23 -0500 Subject: [PATCH 53/69] Catch an error in unsupported field types --- datafusion/ffi/src/arrow_wrappers.rs | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/datafusion/ffi/src/arrow_wrappers.rs b/datafusion/ffi/src/arrow_wrappers.rs index 7b3751dcae82..a00e2be75377 100644 --- a/datafusion/ffi/src/arrow_wrappers.rs +++ b/datafusion/ffi/src/arrow_wrappers.rs @@ -38,7 +38,7 @@ impl From for WrappedSchema { Ok(s) => s, Err(e) => { error!("Unable to convert DataFusion Schema to FFI_ArrowSchema in FFI_PlanProperties. {e}"); - FFI_ArrowSchema::empty() + FFI_ArrowSchema::try_from(Schema::empty()).unwrap() } }; @@ -94,3 +94,29 @@ impl TryFrom<&ArrayRef> for WrappedArray { Ok(WrappedArray { array, schema }) } } + +#[cfg(test)] +mod tests { + use crate::arrow_wrappers::WrappedSchema; + use arrow_schema::{ArrowError, DataType, Field, Schema, SchemaRef}; + use std::sync::Arc; + + /// Test an unsupported field type. This is necessary only so we can get good unit test coverage + /// so that we can also verify memory is properly maintained since we are doing `unsafe` operations. + #[test] + fn test_unsupported_schema() -> Result<(), ArrowError> { + let field = Arc::new(Field::new("a", DataType::Int32, false)); + let schema = Arc::new(Schema::new(vec![Field::new( + "a", + DataType::ListView(field), + false, + )])); + + let wrapped_schema = WrappedSchema::from(schema); + + let schema: SchemaRef = wrapped_schema.into(); + assert!(schema.fields().is_empty()); + + Ok(()) + } +} From 9d3e7682493a9426536df501d6de9d9810cdc180 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 16:17:00 -0500 Subject: [PATCH 54/69] Add coverage for clone --- datafusion/ffi/src/session/config.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/ffi/src/session/config.rs b/datafusion/ffi/src/session/config.rs index afef05ce19ed..e22b09cc8fad 100644 --- a/datafusion/ffi/src/session/config.rs +++ b/datafusion/ffi/src/session/config.rs @@ -168,6 +168,7 @@ mod tests { let original_options = session_config.options().entries(); let mut ffi_config: FFI_SessionConfig = (&session_config).into(); + let _ = ffi_config.clone(); ffi_config.library_marker_id = crate::mock_foreign_marker_id; let foreign_config: SessionConfig = (&ffi_config).try_into()?; From d06f9a244f589b6e757e74b36a1e0ccb17f9e862 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Mon, 10 Nov 2025 22:15:56 -0500 Subject: [PATCH 55/69] Update unit test coverage --- datafusion/ffi/src/session/mod.rs | 61 +++++++++++++++++++++++++++++++ datafusion/ffi/src/udf/mod.rs | 35 ++---------------- datafusion/ffi/tests/ffi_udf.rs | 28 +++++++++----- 3 files changed, 82 insertions(+), 42 deletions(-) diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index c5a45fffca15..62e7b1d50f75 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -567,3 +567,64 @@ impl Session for ForeignSession { unsafe { Arc::new((self.session.task_ctx)(&self.session).into()) } } } + +#[cfg(test)] +mod tests { + use super::*; + use arrow_schema::{DataType, Field, Schema}; + use datafusion::prelude::SessionContext; + use datafusion_common::DataFusionError; + use datafusion_execution::TaskContextAccessor; + use datafusion_expr::col; + use datafusion_expr::registry::FunctionRegistry; + use std::sync::Arc; + + #[tokio::test] + async fn test_ffi_session() -> Result<(), DataFusionError> { + let ctx = Arc::new(SessionContext::new()); + let task_ctx_accessor = Arc::clone(&ctx) as Arc; + let state = ctx.state(); + + let local_session = FFI_Session::new(&state, task_ctx_accessor.into(), None); + let foreign_session = ForeignSession::try_from(&local_session)?; + + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let df_schema = schema.try_into()?; + let physical_expr = foreign_session.create_physical_expr(col("a"), &df_schema)?; + assert_eq!( + format!("{physical_expr:?}"), + "Column { name: \"a\", index: 0 }" + ); + + assert_eq!(foreign_session.session_id(), state.session_id()); + + let logical_plan = LogicalPlan::default(); + let physical_plan = foreign_session.create_physical_plan(&logical_plan).await?; + assert_eq!(format!("{physical_plan:?}"), "EmptyExec { schema: Schema { fields: [], metadata: {} }, partitions: 1, cache: PlanProperties { eq_properties: EquivalenceProperties { eq_group: EquivalenceGroup { map: {}, classes: [] }, oeq_class: OrderingEquivalenceClass { orderings: [] }, oeq_cache: OrderingEquivalenceCache { normal_cls: OrderingEquivalenceClass { orderings: [] }, leading_map: {} }, constraints: Constraints { inner: [] }, schema: Schema { fields: [], metadata: {} } }, partitioning: UnknownPartitioning(1), emission_type: Incremental, boundedness: Bounded, evaluation_type: Lazy, scheduling_type: Cooperative, output_ordering: None } }"); + + assert_eq!( + format!("{:?}", foreign_session.default_table_options()), + format!("{:?}", state.default_table_options()) + ); + + assert_eq!( + format!("{:?}", foreign_session.table_options()), + format!("{:?}", state.table_options()) + ); + + let local_udfs = state.udfs(); + for udf in foreign_session.scalar_functions().keys() { + assert!(local_udfs.contains(udf)); + } + let local_udafs = state.udafs(); + for udaf in foreign_session.aggregate_functions().keys() { + assert!(local_udafs.contains(udaf)); + } + let local_udwfs = state.udwfs(); + for udwf in foreign_session.window_functions().keys() { + assert!(local_udwfs.contains(udwf)); + } + + Ok(()) + } +} diff --git a/datafusion/ffi/src/udf/mod.rs b/datafusion/ffi/src/udf/mod.rs index 6e06ea07fc57..ffd2ca0d8a56 100644 --- a/datafusion/ffi/src/udf/mod.rs +++ b/datafusion/ffi/src/udf/mod.rs @@ -34,6 +34,7 @@ use arrow::{ use arrow_schema::FieldRef; use datafusion_common::config::ConfigOptions; use datafusion_common::error::{DataFusionError, Result}; +use datafusion_common::not_impl_err; use datafusion_expr::{ type_coercion::functions::data_types_with_scalar_udf, ColumnarValue, ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, @@ -60,13 +61,6 @@ pub struct FFI_ScalarUDF { /// FFI equivalent to the `volatility` of a [`ScalarUDF`] pub volatility: FFI_Volatility, - /// Determines the return type of the underlying [`ScalarUDF`] based on the - /// argument types. - pub return_type: unsafe extern "C" fn( - udf: &Self, - arg_types: RVec, - ) -> RResult, - /// Determines the return info of the underlying [`ScalarUDF`]. Either this /// or return_type may be implemented on a UDF. pub return_field_from_args: unsafe extern "C" fn( @@ -128,22 +122,6 @@ impl FFI_ScalarUDF { } } -unsafe extern "C" fn return_type_fn_wrapper( - udf: &FFI_ScalarUDF, - arg_types: RVec, -) -> RResult { - let udf = udf.inner(); - - let arg_types = rresult_return!(rvec_wrapped_to_vec_datatype(&arg_types)); - - let return_type = udf - .return_type(&arg_types) - .and_then(|v| FFI_ArrowSchema::try_from(v).map_err(DataFusionError::from)) - .map(WrappedSchema); - - rresult!(return_type) -} - unsafe extern "C" fn return_field_from_args_fn_wrapper( udf: &FFI_ScalarUDF, args: FFI_ReturnFieldArgs, @@ -263,7 +241,6 @@ impl From> for FFI_ScalarUDF { volatility, short_circuits, invoke_with_args: invoke_with_args_fn_wrapper, - return_type: return_type_fn_wrapper, return_field_from_args: return_field_from_args_fn_wrapper, coerce_types: coerce_types_fn_wrapper, clone: clone_fn_wrapper, @@ -363,14 +340,8 @@ impl ScalarUDFImpl for ForeignScalarUDF { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { - let arg_types = vec_datatype_to_rvec_wrapped(arg_types)?; - - let result = unsafe { (self.udf.return_type)(&self.udf, arg_types) }; - - let result = df_result!(result); - - result.and_then(|r| (&r.0).try_into().map_err(DataFusionError::from)) + fn return_type(&self, _arg_types: &[DataType]) -> Result { + not_impl_err!("return_type is not implemented since return_field_from_args is.") } fn return_field_from_args(&self, args: ReturnFieldArgs) -> Result { diff --git a/datafusion/ffi/tests/ffi_udf.rs b/datafusion/ffi/tests/ffi_udf.rs index 0bf770cd9b93..399836928799 100644 --- a/datafusion/ffi/tests/ffi_udf.rs +++ b/datafusion/ffi/tests/ffi_udf.rs @@ -20,11 +20,10 @@ #[cfg(feature = "integration-tests")] mod tests { use arrow::datatypes::DataType; - use datafusion::common::record_batch; use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::ScalarUDF; use datafusion::prelude::{col, SessionContext}; - use datafusion_expr::ScalarUDFImpl; + use datafusion_expr::{lit, ScalarUDFImpl}; use datafusion_ffi::tests::create_record_batch; use datafusion_ffi::tests::utils::get_module; use std::sync::Arc; @@ -52,19 +51,28 @@ mod tests { let df = df .with_column("abs_a", udf.call(vec![col("a")]))? - .with_column("abs_b", udf.call(vec![col("b")]))?; + .with_column("abs_b", udf.call(vec![col("b")]))? + .with_column("abs_lit", udf.call(vec![lit(-1)]))?; let result = df.collect().await?; + assert!(result.len() == 1); - let expected = record_batch!( - ("a", Int32, vec![-5, -4, -3, -2, -1]), - ("b", Float64, vec![-5., -4., -3., -2., -1.]), - ("abs_a", Int32, vec![5, 4, 3, 2, 1]), - ("abs_b", Float64, vec![5., 4., 3., 2., 1.]) + let expected = arrow::array::record_batch!( + ("a", Int32, [-5, -4, -3, -2, -1]), + ("b", Float64, [-5., -4., -3., -2., -1.]), + ("abs_a", Int32, [5, 4, 3, 2, 1]), + ("abs_b", Float64, [5., 4., 3., 2., 1.]), + ("abs_lit", Int32, [1, 1, 1, 1, 1]) )?; - assert!(result.len() == 1); - assert!(result[0] == expected); + // Literal value will create a non-null schema, so project before comparison + let result = result + .into_iter() + .next() + .unwrap() + .with_schema(expected.schema())?; + + assert!(result == expected); Ok(()) } From 1711843679befdb0101d38098f3cdb3c739f13ab Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 07:01:39 -0500 Subject: [PATCH 56/69] Correct regression on table type when calling into_view --- datafusion/core/src/dataframe/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 98804e424b40..b43c9c671f7d 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1655,7 +1655,7 @@ impl DataFrame { pub fn into_view(self) -> Arc { Arc::new(DataFrameTableProvider { plan: self.plan, - table_type: TableType::Temporary, + table_type: TableType::View, }) } From 00b0693e7a1d7ca998ca862816fc57bc441157fe Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 07:08:35 -0500 Subject: [PATCH 57/69] Add test coverage for view type --- datafusion/core/tests/dataframe/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 05f5a204c096..e221edb7c54b 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -1627,7 +1627,9 @@ async fn register_table() -> Result<()> { let df_impl = DataFrame::new(ctx.state(), df.logical_plan().clone()); // register a dataframe as a table - ctx.register_table("test_table", df_impl.clone().into_view())?; + let table_provider = df_impl.clone().into_view(); + assert_eq!(table_provider.table_type(), TableType::View); + ctx.register_table("test_table", table_provider)?; // pull the table out let table = ctx.table("test_table").await?; From e1215f296b2cbfd804597512cda1705bdbc0f8dc Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 07:22:08 -0500 Subject: [PATCH 58/69] Cleanup use statements a little --- datafusion/ffi/src/session/mod.rs | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 62e7b1d50f75..24f2ee468ca7 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -19,7 +19,6 @@ use crate::arrow_wrappers::WrappedSchema; use crate::execution_plan::FFI_ExecutionPlan; use crate::session::config::FFI_SessionConfig; use crate::session::task_context::FFI_TaskContext; -use crate::session::task_ctx_accessor::FFI_TaskContextAccessor; use crate::udaf::FFI_AggregateUDF; use crate::udf::FFI_ScalarUDF; use crate::udwf::FFI_WindowUDF; @@ -47,12 +46,13 @@ use datafusion_expr::{ use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::ExecutionPlan; use datafusion_proto::bytes::{logical_plan_from_bytes, logical_plan_to_bytes}; -use datafusion_proto::logical_plan::from_proto::parse_expr; -use datafusion_proto::logical_plan::to_proto::serialize_expr; -use datafusion_proto::logical_plan::DefaultLogicalExtensionCodec; -use datafusion_proto::physical_plan::from_proto::parse_physical_expr; -use datafusion_proto::physical_plan::to_proto::serialize_physical_expr; -use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; +use datafusion_proto::logical_plan::{ + from_proto::parse_expr, to_proto::serialize_expr, DefaultLogicalExtensionCodec, +}; +use datafusion_proto::physical_plan::{ + from_proto::parse_physical_expr, to_proto::serialize_physical_expr, + DefaultPhysicalExtensionCodec, +}; use datafusion_proto::protobuf::{LogicalExprNode, PhysicalExprNode}; use prost::Message; use std::any::Any; @@ -63,6 +63,7 @@ use tokio::runtime::Handle; pub mod config; pub mod task_context; pub mod task_ctx_accessor; +pub use task_ctx_accessor::FFI_TaskContextAccessor; /// A stable struct for sharing [`Session`] across FFI boundaries. #[repr(C)] From 3eed2e6a0eaabb00123d15faa4d03ffaaf390e37 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 08:39:35 -0500 Subject: [PATCH 59/69] Add check for invalid columns should return an error --- datafusion/core/src/dataframe/mod.rs | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index b43c9c671f7d..2790c2c74f1a 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -308,13 +308,14 @@ impl DataFrame { /// # } /// ``` pub fn select_columns(self, columns: &[&str]) -> Result { + let schema = self.plan.schema(); + for name in columns { + // validate at least one column will be returned + let _ = schema.qualified_field_with_unqualified_name(name)?; + } let fields = columns .iter() - .flat_map(|name| { - self.plan - .schema() - .qualified_fields_with_unqualified_name(name) - }) + .flat_map(|name| schema.qualified_fields_with_unqualified_name(name)) .collect::>(); let expr: Vec = fields .into_iter() @@ -437,13 +438,14 @@ impl DataFrame { /// # } /// ``` pub fn drop_columns(self, columns: &[&str]) -> Result { + let schema = self.plan.schema(); + for name in columns { + // validate at least one column will be dropped + let _ = schema.qualified_field_with_unqualified_name(name)?; + } let fields_to_drop = columns .iter() - .flat_map(|name| { - self.plan - .schema() - .qualified_fields_with_unqualified_name(name) - }) + .flat_map(|name| schema.qualified_fields_with_unqualified_name(name)) .collect::>(); let expr: Vec = self .plan From b518ecc5328d1c865d887a02a5d9e2a2ea6ee60f Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 08:39:51 -0500 Subject: [PATCH 60/69] Add unit test coverage for invalid column selection and drop --- datafusion/core/tests/dataframe/mod.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index e221edb7c54b..2d2cab3254e3 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -293,7 +293,7 @@ async fn select_columns() -> Result<()> { // build plan using Table API let t = test_table().await?; - let t2 = t.select_columns(&["c1", "c2", "c11"])?; + let t2 = t.clone().select_columns(&["c1", "c2", "c11"])?; let plan = t2.logical_plan().clone(); // build query using SQL @@ -302,6 +302,8 @@ async fn select_columns() -> Result<()> { // the two plans should be identical assert_same_plan(&plan, &sql_plan); + assert!(t.select_columns(&["non_existent_column"]).is_err()); + Ok(()) } @@ -458,7 +460,7 @@ async fn select_columns_duplicated_names_from_different_qualifiers() -> Result<( async fn drop_columns() -> Result<()> { // build plan using Table API let t = test_table().await?; - let t2 = t.drop_columns(&["c2", "c11"])?; + let t2 = t.clone().drop_columns(&["c2", "c11"])?; let plan = t2.logical_plan().clone(); // build query using SQL @@ -469,6 +471,8 @@ async fn drop_columns() -> Result<()> { // the two plans should be identical assert_same_plan(&plan, &sql_plan); + assert!(t.drop_columns(&["non_existent_column"]).is_err()); + Ok(()) } From 3df708698316c4821537c4f32f7b44e18b732764 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 09:54:04 -0500 Subject: [PATCH 61/69] drop_columns does accept non-existent columns. correct select_columns which would error on duplicates --- datafusion/core/src/dataframe/mod.rs | 35 +++++++++++++++----------- datafusion/core/tests/dataframe/mod.rs | 21 ++++++++++------ 2 files changed, 35 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2790c2c74f1a..aa378d42622d 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -52,8 +52,8 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::{CsvOptions, JsonOptions}; use datafusion_common::{ exec_err, internal_datafusion_err, not_impl_err, plan_datafusion_err, plan_err, - Column, DFSchema, DataFusionError, ParamValues, ScalarValue, SchemaError, - TableReference, UnnestOptions, + unqualified_field_not_found, Column, DFSchema, DataFusionError, ParamValues, + ScalarValue, SchemaError, TableReference, UnnestOptions, }; use datafusion_expr::select_expr::SelectExpr; use datafusion_expr::{ @@ -308,14 +308,22 @@ impl DataFrame { /// # } /// ``` pub fn select_columns(self, columns: &[&str]) -> Result { - let schema = self.plan.schema(); - for name in columns { - // validate at least one column will be returned - let _ = schema.qualified_field_with_unqualified_name(name)?; - } let fields = columns .iter() - .flat_map(|name| schema.qualified_fields_with_unqualified_name(name)) + .map(|name| { + let fields = self + .plan + .schema() + .qualified_fields_with_unqualified_name(name); + if fields.is_empty() { + Err(unqualified_field_not_found(name, self.plan.schema())) + } else { + Ok(fields) + } + }) + .collect::, _>>()? + .into_iter() + .flatten() .collect::>(); let expr: Vec = fields .into_iter() @@ -438,14 +446,13 @@ impl DataFrame { /// # } /// ``` pub fn drop_columns(self, columns: &[&str]) -> Result { - let schema = self.plan.schema(); - for name in columns { - // validate at least one column will be dropped - let _ = schema.qualified_field_with_unqualified_name(name)?; - } let fields_to_drop = columns .iter() - .flat_map(|name| schema.qualified_fields_with_unqualified_name(name)) + .flat_map(|name| { + self.plan + .schema() + .qualified_fields_with_unqualified_name(name) + }) .collect::>(); let expr: Vec = self .plan diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 2d2cab3254e3..610d6937deb7 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -66,8 +66,8 @@ use datafusion::test_util::{ use datafusion_catalog::TableProvider; use datafusion_common::test_util::{batches_to_sort_string, batches_to_string}; use datafusion_common::{ - assert_contains, internal_datafusion_err, Constraint, Constraints, DFSchema, - DataFusionError, ScalarValue, TableReference, UnnestOptions, + assert_contains, internal_datafusion_err, internal_err, Constraint, Constraints, + DFSchema, DataFusionError, ScalarValue, TableReference, UnnestOptions, }; use datafusion_common_runtime::SpawnedTask; use datafusion_datasource::file_format::format_as_file_type; @@ -293,7 +293,7 @@ async fn select_columns() -> Result<()> { // build plan using Table API let t = test_table().await?; - let t2 = t.clone().select_columns(&["c1", "c2", "c11"])?; + let t2 = t.select_columns(&["c1", "c2", "c11"])?; let plan = t2.logical_plan().clone(); // build query using SQL @@ -302,7 +302,16 @@ async fn select_columns() -> Result<()> { // the two plans should be identical assert_same_plan(&plan, &sql_plan); - assert!(t.select_columns(&["non_existent_column"]).is_err()); + Ok(()) +} + +#[tokio::test] +async fn select_columns_with_nonexistent_columns() -> Result<()> { + let t = test_table().await?; + let t2 = t.select_columns(&["canada", "c2", "rocks"]); + let Err(DataFusionError::SchemaError(_, _)) = t2 else { + return internal_err!("select_columns with nonexistent columns should error"); + }; Ok(()) } @@ -460,7 +469,7 @@ async fn select_columns_duplicated_names_from_different_qualifiers() -> Result<( async fn drop_columns() -> Result<()> { // build plan using Table API let t = test_table().await?; - let t2 = t.clone().drop_columns(&["c2", "c11"])?; + let t2 = t.drop_columns(&["c2", "c11"])?; let plan = t2.logical_plan().clone(); // build query using SQL @@ -471,8 +480,6 @@ async fn drop_columns() -> Result<()> { // the two plans should be identical assert_same_plan(&plan, &sql_plan); - assert!(t.drop_columns(&["non_existent_column"]).is_err()); - Ok(()) } From f7d46c90b4388edbb7f90998e5db3d1dadcd2dae Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 14:01:17 -0500 Subject: [PATCH 62/69] Add readme information about the usage of the library_marker_id --- datafusion/ffi/README.md | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/datafusion/ffi/README.md b/datafusion/ffi/README.md index ea30e9eeb0f3..68a1bb395e81 100644 --- a/datafusion/ffi/README.md +++ b/datafusion/ffi/README.md @@ -101,6 +101,36 @@ In this crate we have a variety of structs which closely mimic the behavior of their internal counterparts. To see detailed notes about how to use them, see the example in `FFI_TableProvider`. +## Library Marker ID + +When reviewing the code, many of the structs in this crate contain a call to +a `library_maker_id`. The purpose of this call is to determine if a library is +accessing *local* code through the FFI structs. Consider this example: you have +a `primary` program that exposes functions to create a schema provider. You +have a `secondary` library that exposes a function to create a catalog provider +and the `secondary` library uses the schema provider of the `primary` program. +From the point of view of the `secondary` library, the schema provider is +foreign code. + +Now when we register the `secondary` library with the `primary` program as a +catalog provider and we make calls to get a schema, the `secondary` library +will return a FFI wrapped schema provider back to the `primary` program. In +this case that schema provider is actually local code to the `primary` program +except that it is wrapped in the FFI code! + +We work around this by the `library_marker_id` calls. What this does is it +creates a global variable within each library and returns a `u64` address +of that library. This is guaranteed to be unique for every library that contains +FFI code. By comparing these `u64` addresses we can determine if a FFI struct +is local or foreign. + +In our example of the schema provider, if you were to make a call in your +primary program to get the schema provider, it would reach out to the foreign +catalog provider and send back a `FFI_SchemaProvider` object. By then +comparing the `library_marker_id` of this object to the `primary` program, we +determine it is local code. This means it is safe to access the underlying +private data. + ## Testing Coverage Since this library contains a large amount of `unsafe` code, it is important From 6d3f279241ac65723ec3b2cdd33d225b6e3670e0 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 14:41:36 -0500 Subject: [PATCH 63/69] Add documentation about task context accessor --- datafusion/ffi/README.md | 23 +++++++++++++++++++ datafusion/ffi/src/catalog_provider.rs | 2 ++ datafusion/ffi/src/execution_plan.rs | 2 ++ datafusion/ffi/src/plan_properties.rs | 2 ++ datafusion/ffi/src/schema_provider.rs | 2 ++ datafusion/ffi/src/session/mod.rs | 2 ++ datafusion/ffi/src/session/task_context.rs | 2 ++ .../ffi/src/session/task_ctx_accessor.rs | 5 ++++ datafusion/ffi/src/table_provider.rs | 2 ++ datafusion/ffi/src/udaf/accumulator_args.rs | 3 +++ datafusion/ffi/src/udaf/mod.rs | 2 ++ datafusion/ffi/src/udtf.rs | 2 ++ datafusion/ffi/src/udwf/mod.rs | 2 ++ .../ffi/src/udwf/partition_evaluator_args.rs | 3 +++ 14 files changed, 54 insertions(+) diff --git a/datafusion/ffi/README.md b/datafusion/ffi/README.md index 68a1bb395e81..bd9b6f1f2f3f 100644 --- a/datafusion/ffi/README.md +++ b/datafusion/ffi/README.md @@ -101,6 +101,29 @@ In this crate we have a variety of structs which closely mimic the behavior of their internal counterparts. To see detailed notes about how to use them, see the example in `FFI_TableProvider`. +## Task Context Accessor + +Many of the FFI structs in this crate contain a `FFI_TaskContextAccessor`. The +purpose of this struct is to *weakly* hold a reference to a method to +access the current `TaskContext`. The reason we need this accessor is because +we use the `datafusion-proto` crate to serialize and deserialize data across +the FFI boundary. In particular, we need to serialize and deserialize +functions using a `TaskContext`. + +This becomes difficult because we may need to register multiple user defined +functions, table or catalog providers, etc with a `Session`, and each of these +will need the `TaskContext` to perform the processing. For this reason we +cannot simply include the `TaskContext` at the time of registration because +it would not have knowledge of anything registered afterward. + +The `FFI_TaskContextAccessor` is built up from a trait that provides a method +to get the current `TaskContext`. It only holds a `Weak` reference to the +`TaskContextAccessor`, because otherwise we could create a circular dependency +at runtime. It is imperative that if you use these methods that your accessor +remains valid for the lifetime of the calls. The `TaskContextAccessor` is +implemented on `SessionContext` and it is easy to implement on an struct that +implements `Session`. + ## Library Marker ID When reviewing the code, many of the structs in this crate contain a call to diff --git a/datafusion/ffi/src/catalog_provider.rs b/datafusion/ffi/src/catalog_provider.rs index d3db8d43fe7d..0079d296f050 100644 --- a/datafusion/ffi/src/catalog_provider.rs +++ b/datafusion/ffi/src/catalog_provider.rs @@ -58,6 +58,8 @@ pub struct FFI_CatalogProvider { cascade: bool, ) -> RResult, RString>, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should diff --git a/datafusion/ffi/src/execution_plan.rs b/datafusion/ffi/src/execution_plan.rs index 65f20751354d..739ee9266c02 100644 --- a/datafusion/ffi/src/execution_plan.rs +++ b/datafusion/ffi/src/execution_plan.rs @@ -53,6 +53,8 @@ pub struct FFI_ExecutionPlan { partition: usize, ) -> RResult, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index 05c9d4c02c0a..52436dea35bf 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -62,6 +62,8 @@ pub struct FFI_PlanProperties { /// Return the schema of the plan. pub schema: unsafe extern "C" fn(plan: &Self) -> WrappedSchema, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. pub task_ctx_accessor: FFI_TaskContextAccessor, /// Release the memory of the private data when it is no longer being used. diff --git a/datafusion/ffi/src/schema_provider.rs b/datafusion/ffi/src/schema_provider.rs index b615aaf7925d..d05acd700f10 100644 --- a/datafusion/ffi/src/schema_provider.rs +++ b/datafusion/ffi/src/schema_provider.rs @@ -64,6 +64,8 @@ pub struct FFI_SchemaProvider { pub table_exist: unsafe extern "C" fn(provider: &Self, name: RString) -> bool, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index 24f2ee468ca7..84b876707847 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -103,6 +103,8 @@ pub struct FFI_Session { pub task_ctx: unsafe extern "C" fn(&Self) -> FFI_TaskContext, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the registry. This should diff --git a/datafusion/ffi/src/session/task_context.rs b/datafusion/ffi/src/session/task_context.rs index 057f8d98c072..95ae984eb6dd 100644 --- a/datafusion/ffi/src/session/task_context.rs +++ b/datafusion/ffi/src/session/task_context.rs @@ -49,6 +49,8 @@ pub struct FFI_TaskContext { pub window_functions: unsafe extern "C" fn(&Self) -> RHashMap, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. pub task_ctx_accessor: FFI_TaskContextAccessor, /// Release the memory of the private data when it is no longer being used. diff --git a/datafusion/ffi/src/session/task_ctx_accessor.rs b/datafusion/ffi/src/session/task_ctx_accessor.rs index 610fa3e053b6..72525b30a748 100644 --- a/datafusion/ffi/src/session/task_ctx_accessor.rs +++ b/datafusion/ffi/src/session/task_ctx_accessor.rs @@ -24,6 +24,11 @@ use datafusion_execution::{TaskContext, TaskContextAccessor}; use std::sync::Weak; use std::{ffi::c_void, sync::Arc}; +/// Struct for accessing the [`TaskContext`]. This method contains a weak +/// reference, so there are no guarantees that the [`TaskContext`] remains +/// valid. This is used primarily for protobuf encoding and decoding of +/// data passed across the FFI boundary. See the crate README for +/// additional information. #[repr(C)] #[derive(Debug, StableAbi)] #[allow(non_camel_case_types)] diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index d919652645c5..894421b98091 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -138,6 +138,8 @@ pub struct FFI_TableProvider { insert_op: FFI_InsertOp, ) -> FfiFuture>, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the execution plan. This should diff --git a/datafusion/ffi/src/udaf/accumulator_args.rs b/datafusion/ffi/src/udaf/accumulator_args.rs index 5c6caf613c19..ce406fb6ea67 100644 --- a/datafusion/ffi/src/udaf/accumulator_args.rs +++ b/datafusion/ffi/src/udaf/accumulator_args.rs @@ -54,6 +54,9 @@ pub struct FFI_AccumulatorArgs { is_reversed: bool, name: RString, physical_expr_def: RVec, + + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. task_ctx_accessor: FFI_TaskContextAccessor, } diff --git a/datafusion/ffi/src/udaf/mod.rs b/datafusion/ffi/src/udaf/mod.rs index 59e4d928aac2..7b7bb6ac9871 100644 --- a/datafusion/ffi/src/udaf/mod.rs +++ b/datafusion/ffi/src/udaf/mod.rs @@ -133,6 +133,8 @@ pub struct FFI_AggregateUDF { arg_types: RVec, ) -> RResult, RString>, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the udaf. This should diff --git a/datafusion/ffi/src/udtf.rs b/datafusion/ffi/src/udtf.rs index bf23010a07ce..1b574cc7aaa9 100644 --- a/datafusion/ffi/src/udtf.rs +++ b/datafusion/ffi/src/udtf.rs @@ -49,6 +49,8 @@ pub struct FFI_TableFunction { args: RVec, ) -> RResult, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. pub task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the udtf. This should diff --git a/datafusion/ffi/src/udwf/mod.rs b/datafusion/ffi/src/udwf/mod.rs index 0fd5ddf2e3aa..110ba225f28a 100644 --- a/datafusion/ffi/src/udwf/mod.rs +++ b/datafusion/ffi/src/udwf/mod.rs @@ -90,6 +90,8 @@ pub struct FFI_WindowUDF { pub sort_options: ROption, + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. task_ctx_accessor: FFI_TaskContextAccessor, /// Used to create a clone on the provider of the udf. This should diff --git a/datafusion/ffi/src/udwf/partition_evaluator_args.rs b/datafusion/ffi/src/udwf/partition_evaluator_args.rs index 46319668f233..046ad1bbf3d5 100644 --- a/datafusion/ffi/src/udwf/partition_evaluator_args.rs +++ b/datafusion/ffi/src/udwf/partition_evaluator_args.rs @@ -49,6 +49,9 @@ pub struct FFI_PartitionEvaluatorArgs { is_reversed: bool, ignore_nulls: bool, schema: WrappedSchema, + + /// Accessor for TaskContext to be used during protobuf serialization + /// and deserialization. task_ctx_accessor: FFI_TaskContextAccessor, } From 65025eae3c2139cfc126a2ec1e97cb7a7f433c0d Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 15:42:10 -0500 Subject: [PATCH 64/69] Remove unused dependencies --- Cargo.lock | 2 -- datafusion/ffi/Cargo.toml | 2 -- 2 files changed, 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c0c64367fe0a..018f2b738e22 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2296,14 +2296,12 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-expr-common", "datafusion-functions", "datafusion-functions-aggregate", "datafusion-functions-aggregate-common", "datafusion-functions-table", "datafusion-functions-window", "datafusion-physical-expr", - "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-proto", "datafusion-proto-common", diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index c8aee693a37d..f8dad4aa00d0 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -48,7 +48,6 @@ async-ffi = { version = "0.5.0", features = ["abi_stable"] } async-trait = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-expr-common = { workspace = true } datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } datafusion-functions = { workspace = true, optional = true } @@ -57,7 +56,6 @@ datafusion-functions-aggregate-common = { workspace = true } datafusion-functions-table = { workspace = true, optional = true } datafusion-functions-window = { workspace = true, optional = true } datafusion-physical-expr = { workspace = true } -datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-proto = { workspace = true } datafusion-proto-common = { workspace = true } From 648f39634e91dd71aad02a56b3c149df7f5f7738 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 15:43:33 -0500 Subject: [PATCH 65/69] prettier --- datafusion/ffi/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/ffi/README.md b/datafusion/ffi/README.md index bd9b6f1f2f3f..9df415160864 100644 --- a/datafusion/ffi/README.md +++ b/datafusion/ffi/README.md @@ -104,7 +104,7 @@ the example in `FFI_TableProvider`. ## Task Context Accessor Many of the FFI structs in this crate contain a `FFI_TaskContextAccessor`. The -purpose of this struct is to *weakly* hold a reference to a method to +purpose of this struct is to _weakly_ hold a reference to a method to access the current `TaskContext`. The reason we need this accessor is because we use the `datafusion-proto` crate to serialize and deserialize data across the FFI boundary. In particular, we need to serialize and deserialize @@ -128,7 +128,7 @@ implements `Session`. When reviewing the code, many of the structs in this crate contain a call to a `library_maker_id`. The purpose of this call is to determine if a library is -accessing *local* code through the FFI structs. Consider this example: you have +accessing _local_ code through the FFI structs. Consider this example: you have a `primary` program that exposes functions to create a schema provider. You have a `secondary` library that exposes a function to create a catalog provider and the `secondary` library uses the schema provider of the `primary` program. From 00147f9e197c7ea8170a2343a5752a9dc405d948 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 15:45:15 -0500 Subject: [PATCH 66/69] Spell check --- datafusion/ffi/src/session/task_context.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/ffi/src/session/task_context.rs b/datafusion/ffi/src/session/task_context.rs index 95ae984eb6dd..a15227327162 100644 --- a/datafusion/ffi/src/session/task_context.rs +++ b/datafusion/ffi/src/session/task_context.rs @@ -176,7 +176,7 @@ impl From for TaskContext { } let task_id = (ffi_ctx.task_id)(&ffi_ctx).map(|s| s.to_string()).into(); - let sesion_id = (ffi_ctx.session_id)(&ffi_ctx).into(); + let session_id = (ffi_ctx.session_id)(&ffi_ctx).into(); let session_config = (ffi_ctx.session_config)(&ffi_ctx); let session_config = SessionConfig::try_from(&session_config).unwrap_or_default(); @@ -237,7 +237,7 @@ impl From for TaskContext { TaskContext::new( task_id, - sesion_id, + session_id, session_config, scalar_functions, aggregate_functions, From 91afcd7725304251d002f834249a9bc25718f33d Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 15:45:25 -0500 Subject: [PATCH 67/69] doc check --- datafusion/ffi/src/table_provider.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/ffi/src/table_provider.rs b/datafusion/ffi/src/table_provider.rs index 894421b98091..038ba3c6d0bf 100644 --- a/datafusion/ffi/src/table_provider.rs +++ b/datafusion/ffi/src/table_provider.rs @@ -153,7 +153,7 @@ pub struct FFI_TableProvider { pub version: unsafe extern "C" fn() -> u64, /// Internal data. This is only to be accessed by the provider of the plan. - /// A [`ForeignExecutionPlan`] should never attempt to access this data. + /// A [`ForeignTableProvider`] should never attempt to access this data. pub private_data: *mut c_void, /// Utility to identify when FFI objects are accessed locally through From e747ff0fb1fa8a1dba86bf22fa8c4537f292a765 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 15:45:57 -0500 Subject: [PATCH 68/69] taplo format --- datafusion/ffi/Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index f8dad4aa00d0..131b90102a18 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -46,10 +46,10 @@ arrow = { workspace = true, features = ["ffi"] } arrow-schema = { workspace = true } async-ffi = { version = "0.5.0", features = ["abi_stable"] } async-trait = { workspace = true } -datafusion-execution = { workspace = true } -datafusion-expr = { workspace = true } datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } datafusion-functions = { workspace = true, optional = true } datafusion-functions-aggregate = { workspace = true, optional = true } datafusion-functions-aggregate-common = { workspace = true } From b11194cc60e53207bfb2cde55cc2fc3814dc2683 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Tue, 11 Nov 2025 15:50:06 -0500 Subject: [PATCH 69/69] remove unused dependency --- Cargo.lock | 1 - datafusion-examples/examples/ffi/ffi_module_loader/Cargo.toml | 1 - 2 files changed, 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 018f2b738e22..2a462d360782 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3046,7 +3046,6 @@ version = "0.1.0" dependencies = [ "abi_stable", "datafusion", - "datafusion-ffi", "ffi_module_interface", "tokio", ] diff --git a/datafusion-examples/examples/ffi/ffi_module_loader/Cargo.toml b/datafusion-examples/examples/ffi/ffi_module_loader/Cargo.toml index 028a366aab1c..1f68bb1bb1be 100644 --- a/datafusion-examples/examples/ffi/ffi_module_loader/Cargo.toml +++ b/datafusion-examples/examples/ffi/ffi_module_loader/Cargo.toml @@ -24,6 +24,5 @@ publish = false [dependencies] abi_stable = "0.11.3" datafusion = { workspace = true } -datafusion-ffi = { workspace = true } ffi_module_interface = { path = "../ffi_module_interface" } tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] }