@@ -19,8 +19,8 @@ use std::{ffi::c_void, sync::Arc};
19
19
20
20
use abi_stable:: {
21
21
std_types:: {
22
- RResult :: { self , RErr , ROk } ,
23
- RStr , RVec ,
22
+ RResult :: { self , ROk } ,
23
+ RString , RVec ,
24
24
} ,
25
25
StableAbi ,
26
26
} ;
@@ -44,7 +44,7 @@ use datafusion_proto::{
44
44
} ;
45
45
use prost:: Message ;
46
46
47
- use crate :: arrow_wrappers:: WrappedSchema ;
47
+ use crate :: { arrow_wrappers:: WrappedSchema , df_result , rresult_return } ;
48
48
49
49
/// A stable struct for sharing [`PlanProperties`] across FFI boundaries.
50
50
#[ repr( C ) ]
@@ -54,7 +54,7 @@ pub struct FFI_PlanProperties {
54
54
/// The output partitioning is a [`Partitioning`] protobuf message serialized
55
55
/// into bytes to pass across the FFI boundary.
56
56
pub output_partitioning :
57
- unsafe extern "C" fn ( plan : & Self ) -> RResult < RVec < u8 > , RStr < ' static > > ,
57
+ unsafe extern "C" fn ( plan : & Self ) -> RResult < RVec < u8 > , RString > ,
58
58
59
59
/// Return the emission type of the plan.
60
60
pub emission_type : unsafe extern "C" fn ( plan : & Self ) -> FFI_EmissionType ,
@@ -64,8 +64,7 @@ pub struct FFI_PlanProperties {
64
64
65
65
/// The output ordering is a [`PhysicalSortExprNodeCollection`] protobuf message
66
66
/// serialized into bytes to pass across the FFI boundary.
67
- pub output_ordering :
68
- unsafe extern "C" fn ( plan : & Self ) -> RResult < RVec < u8 > , RStr < ' static > > ,
67
+ pub output_ordering : unsafe extern "C" fn ( plan : & Self ) -> RResult < RVec < u8 > , RString > ,
69
68
70
69
/// Return the schema of the plan.
71
70
pub schema : unsafe extern "C" fn ( plan : & Self ) -> WrappedSchema ,
@@ -84,21 +83,13 @@ struct PlanPropertiesPrivateData {
84
83
85
84
unsafe extern "C" fn output_partitioning_fn_wrapper (
86
85
properties : & FFI_PlanProperties ,
87
- ) -> RResult < RVec < u8 > , RStr < ' static > > {
86
+ ) -> RResult < RVec < u8 > , RString > {
88
87
let private_data = properties. private_data as * const PlanPropertiesPrivateData ;
89
88
let props = & ( * private_data) . props ;
90
89
91
90
let codec = DefaultPhysicalExtensionCodec { } ;
92
91
let partitioning_data =
93
- match serialize_partitioning ( props. output_partitioning ( ) , & codec) {
94
- Ok ( p) => p,
95
- Err ( _) => {
96
- return RErr (
97
- "unable to serialize output_partitioning in FFI_PlanProperties"
98
- . into ( ) ,
99
- )
100
- }
101
- } ;
92
+ rresult_return ! ( serialize_partitioning( props. output_partitioning( ) , & codec) ) ;
102
93
let output_partitioning = partitioning_data. encode_to_vec ( ) ;
103
94
104
95
ROk ( output_partitioning. into ( ) )
@@ -122,31 +113,24 @@ unsafe extern "C" fn boundedness_fn_wrapper(
122
113
123
114
unsafe extern "C" fn output_ordering_fn_wrapper (
124
115
properties : & FFI_PlanProperties ,
125
- ) -> RResult < RVec < u8 > , RStr < ' static > > {
116
+ ) -> RResult < RVec < u8 > , RString > {
126
117
let private_data = properties. private_data as * const PlanPropertiesPrivateData ;
127
118
let props = & ( * private_data) . props ;
128
119
129
120
let codec = DefaultPhysicalExtensionCodec { } ;
130
- let output_ordering =
131
- match props. output_ordering ( ) {
132
- Some ( ordering) => {
133
- let physical_sort_expr_nodes =
134
- match serialize_physical_sort_exprs ( ordering. to_owned ( ) , & codec) {
135
- Ok ( v) => v,
136
- Err ( _) => return RErr (
137
- "unable to serialize output_ordering in FFI_PlanProperties"
138
- . into ( ) ,
139
- ) ,
140
- } ;
141
-
142
- let ordering_data = PhysicalSortExprNodeCollection {
143
- physical_sort_expr_nodes,
144
- } ;
145
-
146
- ordering_data. encode_to_vec ( )
147
- }
148
- None => Vec :: default ( ) ,
149
- } ;
121
+ let output_ordering = match props. output_ordering ( ) {
122
+ Some ( ordering) => {
123
+ let physical_sort_expr_nodes = rresult_return ! (
124
+ serialize_physical_sort_exprs( ordering. to_owned( ) , & codec)
125
+ ) ;
126
+ let ordering_data = PhysicalSortExprNodeCollection {
127
+ physical_sort_expr_nodes,
128
+ } ;
129
+
130
+ ordering_data. encode_to_vec ( )
131
+ }
132
+ None => Vec :: default ( ) ,
133
+ } ;
150
134
ROk ( output_ordering. into ( ) )
151
135
}
152
136
@@ -200,40 +184,32 @@ impl TryFrom<FFI_PlanProperties> for PlanProperties {
200
184
let codex = DefaultPhysicalExtensionCodec { } ;
201
185
202
186
let ffi_orderings = unsafe { ( ffi_props. output_ordering ) ( & ffi_props) } ;
203
- let orderings = match ffi_orderings {
204
- ROk ( ordering_vec) => {
205
- let proto_output_ordering =
206
- PhysicalSortExprNodeCollection :: decode ( ordering_vec. as_ref ( ) )
207
- . map_err ( |e| DataFusionError :: External ( Box :: new ( e) ) ) ?;
208
- Some ( parse_physical_sort_exprs (
209
- & proto_output_ordering. physical_sort_expr_nodes ,
210
- & default_ctx,
211
- & schema,
212
- & codex,
213
- ) ?)
214
- }
215
- RErr ( e) => return Err ( DataFusionError :: Plan ( e. to_string ( ) ) ) ,
216
- } ;
217
187
218
- let ffi_partitioning = unsafe { ( ffi_props. output_partitioning ) ( & ffi_props) } ;
219
- let partitioning = match ffi_partitioning {
220
- ROk ( partitioning_vec) => {
221
- let proto_output_partitioning =
222
- Partitioning :: decode ( partitioning_vec. as_ref ( ) )
223
- . map_err ( |e| DataFusionError :: External ( Box :: new ( e) ) ) ?;
224
- parse_protobuf_partitioning (
225
- Some ( & proto_output_partitioning) ,
226
- & default_ctx,
227
- & schema,
228
- & codex,
229
- ) ?
230
- . ok_or ( DataFusionError :: Plan (
231
- "Unable to deserialize partitioning protobuf in FFI_PlanProperties"
232
- . to_string ( ) ,
233
- ) )
234
- }
235
- RErr ( e) => Err ( DataFusionError :: Plan ( e. to_string ( ) ) ) ,
236
- } ?;
188
+ let proto_output_ordering =
189
+ PhysicalSortExprNodeCollection :: decode ( df_result ! ( ffi_orderings) ?. as_ref ( ) )
190
+ . map_err ( |e| DataFusionError :: External ( Box :: new ( e) ) ) ?;
191
+ let orderings = Some ( parse_physical_sort_exprs (
192
+ & proto_output_ordering. physical_sort_expr_nodes ,
193
+ & default_ctx,
194
+ & schema,
195
+ & codex,
196
+ ) ?) ;
197
+
198
+ let partitioning_vec =
199
+ unsafe { df_result ! ( ( ffi_props. output_partitioning) ( & ffi_props) ) ? } ;
200
+ let proto_output_partitioning =
201
+ Partitioning :: decode ( partitioning_vec. as_ref ( ) )
202
+ . map_err ( |e| DataFusionError :: External ( Box :: new ( e) ) ) ?;
203
+ let partitioning = parse_protobuf_partitioning (
204
+ Some ( & proto_output_partitioning) ,
205
+ & default_ctx,
206
+ & schema,
207
+ & codex,
208
+ ) ?
209
+ . ok_or ( DataFusionError :: Plan (
210
+ "Unable to deserialize partitioning protobuf in FFI_PlanProperties"
211
+ . to_string ( ) ,
212
+ ) ) ?;
237
213
238
214
let eq_properties = match orderings {
239
215
Some ( ordering) => {
0 commit comments