datafusion-ffi 53.1.0

Foreign Function Interface implementation for DataFusion
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
// 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::ffi::c_void;
use std::hash::{Hash, Hasher};
use std::sync::Arc;

use abi_stable::StableAbi;
use abi_stable::std_types::{ROption, RResult, RStr, RString, RVec};
use accumulator::FFI_Accumulator;
use accumulator_args::{FFI_AccumulatorArgs, ForeignAccumulatorArgs};
use arrow::datatypes::{DataType, Field};
use arrow::ffi::FFI_ArrowSchema;
use arrow_schema::FieldRef;
use datafusion_common::{DataFusionError, Result, ffi_datafusion_err};
use datafusion_expr::function::AggregateFunctionSimplification;
use datafusion_expr::type_coercion::functions::fields_with_udf;
use datafusion_expr::{
    Accumulator, AggregateUDF, AggregateUDFImpl, GroupsAccumulator, Signature,
};
use datafusion_functions_aggregate_common::accumulator::{
    AccumulatorArgs, StateFieldsArgs,
};
use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity;
use datafusion_proto_common::from_proto::parse_proto_fields_to_fields;
use groups_accumulator::FFI_GroupsAccumulator;
use prost::{DecodeError, Message};

use crate::arrow_wrappers::WrappedSchema;
use crate::util::{
    FFIResult, rvec_wrapped_to_vec_datatype, rvec_wrapped_to_vec_fieldref,
    vec_datatype_to_rvec_wrapped, vec_fieldref_to_rvec_wrapped,
};
use crate::volatility::FFI_Volatility;
use crate::{df_result, rresult, rresult_return};

mod accumulator;
mod accumulator_args;
mod groups_accumulator;

/// A stable struct for sharing a [`AggregateUDF`] across FFI boundaries.
#[repr(C)]
#[derive(Debug, StableAbi)]
pub struct FFI_AggregateUDF {
    /// FFI equivalent to the `name` of a [`AggregateUDF`]
    pub name: RString,

    /// FFI equivalent to the `aliases` of a [`AggregateUDF`]
    pub aliases: RVec<RString>,

    /// FFI equivalent to the `volatility` of a [`AggregateUDF`]
    pub volatility: FFI_Volatility,

    /// Determines the return field of the underlying [`AggregateUDF`] based on the
    /// argument fields.
    pub return_field: unsafe extern "C" fn(
        udaf: &Self,
        arg_fields: RVec<WrappedSchema>,
    ) -> FFIResult<WrappedSchema>,

    /// FFI equivalent to the `is_nullable` of a [`AggregateUDF`]
    pub is_nullable: bool,

    /// FFI equivalent to [`AggregateUDF::groups_accumulator_supported`]
    pub groups_accumulator_supported:
        unsafe extern "C" fn(udaf: &FFI_AggregateUDF, args: FFI_AccumulatorArgs) -> bool,

    /// FFI equivalent to [`AggregateUDF::accumulator`]
    pub accumulator: unsafe extern "C" fn(
        udaf: &FFI_AggregateUDF,
        args: FFI_AccumulatorArgs,
    ) -> FFIResult<FFI_Accumulator>,

    /// FFI equivalent to [`AggregateUDF::create_sliding_accumulator`]
    pub create_sliding_accumulator: unsafe extern "C" fn(
        udaf: &FFI_AggregateUDF,
        args: FFI_AccumulatorArgs,
    )
        -> FFIResult<FFI_Accumulator>,

    /// FFI equivalent to [`AggregateUDF::state_fields`]
    pub state_fields: unsafe extern "C" fn(
        udaf: &FFI_AggregateUDF,
        name: &RStr,
        input_fields: RVec<WrappedSchema>,
        return_field: WrappedSchema,
        ordering_fields: RVec<RVec<u8>>,
        is_distinct: bool,
    ) -> FFIResult<RVec<RVec<u8>>>,

    /// FFI equivalent to [`AggregateUDF::create_groups_accumulator`]
    pub create_groups_accumulator:
        unsafe extern "C" fn(
            udaf: &FFI_AggregateUDF,
            args: FFI_AccumulatorArgs,
        ) -> FFIResult<FFI_GroupsAccumulator>,

    /// FFI equivalent to [`AggregateUDF::with_beneficial_ordering`]
    pub with_beneficial_ordering:
        unsafe extern "C" fn(
            udaf: &FFI_AggregateUDF,
            beneficial_ordering: bool,
        ) -> FFIResult<ROption<FFI_AggregateUDF>>,

    /// FFI equivalent to [`AggregateUDF::order_sensitivity`]
    pub order_sensitivity:
        unsafe extern "C" fn(udaf: &FFI_AggregateUDF) -> FFI_AggregateOrderSensitivity,

    /// Performs type coercion. To simply this interface, all UDFs are treated as having
    /// user defined signatures, which will in turn call coerce_types to be called. This
    /// call should be transparent to most users as the internal function performs the
    /// appropriate calls on the underlying [`AggregateUDF`]
    pub coerce_types: unsafe extern "C" fn(
        udf: &Self,
        arg_types: RVec<WrappedSchema>,
    ) -> FFIResult<RVec<WrappedSchema>>,

    /// 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,

    /// Release the memory of the private data when it is no longer being used.
    pub release: unsafe extern "C" fn(udaf: &mut Self),

    /// 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. See [`crate::get_library_marker_id`] and
    /// the crate's `README.md` for more information.
    pub library_marker_id: extern "C" fn() -> usize,
}

unsafe impl Send for FFI_AggregateUDF {}
unsafe impl Sync for FFI_AggregateUDF {}

pub struct AggregateUDFPrivateData {
    pub udaf: Arc<AggregateUDF>,
}

impl FFI_AggregateUDF {
    unsafe fn inner(&self) -> &Arc<AggregateUDF> {
        unsafe {
            let private_data = self.private_data as *const AggregateUDFPrivateData;
            &(*private_data).udaf
        }
    }
}

unsafe extern "C" fn return_field_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    arg_fields: RVec<WrappedSchema>,
) -> FFIResult<WrappedSchema> {
    unsafe {
        let udaf = udaf.inner();

        let arg_fields = rresult_return!(rvec_wrapped_to_vec_fieldref(&arg_fields));

        let return_field = udaf
            .return_field(&arg_fields)
            .and_then(|v| {
                FFI_ArrowSchema::try_from(v.as_ref()).map_err(DataFusionError::from)
            })
            .map(WrappedSchema);

        rresult!(return_field)
    }
}

unsafe extern "C" fn accumulator_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    args: FFI_AccumulatorArgs,
) -> FFIResult<FFI_Accumulator> {
    unsafe {
        let udaf = udaf.inner();

        let accumulator_args = &rresult_return!(ForeignAccumulatorArgs::try_from(args));

        rresult!(
            udaf.accumulator(accumulator_args.into())
                .map(FFI_Accumulator::from)
        )
    }
}

unsafe extern "C" fn create_sliding_accumulator_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    args: FFI_AccumulatorArgs,
) -> FFIResult<FFI_Accumulator> {
    unsafe {
        let udaf = udaf.inner();

        let accumulator_args = &rresult_return!(ForeignAccumulatorArgs::try_from(args));

        rresult!(
            udaf.create_sliding_accumulator(accumulator_args.into())
                .map(FFI_Accumulator::from)
        )
    }
}

unsafe extern "C" fn create_groups_accumulator_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    args: FFI_AccumulatorArgs,
) -> FFIResult<FFI_GroupsAccumulator> {
    unsafe {
        let udaf = udaf.inner();

        let accumulator_args = &rresult_return!(ForeignAccumulatorArgs::try_from(args));

        rresult!(
            udaf.create_groups_accumulator(accumulator_args.into())
                .map(FFI_GroupsAccumulator::from)
        )
    }
}

unsafe extern "C" fn groups_accumulator_supported_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    args: FFI_AccumulatorArgs,
) -> bool {
    unsafe {
        let udaf = udaf.inner();

        ForeignAccumulatorArgs::try_from(args)
            .map(|a| udaf.groups_accumulator_supported((&a).into()))
            .unwrap_or_else(|e| {
                log::warn!("Unable to parse accumulator args. {e}");
                false
            })
    }
}

unsafe extern "C" fn with_beneficial_ordering_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    beneficial_ordering: bool,
) -> FFIResult<ROption<FFI_AggregateUDF>> {
    unsafe {
        let udaf = udaf.inner().as_ref().clone();

        let result = rresult_return!(udaf.with_beneficial_ordering(beneficial_ordering));
        let result = rresult_return!(
            result
                .map(|func| func.with_beneficial_ordering(beneficial_ordering))
                .transpose()
        )
        .flatten()
        .map(|func| FFI_AggregateUDF::from(Arc::new(func)));

        RResult::ROk(result.into())
    }
}

unsafe extern "C" fn state_fields_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    name: &RStr,
    input_fields: RVec<WrappedSchema>,
    return_field: WrappedSchema,
    ordering_fields: RVec<RVec<u8>>,
    is_distinct: bool,
) -> FFIResult<RVec<RVec<u8>>> {
    unsafe {
        let udaf = udaf.inner();

        let input_fields = &rresult_return!(rvec_wrapped_to_vec_fieldref(&input_fields));
        let return_field = rresult_return!(Field::try_from(&return_field.0)).into();

        let ordering_fields = &rresult_return!(
            ordering_fields
                .into_iter()
                .map(|field_bytes| datafusion_proto_common::Field::decode(
                    field_bytes.as_ref()
                ))
                .collect::<std::result::Result<Vec<_>, DecodeError>>()
        );

        let ordering_fields =
            &rresult_return!(parse_proto_fields_to_fields(ordering_fields))
                .into_iter()
                .map(Arc::new)
                .collect::<Vec<_>>();

        let args = StateFieldsArgs {
            name: name.as_str(),
            input_fields,
            return_field,
            ordering_fields,
            is_distinct,
        };

        let state_fields = rresult_return!(udaf.state_fields(args));
        let state_fields = rresult_return!(
            state_fields
                .iter()
                .map(|f| f.as_ref())
                .map(datafusion_proto::protobuf::Field::try_from)
                .map(|v| v.map_err(DataFusionError::from))
                .collect::<Result<Vec<_>>>()
        )
        .into_iter()
        .map(|field| field.encode_to_vec().into())
        .collect();

        RResult::ROk(state_fields)
    }
}

unsafe extern "C" fn order_sensitivity_fn_wrapper(
    udaf: &FFI_AggregateUDF,
) -> FFI_AggregateOrderSensitivity {
    unsafe { udaf.inner().order_sensitivity().into() }
}

unsafe extern "C" fn coerce_types_fn_wrapper(
    udaf: &FFI_AggregateUDF,
    arg_types: RVec<WrappedSchema>,
) -> FFIResult<RVec<WrappedSchema>> {
    unsafe {
        let udaf = udaf.inner();

        let arg_types = rresult_return!(rvec_wrapped_to_vec_datatype(&arg_types));

        let arg_fields = arg_types
            .iter()
            .map(|dt| Field::new("f", dt.clone(), true))
            .map(Arc::new)
            .collect::<Vec<_>>();
        let return_types = rresult_return!(fields_with_udf(&arg_fields, udaf.as_ref()))
            .into_iter()
            .map(|f| f.data_type().to_owned())
            .collect::<Vec<_>>();

        rresult!(vec_datatype_to_rvec_wrapped(&return_types))
    }
}

unsafe extern "C" fn release_fn_wrapper(udaf: &mut FFI_AggregateUDF) {
    unsafe {
        debug_assert!(!udaf.private_data.is_null());
        let private_data =
            Box::from_raw(udaf.private_data as *mut AggregateUDFPrivateData);
        drop(private_data);
        udaf.private_data = std::ptr::null_mut();
    }
}

unsafe extern "C" fn clone_fn_wrapper(udaf: &FFI_AggregateUDF) -> FFI_AggregateUDF {
    unsafe { Arc::clone(udaf.inner()).into() }
}

impl Clone for FFI_AggregateUDF {
    fn clone(&self) -> Self {
        unsafe { (self.clone)(self) }
    }
}

impl From<Arc<AggregateUDF>> for FFI_AggregateUDF {
    fn from(udaf: Arc<AggregateUDF>) -> Self {
        let name = udaf.name().into();
        let aliases = udaf.aliases().iter().map(|a| a.to_owned().into()).collect();
        let is_nullable = udaf.is_nullable();
        let volatility = udaf.signature().volatility.into();

        let private_data = Box::new(AggregateUDFPrivateData { udaf });

        Self {
            name,
            is_nullable,
            volatility,
            aliases,
            return_field: return_field_fn_wrapper,
            accumulator: accumulator_fn_wrapper,
            create_sliding_accumulator: create_sliding_accumulator_fn_wrapper,
            create_groups_accumulator: create_groups_accumulator_fn_wrapper,
            groups_accumulator_supported: groups_accumulator_supported_fn_wrapper,
            with_beneficial_ordering: with_beneficial_ordering_fn_wrapper,
            state_fields: state_fields_fn_wrapper,
            order_sensitivity: order_sensitivity_fn_wrapper,
            coerce_types: coerce_types_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 Drop for FFI_AggregateUDF {
    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.
///
/// The ForeignAggregateUDF is to be used by the caller of the UDF, so it has
/// no knowledge or access to the private data. All interaction with the UDF
/// must occur through the functions defined in FFI_AggregateUDF.
#[derive(Debug)]
pub struct ForeignAggregateUDF {
    signature: Signature,
    aliases: Vec<String>,
    udaf: FFI_AggregateUDF,
}

unsafe impl Send for ForeignAggregateUDF {}
unsafe impl Sync for ForeignAggregateUDF {}

impl PartialEq for ForeignAggregateUDF {
    fn eq(&self, other: &Self) -> bool {
        // FFI_AggregateUDF cannot be compared, so identity equality is the best we can do.
        std::ptr::eq(self, other)
    }
}
impl Eq for ForeignAggregateUDF {}
impl Hash for ForeignAggregateUDF {
    fn hash<H: Hasher>(&self, state: &mut H) {
        std::ptr::hash(self, state)
    }
}

impl From<&FFI_AggregateUDF> for Arc<dyn AggregateUDFImpl> {
    fn from(udaf: &FFI_AggregateUDF) -> Self {
        if (udaf.library_marker_id)() == crate::get_library_marker_id() {
            return Arc::clone(unsafe { udaf.inner().inner() });
        }

        let signature = Signature::user_defined((&udaf.volatility).into());
        let aliases = udaf.aliases.iter().map(|s| s.to_string()).collect();

        Arc::new(ForeignAggregateUDF {
            udaf: udaf.clone(),
            signature,
            aliases,
        })
    }
}

impl AggregateUDFImpl for ForeignAggregateUDF {
    fn as_any(&self) -> &dyn std::any::Any {
        self
    }

    fn name(&self) -> &str {
        self.udaf.name.as_str()
    }

    fn signature(&self) -> &Signature {
        &self.signature
    }

    fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
        unimplemented!()
    }

    fn return_field(&self, arg_fields: &[FieldRef]) -> Result<FieldRef> {
        let arg_fields = vec_fieldref_to_rvec_wrapped(arg_fields)?;

        let result = unsafe { (self.udaf.return_field)(&self.udaf, arg_fields) };

        let result = df_result!(result);

        result.and_then(|r| {
            Field::try_from(&r.0)
                .map(Arc::new)
                .map_err(DataFusionError::from)
        })
    }

    fn is_nullable(&self) -> bool {
        self.udaf.is_nullable
    }

    fn accumulator(&self, acc_args: AccumulatorArgs) -> Result<Box<dyn Accumulator>> {
        let args = acc_args.try_into()?;
        unsafe {
            df_result!((self.udaf.accumulator)(&self.udaf, args))
                .map(<Box<dyn Accumulator>>::from)
        }
    }

    fn state_fields(&self, args: StateFieldsArgs) -> Result<Vec<FieldRef>> {
        unsafe {
            let name = RStr::from_str(args.name);
            let input_fields = vec_fieldref_to_rvec_wrapped(args.input_fields)?;
            let return_field =
                WrappedSchema(FFI_ArrowSchema::try_from(args.return_field.as_ref())?);
            let ordering_fields = args
                .ordering_fields
                .iter()
                .map(|f| f.as_ref())
                .map(datafusion_proto::protobuf::Field::try_from)
                .map(|v| v.map_err(DataFusionError::from))
                .collect::<Result<Vec<_>>>()?
                .into_iter()
                .map(|proto_field| proto_field.encode_to_vec().into())
                .collect();

            let fields = df_result!((self.udaf.state_fields)(
                &self.udaf,
                &name,
                input_fields,
                return_field,
                ordering_fields,
                args.is_distinct
            ))?;
            let fields = fields
                .into_iter()
                .map(|field_bytes| {
                    datafusion_proto_common::Field::decode(field_bytes.as_ref())
                        .map_err(|e| ffi_datafusion_err!("{e}"))
                })
                .collect::<Result<Vec<_>>>()?;

            parse_proto_fields_to_fields(fields.iter())
                .map(|fields| fields.into_iter().map(Arc::new).collect())
                .map_err(|e| ffi_datafusion_err!("{e}"))
        }
    }

    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;
            }
        };

        unsafe { (self.udaf.groups_accumulator_supported)(&self.udaf, args) }
    }

    fn create_groups_accumulator(
        &self,
        args: AccumulatorArgs,
    ) -> Result<Box<dyn GroupsAccumulator>> {
        let args = FFI_AccumulatorArgs::try_from(args)?;

        unsafe {
            df_result!((self.udaf.create_groups_accumulator)(&self.udaf, args))
                .map(<Box<dyn GroupsAccumulator>>::from)
        }
    }

    fn aliases(&self) -> &[String] {
        &self.aliases
    }

    fn create_sliding_accumulator(
        &self,
        args: AccumulatorArgs,
    ) -> Result<Box<dyn Accumulator>> {
        let args = args.try_into()?;
        unsafe {
            df_result!((self.udaf.create_sliding_accumulator)(&self.udaf, args))
                .map(<Box<dyn Accumulator>>::from)
        }
    }

    fn with_beneficial_ordering(
        self: Arc<Self>,
        beneficial_ordering: bool,
    ) -> Result<Option<Arc<dyn AggregateUDFImpl>>> {
        unsafe {
            let result = df_result!((self.udaf.with_beneficial_ordering)(
                &self.udaf,
                beneficial_ordering
            ))?
            .into_option();

            let result = result.map(|func| <Arc<dyn AggregateUDFImpl>>::from(&func));

            Ok(result)
        }
    }

    fn order_sensitivity(&self) -> AggregateOrderSensitivity {
        unsafe { (self.udaf.order_sensitivity)(&self.udaf).into() }
    }

    fn simplify(&self) -> Option<AggregateFunctionSimplification> {
        None
    }

    fn coerce_types(&self, arg_types: &[DataType]) -> Result<Vec<DataType>> {
        unsafe {
            let arg_types = vec_datatype_to_rvec_wrapped(arg_types)?;
            let result_types =
                df_result!((self.udaf.coerce_types)(&self.udaf, arg_types))?;
            Ok(rvec_wrapped_to_vec_datatype(&result_types)?)
        }
    }
}

#[repr(C)]
#[derive(Debug, StableAbi)]
pub enum FFI_AggregateOrderSensitivity {
    Insensitive,
    HardRequirement,
    SoftRequirement,
    Beneficial,
}

impl From<FFI_AggregateOrderSensitivity> for AggregateOrderSensitivity {
    fn from(value: FFI_AggregateOrderSensitivity) -> Self {
        match value {
            FFI_AggregateOrderSensitivity::Insensitive => Self::Insensitive,
            FFI_AggregateOrderSensitivity::HardRequirement => Self::HardRequirement,
            FFI_AggregateOrderSensitivity::SoftRequirement => Self::SoftRequirement,
            FFI_AggregateOrderSensitivity::Beneficial => Self::Beneficial,
        }
    }
}

impl From<AggregateOrderSensitivity> for FFI_AggregateOrderSensitivity {
    fn from(value: AggregateOrderSensitivity) -> Self {
        match value {
            AggregateOrderSensitivity::Insensitive => Self::Insensitive,
            AggregateOrderSensitivity::HardRequirement => Self::HardRequirement,
            AggregateOrderSensitivity::SoftRequirement => Self::SoftRequirement,
            AggregateOrderSensitivity::Beneficial => Self::Beneficial,
        }
    }
}

#[cfg(test)]
mod tests {
    use std::any::Any;
    use std::collections::HashMap;

    use arrow::datatypes::Schema;
    use datafusion::common::create_array;
    use datafusion::functions_aggregate::sum::Sum;
    use datafusion::physical_expr::PhysicalSortExpr;
    use datafusion::physical_plan::expressions::col;
    use datafusion::scalar::ScalarValue;

    use super::*;

    #[derive(Default, Debug, Hash, Eq, PartialEq)]
    struct SumWithCopiedMetadata {
        inner: Sum,
    }

    impl AggregateUDFImpl for SumWithCopiedMetadata {
        fn as_any(&self) -> &dyn Any {
            self
        }

        fn name(&self) -> &str {
            self.inner.name()
        }

        fn signature(&self) -> &Signature {
            self.inner.signature()
        }

        fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
            unimplemented!()
        }

        fn return_field(&self, arg_fields: &[FieldRef]) -> Result<FieldRef> {
            // Copy the input field, so any metadata gets returned
            Ok(Arc::clone(&arg_fields[0]))
        }

        fn accumulator(&self, acc_args: AccumulatorArgs) -> Result<Box<dyn Accumulator>> {
            self.inner.accumulator(acc_args)
        }
    }

    fn create_test_foreign_udaf(
        original_udaf: impl AggregateUDFImpl + 'static,
    ) -> Result<AggregateUDF> {
        let original_udaf = Arc::new(AggregateUDF::from(original_udaf));

        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<dyn AggregateUDFImpl> = (&local_udaf).into();
        Ok(AggregateUDF::new_from_shared_impl(foreign_udaf))
    }

    #[test]
    fn test_round_trip_udaf() -> Result<()> {
        let original_udaf = Sum::new();
        let original_name = original_udaf.name().to_owned();
        let original_udaf = Arc::new(AggregateUDF::from(original_udaf));

        // Convert to FFI format
        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<dyn AggregateUDFImpl> = (&local_udaf).into();
        let foreign_udaf = AggregateUDF::new_from_shared_impl(foreign_udaf);

        assert_eq!(original_name, foreign_udaf.name());
        Ok(())
    }

    #[test]
    fn test_foreign_udaf_aliases() -> Result<()> {
        let foreign_udaf =
            create_test_foreign_udaf(Sum::new())?.with_aliases(["my_function"]);

        let return_field =
            foreign_udaf
                .return_field(&[Field::new("a", DataType::Float64, true).into()])?;
        let return_type = return_field.data_type();
        assert_eq!(return_type, &DataType::Float64);
        Ok(())
    }

    #[test]
    fn test_foreign_udaf_accumulator() -> Result<()> {
        let foreign_udaf = create_test_foreign_udaf(Sum::new())?;

        let schema = Schema::new(vec![Field::new("a", DataType::Float64, true)]);
        let acc_args = AccumulatorArgs {
            return_field: Field::new("f", DataType::Float64, true).into(),
            schema: &schema,
            expr_fields: &[Field::new("a", DataType::Float64, true).into()],
            ignore_nulls: true,
            order_bys: &[PhysicalSortExpr::new_default(col("a", &schema)?)],
            is_reversed: false,
            name: "round_trip",
            is_distinct: true,
            exprs: &[col("a", &schema)?],
        };
        let mut accumulator = foreign_udaf.accumulator(acc_args)?;
        let values = create_array!(Float64, vec![10., 20., 30., 40., 50.]);
        accumulator.update_batch(&[values])?;
        let resultant_value = accumulator.evaluate()?;
        assert_eq!(resultant_value, ScalarValue::Float64(Some(150.)));

        Ok(())
    }

    #[test]
    fn test_round_trip_udaf_metadata() -> Result<()> {
        let original_udaf = SumWithCopiedMetadata::default();
        let original_udaf = Arc::new(AggregateUDF::from(original_udaf));

        // Convert to FFI format
        let local_udaf: FFI_AggregateUDF = Arc::clone(&original_udaf).into();

        // Convert back to native format
        let foreign_udaf: Arc<dyn AggregateUDFImpl> = (&local_udaf).into();
        let foreign_udaf = AggregateUDF::new_from_shared_impl(foreign_udaf);

        let metadata: HashMap<String, String> =
            [("a_key".to_string(), "a_value".to_string())]
                .into_iter()
                .collect();
        let input_field = Arc::new(
            Field::new("a", DataType::Float64, false).with_metadata(metadata.clone()),
        );
        let return_field = foreign_udaf.return_field(&[input_field])?;

        assert_eq!(&metadata, return_field.metadata());
        Ok(())
    }

    #[test]
    fn test_beneficial_ordering() -> Result<()> {
        let foreign_udaf = create_test_foreign_udaf(
            datafusion::functions_aggregate::first_last::FirstValue::new(),
        )?;

        let foreign_udaf = foreign_udaf.with_beneficial_ordering(true)?.unwrap();

        assert_eq!(
            foreign_udaf.order_sensitivity(),
            AggregateOrderSensitivity::Beneficial
        );

        let a_field = Arc::new(Field::new("a", DataType::Float64, true));
        let state_fields = foreign_udaf.state_fields(StateFieldsArgs {
            name: "a",
            input_fields: &[Field::new("f", DataType::Float64, true).into()],
            return_field: Field::new("f", DataType::Float64, true).into(),
            ordering_fields: &[Arc::clone(&a_field)],
            is_distinct: false,
        })?;

        assert_eq!(state_fields.len(), 3);
        assert_eq!(state_fields[1], a_field);
        Ok(())
    }

    #[test]
    fn test_sliding_accumulator() -> Result<()> {
        let foreign_udaf = create_test_foreign_udaf(Sum::new())?;

        let schema = Schema::new(vec![Field::new("a", DataType::Float64, true)]);
        // Note: sum distinct is only support Int64 until now
        let acc_args = AccumulatorArgs {
            return_field: Field::new("f", DataType::Float64, true).into(),
            schema: &schema,
            expr_fields: &[Field::new("a", DataType::Float64, true).into()],
            ignore_nulls: true,
            order_bys: &[PhysicalSortExpr::new_default(col("a", &schema)?)],
            is_reversed: false,
            name: "round_trip",
            is_distinct: false,
            exprs: &[col("a", &schema)?],
        };

        let mut accumulator = foreign_udaf.create_sliding_accumulator(acc_args)?;
        let values = create_array!(Float64, vec![10., 20., 30., 40., 50.]);
        accumulator.update_batch(&[values])?;
        let resultant_value = accumulator.evaluate()?;
        assert_eq!(resultant_value, ScalarValue::Float64(Some(150.)));

        Ok(())
    }

    fn test_round_trip_order_sensitivity(sensitivity: AggregateOrderSensitivity) {
        let ffi_sensitivity: FFI_AggregateOrderSensitivity = sensitivity.into();
        let round_trip_sensitivity: AggregateOrderSensitivity = ffi_sensitivity.into();

        assert_eq!(sensitivity, round_trip_sensitivity);
    }

    #[test]
    fn test_round_trip_all_order_sensitivities() {
        test_round_trip_order_sensitivity(AggregateOrderSensitivity::Insensitive);
        test_round_trip_order_sensitivity(AggregateOrderSensitivity::HardRequirement);
        test_round_trip_order_sensitivity(AggregateOrderSensitivity::SoftRequirement);
        test_round_trip_order_sensitivity(AggregateOrderSensitivity::Beneficial);
    }

    #[test]
    fn test_ffi_udaf_local_bypass() -> Result<()> {
        let original_udaf = Sum::new();
        let original_udaf = Arc::new(AggregateUDF::from(original_udaf));

        let mut ffi_udaf = FFI_AggregateUDF::from(original_udaf);

        // Verify local libraries can be downcast to their original
        let foreign_udaf: Arc<dyn AggregateUDFImpl> = (&ffi_udaf).into();
        assert!(foreign_udaf.as_any().downcast_ref::<Sum>().is_some());

        // Verify different library markers generate foreign providers
        ffi_udaf.library_marker_id = crate::mock_foreign_marker_id;
        let foreign_udaf: Arc<dyn AggregateUDFImpl> = (&ffi_udaf).into();
        assert!(
            foreign_udaf
                .as_any()
                .downcast_ref::<ForeignAggregateUDF>()
                .is_some()
        );

        Ok(())
    }
}