datafusion-physical-plan 53.1.0

Physical (ExecutionPlan) implementations for DataFusion query engine
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
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
// 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.

// Some of these functions reference the Postgres documentation
// or implementation to ensure compatibility and are subject to
// the Postgres license.

//! The Union operator combines multiple inputs with the same schema

use std::borrow::Borrow;
use std::pin::Pin;
use std::task::{Context, Poll};
use std::{any::Any, sync::Arc};

use super::{
    ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan,
    ExecutionPlanProperties, Partitioning, PlanProperties, RecordBatchStream,
    SendableRecordBatchStream, Statistics,
    metrics::{ExecutionPlanMetricsSet, MetricsSet},
};
use crate::check_if_same_properties;
use crate::execution_plan::{
    InvariantLevel, boundedness_from_children, check_default_invariants,
    emission_type_from_children,
};
use crate::filter::FilterExec;
use crate::filter_pushdown::{
    ChildPushdownResult, FilterDescription, FilterPushdownPhase,
    FilterPushdownPropagation, PushedDown,
};
use crate::metrics::BaselineMetrics;
use crate::projection::{ProjectionExec, make_with_child};
use crate::stream::ObservedStream;

use arrow::datatypes::{Field, Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use datafusion_common::config::ConfigOptions;
use datafusion_common::stats::Precision;
use datafusion_common::{
    Result, assert_or_internal_err, exec_err, internal_datafusion_err,
};
use datafusion_execution::TaskContext;
use datafusion_physical_expr::{
    EquivalenceProperties, PhysicalExpr, calculate_union, conjunction,
};

use futures::Stream;
use itertools::Itertools;
use log::{debug, trace, warn};
use tokio::macros::support::thread_rng_n;

/// `UnionExec`: `UNION ALL` execution plan.
///
/// `UnionExec` combines multiple inputs with the same schema by
/// concatenating the partitions.  It does not mix or copy data within
/// or across partitions. Thus if the input partitions are sorted, the
/// output partitions of the union are also sorted.
///
/// For example, given a `UnionExec` of two inputs, with `N`
/// partitions, and `M` partitions, there will be `N+M` output
/// partitions. The first `N` output partitions are from Input 1
/// partitions, and then next `M` output partitions are from Input 2.
///
/// ```text
///                        ▲       ▲           ▲         ▲
///                        │       │           │         │
///      Output            │  ...  │           │         │
///    Partitions          │0      │N-1        │ N       │N+M-1
/// (passes through   ┌────┴───────┴───────────┴─────────┴───┐
///  the N+M input    │              UnionExec               │
///   partitions)     │                                      │
///                   └──────────────────────────────────────┘
////////////       Input           ┌────────┬─────┴────┬──────────┐
///     Partitions        │ ...    │          │     ...  │
///                    0  │        │ N-1      │ 0        │  M-1
///                  ┌────┴────────┴───┐  ┌───┴──────────┴───┐
///                  │                 │  │                  │
///                  │                 │  │                  │
///                  │                 │  │                  │
///                  │                 │  │                  │
///                  │                 │  │                  │
///                  │                 │  │                  │
///                  │Input 1          │  │Input 2           │
///                  └─────────────────┘  └──────────────────┘
/// ```
#[derive(Debug, Clone)]
pub struct UnionExec {
    /// Input execution plan
    inputs: Vec<Arc<dyn ExecutionPlan>>,
    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
    /// Cache holding plan properties like equivalences, output partitioning etc.
    cache: Arc<PlanProperties>,
}

impl UnionExec {
    /// Create a new UnionExec
    #[deprecated(since = "44.0.0", note = "Use UnionExec::try_new instead")]
    pub fn new(inputs: Vec<Arc<dyn ExecutionPlan>>) -> Self {
        let schema =
            union_schema(&inputs).expect("UnionExec::new called with empty inputs");
        // The schema of the inputs and the union schema is consistent when:
        // - They have the same number of fields, and
        // - Their fields have same types at the same indices.
        // Here, we know that schemas are consistent and the call below can
        // not return an error.
        let cache = Self::compute_properties(&inputs, schema).unwrap();
        UnionExec {
            inputs,
            metrics: ExecutionPlanMetricsSet::new(),
            cache: Arc::new(cache),
        }
    }

    /// Try to create a new UnionExec.
    ///
    /// # Errors
    /// Returns an error if:
    /// - `inputs` is empty
    ///
    /// # Optimization
    /// If there is only one input, returns that input directly rather than wrapping it in a UnionExec
    pub fn try_new(
        inputs: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        match inputs.len() {
            0 => exec_err!("UnionExec requires at least one input"),
            1 => Ok(inputs.into_iter().next().unwrap()),
            _ => {
                let schema = union_schema(&inputs)?;
                // The schema of the inputs and the union schema is consistent when:
                // - They have the same number of fields, and
                // - Their fields have same types at the same indices.
                // Here, we know that schemas are consistent and the call below can
                // not return an error.
                let cache = Self::compute_properties(&inputs, schema).unwrap();
                Ok(Arc::new(UnionExec {
                    inputs,
                    metrics: ExecutionPlanMetricsSet::new(),
                    cache: Arc::new(cache),
                }))
            }
        }
    }

    /// Get inputs of the execution plan
    pub fn inputs(&self) -> &Vec<Arc<dyn ExecutionPlan>> {
        &self.inputs
    }

    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
    fn compute_properties(
        inputs: &[Arc<dyn ExecutionPlan>],
        schema: SchemaRef,
    ) -> Result<PlanProperties> {
        // Calculate equivalence properties:
        let children_eqps = inputs
            .iter()
            .map(|child| child.equivalence_properties().clone())
            .collect::<Vec<_>>();
        let eq_properties = calculate_union(children_eqps, schema)?;

        // Calculate output partitioning; i.e. sum output partitions of the inputs.
        let num_partitions = inputs
            .iter()
            .map(|plan| plan.output_partitioning().partition_count())
            .sum();
        let output_partitioning = Partitioning::UnknownPartitioning(num_partitions);
        Ok(PlanProperties::new(
            eq_properties,
            output_partitioning,
            emission_type_from_children(inputs),
            boundedness_from_children(inputs),
        ))
    }

    fn with_new_children_and_same_properties(
        &self,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Self {
        Self {
            inputs: children,
            metrics: ExecutionPlanMetricsSet::new(),
            ..Self::clone(self)
        }
    }
}

impl DisplayAs for UnionExec {
    fn fmt_as(
        &self,
        t: DisplayFormatType,
        f: &mut std::fmt::Formatter,
    ) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(f, "UnionExec")
            }
            DisplayFormatType::TreeRender => Ok(()),
        }
    }
}

impl ExecutionPlan for UnionExec {
    fn name(&self) -> &'static str {
        "UnionExec"
    }

    /// Return a reference to Any that can be used for downcasting
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn properties(&self) -> &Arc<PlanProperties> {
        &self.cache
    }

    fn check_invariants(&self, check: InvariantLevel) -> Result<()> {
        check_default_invariants(self, check)?;

        (self.inputs().len() >= 2).then_some(()).ok_or_else(|| {
            internal_datafusion_err!("UnionExec should have at least 2 children")
        })
    }

    fn maintains_input_order(&self) -> Vec<bool> {
        // If the Union has an output ordering, it maintains at least one
        // child's ordering (i.e. the meet).
        // For instance, assume that the first child is SortExpr('a','b','c'),
        // the second child is SortExpr('a','b') and the third child is
        // SortExpr('a','b'). The output ordering would be SortExpr('a','b'),
        // which is the "meet" of all input orderings. In this example, this
        // function will return vec![false, true, true], indicating that we
        // preserve the orderings for the 2nd and the 3rd children.
        if let Some(output_ordering) = self.properties().output_ordering() {
            self.inputs()
                .iter()
                .map(|child| {
                    if let Some(child_ordering) = child.output_ordering() {
                        output_ordering.len() == child_ordering.len()
                    } else {
                        false
                    }
                })
                .collect()
        } else {
            vec![false; self.inputs().len()]
        }
    }

    fn benefits_from_input_partitioning(&self) -> Vec<bool> {
        vec![false; self.children().len()]
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        self.inputs.iter().collect()
    }

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        check_if_same_properties!(self, children);
        UnionExec::try_new(children)
    }

    fn execute(
        &self,
        mut partition: usize,
        context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        trace!(
            "Start UnionExec::execute for partition {} of context session_id {} and task_id {:?}",
            partition,
            context.session_id(),
            context.task_id()
        );
        let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
        // record the tiny amount of work done in this function so
        // elapsed_compute is reported as non zero
        let elapsed_compute = baseline_metrics.elapsed_compute().clone();
        let _timer = elapsed_compute.timer(); // record on drop

        // find partition to execute
        for input in self.inputs.iter() {
            // Calculate whether partition belongs to the current partition
            if partition < input.output_partitioning().partition_count() {
                let stream = input.execute(partition, context)?;
                debug!("Found a Union partition to execute");
                return Ok(Box::pin(ObservedStream::new(
                    stream,
                    baseline_metrics,
                    None,
                )));
            } else {
                partition -= input.output_partitioning().partition_count();
            }
        }

        warn!("Error in Union: Partition {partition} not found");

        exec_err!("Partition {partition} not found in Union")
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
        if let Some(partition_idx) = partition {
            // For a specific partition, find which input it belongs to
            let mut remaining_idx = partition_idx;
            for input in &self.inputs {
                let input_partition_count = input.output_partitioning().partition_count();
                if remaining_idx < input_partition_count {
                    // This partition belongs to this input
                    return input.partition_statistics(Some(remaining_idx));
                }
                remaining_idx -= input_partition_count;
            }
            // If we get here, the partition index is out of bounds
            Ok(Statistics::new_unknown(&self.schema()))
        } else {
            // Collect statistics from all inputs
            let stats = self
                .inputs
                .iter()
                .map(|input_exec| input_exec.partition_statistics(None))
                .collect::<Result<Vec<_>>>()?;

            Ok(stats
                .into_iter()
                .reduce(stats_union)
                .unwrap_or_else(|| Statistics::new_unknown(&self.schema())))
        }
    }

    fn supports_limit_pushdown(&self) -> bool {
        true
    }

    /// Tries to push `projection` down through `union`. If possible, performs the
    /// pushdown and returns a new [`UnionExec`] as the top plan which has projections
    /// as its children. Otherwise, returns `None`.
    fn try_swapping_with_projection(
        &self,
        projection: &ProjectionExec,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        // If the projection doesn't narrow the schema, we shouldn't try to push it down.
        if projection.expr().len() >= projection.input().schema().fields().len() {
            return Ok(None);
        }

        let new_children = self
            .children()
            .into_iter()
            .map(|child| make_with_child(projection, child))
            .collect::<Result<Vec<_>>>()?;

        Ok(Some(UnionExec::try_new(new_children.clone())?))
    }

    fn gather_filters_for_pushdown(
        &self,
        _phase: FilterPushdownPhase,
        parent_filters: Vec<Arc<dyn PhysicalExpr>>,
        _config: &ConfigOptions,
    ) -> Result<FilterDescription> {
        FilterDescription::from_children(parent_filters, &self.children())
    }

    fn handle_child_pushdown_result(
        &self,
        phase: FilterPushdownPhase,
        child_pushdown_result: ChildPushdownResult,
        _config: &ConfigOptions,
    ) -> Result<FilterPushdownPropagation<Arc<dyn ExecutionPlan>>> {
        // Pre phase: handle heterogeneous pushdown by wrapping individual
        // children with FilterExec and reporting all filters as handled.
        // Post phase: use default behavior to let the filter creator decide how to handle
        // filters that weren't fully pushed down.
        if phase != FilterPushdownPhase::Pre {
            return Ok(FilterPushdownPropagation::if_all(child_pushdown_result));
        }

        // UnionExec needs specialized filter pushdown handling when children have
        // heterogeneous pushdown support. Without this, when some children support
        // pushdown and others don't, the default behavior would leave FilterExec
        // above UnionExec, re-applying filters to outputs of all children—including
        // those that already applied the filters via pushdown. This specialized
        // implementation adds FilterExec only to children that don't support
        // pushdown, avoiding redundant filtering and improving performance.
        //
        // Example: Given Child1 (no pushdown support) and Child2 (has pushdown support)
        //   Default behavior:          This implementation:
        //   FilterExec                 UnionExec
        //     UnionExec                  FilterExec
        //       Child1                     Child1
        //       Child2(filter)           Child2(filter)

        // Collect unsupported filters for each child
        let mut unsupported_filters_per_child = vec![Vec::new(); self.inputs.len()];
        for parent_filter_result in child_pushdown_result.parent_filters.iter() {
            for (child_idx, &child_result) in
                parent_filter_result.child_results.iter().enumerate()
            {
                if matches!(child_result, PushedDown::No) {
                    unsupported_filters_per_child[child_idx]
                        .push(Arc::clone(&parent_filter_result.filter));
                }
            }
        }

        // Wrap children that have unsupported filters with FilterExec
        let mut new_children = self.inputs.clone();
        for (child_idx, unsupported_filters) in
            unsupported_filters_per_child.iter().enumerate()
        {
            if !unsupported_filters.is_empty() {
                let combined_filter = conjunction(unsupported_filters.clone());
                new_children[child_idx] = Arc::new(FilterExec::try_new(
                    combined_filter,
                    Arc::clone(&self.inputs[child_idx]),
                )?);
            }
        }

        // Check if any children were modified
        let children_modified = new_children
            .iter()
            .zip(self.inputs.iter())
            .any(|(new, old)| !Arc::ptr_eq(new, old));

        let all_filters_pushed =
            vec![PushedDown::Yes; child_pushdown_result.parent_filters.len()];
        let propagation = if children_modified {
            let updated_node = UnionExec::try_new(new_children)?;
            FilterPushdownPropagation::with_parent_pushdown_result(all_filters_pushed)
                .with_updated_node(updated_node)
        } else {
            FilterPushdownPropagation::with_parent_pushdown_result(all_filters_pushed)
        };

        // Report all parent filters as supported since we've ensured they're applied
        // on all children (either pushed down or via FilterExec)
        Ok(propagation)
    }
}

/// Combines multiple input streams by interleaving them.
///
/// This only works if all inputs have the same hash-partitioning.
///
/// # Data Flow
/// ```text
/// +---------+
/// |         |---+
/// | Input 1 |   |
/// |         |-------------+
/// +---------+   |         |
///               |         |         +---------+
///               +------------------>|         |
///                 +---------------->| Combine |-->
///                 | +-------------->|         |
///                 | |     |         +---------+
/// +---------+     | |     |
/// |         |-----+ |     |
/// | Input 2 |       |     |
/// |         |---------------+
/// +---------+       |     | |
///                   |     | |       +---------+
///                   |     +-------->|         |
///                   |       +------>| Combine |-->
///                   |         +---->|         |
///                   |         |     +---------+
/// +---------+       |         |
/// |         |-------+         |
/// | Input 3 |                 |
/// |         |-----------------+
/// +---------+
/// ```
#[derive(Debug, Clone)]
pub struct InterleaveExec {
    /// Input execution plan
    inputs: Vec<Arc<dyn ExecutionPlan>>,
    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
    /// Cache holding plan properties like equivalences, output partitioning etc.
    cache: Arc<PlanProperties>,
}

impl InterleaveExec {
    /// Create a new InterleaveExec
    pub fn try_new(inputs: Vec<Arc<dyn ExecutionPlan>>) -> Result<Self> {
        assert_or_internal_err!(
            can_interleave(inputs.iter()),
            "Not all InterleaveExec children have a consistent hash partitioning"
        );
        let cache = Self::compute_properties(&inputs)?;
        Ok(InterleaveExec {
            inputs,
            metrics: ExecutionPlanMetricsSet::new(),
            cache: Arc::new(cache),
        })
    }

    /// Get inputs of the execution plan
    pub fn inputs(&self) -> &Vec<Arc<dyn ExecutionPlan>> {
        &self.inputs
    }

    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
    fn compute_properties(inputs: &[Arc<dyn ExecutionPlan>]) -> Result<PlanProperties> {
        let schema = union_schema(inputs)?;
        let eq_properties = EquivalenceProperties::new(schema);
        // Get output partitioning:
        let output_partitioning = inputs[0].output_partitioning().clone();
        Ok(PlanProperties::new(
            eq_properties,
            output_partitioning,
            emission_type_from_children(inputs),
            boundedness_from_children(inputs),
        ))
    }

    fn with_new_children_and_same_properties(
        &self,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Self {
        Self {
            inputs: children,
            metrics: ExecutionPlanMetricsSet::new(),
            ..Self::clone(self)
        }
    }
}

impl DisplayAs for InterleaveExec {
    fn fmt_as(
        &self,
        t: DisplayFormatType,
        f: &mut std::fmt::Formatter,
    ) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(f, "InterleaveExec")
            }
            DisplayFormatType::TreeRender => Ok(()),
        }
    }
}

impl ExecutionPlan for InterleaveExec {
    fn name(&self) -> &'static str {
        "InterleaveExec"
    }

    /// Return a reference to Any that can be used for downcasting
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn properties(&self) -> &Arc<PlanProperties> {
        &self.cache
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        self.inputs.iter().collect()
    }

    fn maintains_input_order(&self) -> Vec<bool> {
        vec![false; self.inputs().len()]
    }

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        // New children are no longer interleavable, which might be a bug of optimization rewrite.
        assert_or_internal_err!(
            can_interleave(children.iter()),
            "Can not create InterleaveExec: new children can not be interleaved"
        );
        check_if_same_properties!(self, children);
        Ok(Arc::new(InterleaveExec::try_new(children)?))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        trace!(
            "Start InterleaveExec::execute for partition {} of context session_id {} and task_id {:?}",
            partition,
            context.session_id(),
            context.task_id()
        );
        let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
        // record the tiny amount of work done in this function so
        // elapsed_compute is reported as non zero
        let elapsed_compute = baseline_metrics.elapsed_compute().clone();
        let _timer = elapsed_compute.timer(); // record on drop

        let mut input_stream_vec = vec![];
        for input in self.inputs.iter() {
            if partition < input.output_partitioning().partition_count() {
                input_stream_vec.push(input.execute(partition, Arc::clone(&context))?);
            } else {
                // Do not find a partition to execute
                break;
            }
        }
        if input_stream_vec.len() == self.inputs.len() {
            let stream = Box::pin(CombinedRecordBatchStream::new(
                self.schema(),
                input_stream_vec,
            ));
            return Ok(Box::pin(ObservedStream::new(
                stream,
                baseline_metrics,
                None,
            )));
        }

        warn!("Error in InterleaveExec: Partition {partition} not found");

        exec_err!("Partition {partition} not found in InterleaveExec")
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
        let stats = self
            .inputs
            .iter()
            .map(|stat| stat.partition_statistics(partition))
            .collect::<Result<Vec<_>>>()?;

        Ok(stats
            .into_iter()
            .reduce(stats_union)
            .unwrap_or_else(|| Statistics::new_unknown(&self.schema())))
    }

    fn benefits_from_input_partitioning(&self) -> Vec<bool> {
        vec![false; self.children().len()]
    }
}

/// If all the input partitions have the same Hash partition spec with the first_input_partition
/// The InterleaveExec is partition aware.
///
/// It might be too strict here in the case that the input partition specs are compatible but not exactly the same.
/// For example one input partition has the partition spec Hash('a','b','c') and
/// other has the partition spec Hash('a'), It is safe to derive the out partition with the spec Hash('a','b','c').
pub fn can_interleave<T: Borrow<Arc<dyn ExecutionPlan>>>(
    mut inputs: impl Iterator<Item = T>,
) -> bool {
    let Some(first) = inputs.next() else {
        return false;
    };

    let reference = first.borrow().output_partitioning();
    matches!(reference, Partitioning::Hash(_, _))
        && inputs
            .map(|plan| plan.borrow().output_partitioning().clone())
            .all(|partition| partition == *reference)
}

fn union_schema(inputs: &[Arc<dyn ExecutionPlan>]) -> Result<SchemaRef> {
    if inputs.is_empty() {
        return exec_err!("Cannot create union schema from empty inputs");
    }

    let first_schema = inputs[0].schema();
    let first_field_count = first_schema.fields().len();

    // validate that all inputs have the same number of fields
    for (idx, input) in inputs.iter().enumerate().skip(1) {
        let field_count = input.schema().fields().len();
        if field_count != first_field_count {
            return exec_err!(
                "UnionExec/InterleaveExec requires all inputs to have the same number of fields. \
                 Input 0 has {first_field_count} fields, but input {idx} has {field_count} fields"
            );
        }
    }

    let fields = (0..first_field_count)
        .map(|i| {
            // We take the name from the left side of the union to match how names are coerced during logical planning,
            // which also uses the left side names.
            let base_field = first_schema.field(i).clone();

            // Coerce metadata and nullability across all inputs

            inputs
                .iter()
                .enumerate()
                .map(|(input_idx, input)| {
                    let field = input.schema().field(i).clone();
                    let mut metadata = field.metadata().clone();

                    let other_metadatas = inputs
                        .iter()
                        .enumerate()
                        .filter(|(other_idx, _)| *other_idx != input_idx)
                        .flat_map(|(_, other_input)| {
                            other_input.schema().field(i).metadata().clone().into_iter()
                        });

                    metadata.extend(other_metadatas);
                    field.with_metadata(metadata)
                })
                .find_or_first(Field::is_nullable)
                // We can unwrap this because if inputs was empty, this would've already panic'ed when we
                // indexed into inputs[0].
                .unwrap()
                .with_name(base_field.name())
        })
        .collect::<Vec<_>>();

    let all_metadata_merged = inputs
        .iter()
        .flat_map(|i| i.schema().metadata().clone().into_iter())
        .collect();

    Ok(Arc::new(Schema::new_with_metadata(
        fields,
        all_metadata_merged,
    )))
}

/// CombinedRecordBatchStream can be used to combine a Vec of SendableRecordBatchStreams into one
struct CombinedRecordBatchStream {
    /// Schema wrapped by Arc
    schema: SchemaRef,
    /// Stream entries
    entries: Vec<SendableRecordBatchStream>,
}

impl CombinedRecordBatchStream {
    /// Create an CombinedRecordBatchStream
    pub fn new(schema: SchemaRef, entries: Vec<SendableRecordBatchStream>) -> Self {
        Self { schema, entries }
    }
}

impl RecordBatchStream for CombinedRecordBatchStream {
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }
}

impl Stream for CombinedRecordBatchStream {
    type Item = Result<RecordBatch>;

    fn poll_next(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<Option<Self::Item>> {
        use Poll::*;

        let start = thread_rng_n(self.entries.len() as u32) as usize;
        let mut idx = start;

        for _ in 0..self.entries.len() {
            let stream = self.entries.get_mut(idx).unwrap();

            match Pin::new(stream).poll_next(cx) {
                Ready(Some(val)) => return Ready(Some(val)),
                Ready(None) => {
                    // Remove the entry
                    self.entries.swap_remove(idx);

                    // Check if this was the last entry, if so the cursor needs
                    // to wrap
                    if idx == self.entries.len() {
                        idx = 0;
                    } else if idx < start && start <= self.entries.len() {
                        // The stream being swapped into the current index has
                        // already been polled, so skip it.
                        idx = idx.wrapping_add(1) % self.entries.len();
                    }
                }
                Pending => {
                    idx = idx.wrapping_add(1) % self.entries.len();
                }
            }
        }

        // If the map is empty, then the stream is complete.
        if self.entries.is_empty() {
            Ready(None)
        } else {
            Pending
        }
    }
}

fn col_stats_union(
    mut left: ColumnStatistics,
    right: &ColumnStatistics,
) -> ColumnStatistics {
    left.distinct_count = Precision::Absent;
    left.min_value = left.min_value.min(&right.min_value);
    left.max_value = left.max_value.max(&right.max_value);
    left.sum_value = left.sum_value.add(&right.sum_value);
    left.null_count = left.null_count.add(&right.null_count);

    left
}

fn stats_union(mut left: Statistics, right: Statistics) -> Statistics {
    let Statistics {
        num_rows: right_num_rows,
        total_byte_size: right_total_bytes,
        column_statistics: right_column_statistics,
        ..
    } = right;
    left.num_rows = left.num_rows.add(&right_num_rows);
    left.total_byte_size = left.total_byte_size.add(&right_total_bytes);
    left.column_statistics = left
        .column_statistics
        .into_iter()
        .zip(right_column_statistics.iter())
        .map(|(a, b)| col_stats_union(a, b))
        .collect::<Vec<_>>();
    left
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::collect;
    use crate::test::{self, TestMemoryExec};

    use arrow::compute::SortOptions;
    use arrow::datatypes::DataType;
    use datafusion_common::ScalarValue;
    use datafusion_physical_expr::equivalence::convert_to_orderings;
    use datafusion_physical_expr::expressions::col;

    // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g)
    fn create_test_schema() -> Result<SchemaRef> {
        let a = Field::new("a", DataType::Int32, true);
        let b = Field::new("b", DataType::Int32, true);
        let c = Field::new("c", DataType::Int32, true);
        let d = Field::new("d", DataType::Int32, true);
        let e = Field::new("e", DataType::Int32, true);
        let f = Field::new("f", DataType::Int32, true);
        let g = Field::new("g", DataType::Int32, true);
        let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g]));

        Ok(schema)
    }

    fn create_test_schema2() -> Result<SchemaRef> {
        let a = Field::new("a", DataType::Int32, true);
        let b = Field::new("b", DataType::Int32, true);
        let c = Field::new("c", DataType::Int32, true);
        let d = Field::new("d", DataType::Int32, true);
        let e = Field::new("e", DataType::Int32, true);
        let f = Field::new("f", DataType::Int32, true);
        let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f]));

        Ok(schema)
    }

    #[tokio::test]
    async fn test_union_partitions() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());

        // Create inputs with different partitioning
        let csv = test::scan_partitioned(4);
        let csv2 = test::scan_partitioned(5);

        let union_exec: Arc<dyn ExecutionPlan> = UnionExec::try_new(vec![csv, csv2])?;

        // Should have 9 partitions and 9 output batches
        assert_eq!(
            union_exec
                .properties()
                .output_partitioning()
                .partition_count(),
            9
        );

        let result: Vec<RecordBatch> = collect(union_exec, task_ctx).await?;
        assert_eq!(result.len(), 9);

        Ok(())
    }

    #[tokio::test]
    async fn test_stats_union() {
        let left = Statistics {
            num_rows: Precision::Exact(5),
            total_byte_size: Precision::Exact(23),
            column_statistics: vec![
                ColumnStatistics {
                    distinct_count: Precision::Exact(5),
                    max_value: Precision::Exact(ScalarValue::Int64(Some(21))),
                    min_value: Precision::Exact(ScalarValue::Int64(Some(-4))),
                    sum_value: Precision::Exact(ScalarValue::Int64(Some(42))),
                    null_count: Precision::Exact(0),
                    byte_size: Precision::Absent,
                },
                ColumnStatistics {
                    distinct_count: Precision::Exact(1),
                    max_value: Precision::Exact(ScalarValue::from("x")),
                    min_value: Precision::Exact(ScalarValue::from("a")),
                    sum_value: Precision::Absent,
                    null_count: Precision::Exact(3),
                    byte_size: Precision::Absent,
                },
                ColumnStatistics {
                    distinct_count: Precision::Absent,
                    max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))),
                    min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))),
                    sum_value: Precision::Exact(ScalarValue::Float32(Some(42.0))),
                    null_count: Precision::Absent,
                    byte_size: Precision::Absent,
                },
            ],
        };

        let right = Statistics {
            num_rows: Precision::Exact(7),
            total_byte_size: Precision::Exact(29),
            column_statistics: vec![
                ColumnStatistics {
                    distinct_count: Precision::Exact(3),
                    max_value: Precision::Exact(ScalarValue::Int64(Some(34))),
                    min_value: Precision::Exact(ScalarValue::Int64(Some(1))),
                    sum_value: Precision::Exact(ScalarValue::Int64(Some(42))),
                    null_count: Precision::Exact(1),
                    byte_size: Precision::Absent,
                },
                ColumnStatistics {
                    distinct_count: Precision::Absent,
                    max_value: Precision::Exact(ScalarValue::from("c")),
                    min_value: Precision::Exact(ScalarValue::from("b")),
                    sum_value: Precision::Absent,
                    null_count: Precision::Absent,
                    byte_size: Precision::Absent,
                },
                ColumnStatistics {
                    distinct_count: Precision::Absent,
                    max_value: Precision::Absent,
                    min_value: Precision::Absent,
                    sum_value: Precision::Absent,
                    null_count: Precision::Absent,
                    byte_size: Precision::Absent,
                },
            ],
        };

        let result = stats_union(left, right);
        let expected = Statistics {
            num_rows: Precision::Exact(12),
            total_byte_size: Precision::Exact(52),
            column_statistics: vec![
                ColumnStatistics {
                    distinct_count: Precision::Absent,
                    max_value: Precision::Exact(ScalarValue::Int64(Some(34))),
                    min_value: Precision::Exact(ScalarValue::Int64(Some(-4))),
                    sum_value: Precision::Exact(ScalarValue::Int64(Some(84))),
                    null_count: Precision::Exact(1),
                    byte_size: Precision::Absent,
                },
                ColumnStatistics {
                    distinct_count: Precision::Absent,
                    max_value: Precision::Exact(ScalarValue::from("x")),
                    min_value: Precision::Exact(ScalarValue::from("a")),
                    sum_value: Precision::Absent,
                    null_count: Precision::Absent,
                    byte_size: Precision::Absent,
                },
                ColumnStatistics {
                    distinct_count: Precision::Absent,
                    max_value: Precision::Absent,
                    min_value: Precision::Absent,
                    sum_value: Precision::Absent,
                    null_count: Precision::Absent,
                    byte_size: Precision::Absent,
                },
            ],
        };

        assert_eq!(result, expected);
    }

    #[tokio::test]
    async fn test_union_equivalence_properties() -> Result<()> {
        let schema = create_test_schema()?;
        let col_a = &col("a", &schema)?;
        let col_b = &col("b", &schema)?;
        let col_c = &col("c", &schema)?;
        let col_d = &col("d", &schema)?;
        let col_e = &col("e", &schema)?;
        let col_f = &col("f", &schema)?;
        let options = SortOptions::default();
        let test_cases = [
            //-----------TEST CASE 1----------//
            (
                // First child orderings
                vec![
                    // [a ASC, b ASC, f ASC]
                    vec![(col_a, options), (col_b, options), (col_f, options)],
                ],
                // Second child orderings
                vec![
                    // [a ASC, b ASC, c ASC]
                    vec![(col_a, options), (col_b, options), (col_c, options)],
                    // [a ASC, b ASC, f ASC]
                    vec![(col_a, options), (col_b, options), (col_f, options)],
                ],
                // Union output orderings
                vec![
                    // [a ASC, b ASC, f ASC]
                    vec![(col_a, options), (col_b, options), (col_f, options)],
                ],
            ),
            //-----------TEST CASE 2----------//
            (
                // First child orderings
                vec![
                    // [a ASC, b ASC, f ASC]
                    vec![(col_a, options), (col_b, options), (col_f, options)],
                    // d ASC
                    vec![(col_d, options)],
                ],
                // Second child orderings
                vec![
                    // [a ASC, b ASC, c ASC]
                    vec![(col_a, options), (col_b, options), (col_c, options)],
                    // [e ASC]
                    vec![(col_e, options)],
                ],
                // Union output orderings
                vec![
                    // [a ASC, b ASC]
                    vec![(col_a, options), (col_b, options)],
                ],
            ),
        ];

        for (
            test_idx,
            (first_child_orderings, second_child_orderings, union_orderings),
        ) in test_cases.iter().enumerate()
        {
            let first_orderings = convert_to_orderings(first_child_orderings);
            let second_orderings = convert_to_orderings(second_child_orderings);
            let union_expected_orderings = convert_to_orderings(union_orderings);
            let child1_exec = TestMemoryExec::try_new(&[], Arc::clone(&schema), None)?
                .try_with_sort_information(first_orderings)?;
            let child1 = Arc::new(child1_exec);
            let child1 = Arc::new(TestMemoryExec::update_cache(&child1));
            let child2_exec = TestMemoryExec::try_new(&[], Arc::clone(&schema), None)?
                .try_with_sort_information(second_orderings)?;
            let child2 = Arc::new(child2_exec);
            let child2 = Arc::new(TestMemoryExec::update_cache(&child2));

            let mut union_expected_eq = EquivalenceProperties::new(Arc::clone(&schema));
            union_expected_eq.add_orderings(union_expected_orderings);

            let union: Arc<dyn ExecutionPlan> = UnionExec::try_new(vec![child1, child2])?;
            let union_eq_properties = union.properties().equivalence_properties();
            let err_msg = format!(
                "Error in test id: {:?}, test case: {:?}",
                test_idx, test_cases[test_idx]
            );
            assert_eq_properties_same(union_eq_properties, &union_expected_eq, err_msg);
        }
        Ok(())
    }

    fn assert_eq_properties_same(
        lhs: &EquivalenceProperties,
        rhs: &EquivalenceProperties,
        err_msg: String,
    ) {
        // Check whether orderings are same.
        let lhs_orderings = lhs.oeq_class();
        let rhs_orderings = rhs.oeq_class();
        assert_eq!(lhs_orderings.len(), rhs_orderings.len(), "{err_msg}");
        for rhs_ordering in rhs_orderings.iter() {
            assert!(lhs_orderings.contains(rhs_ordering), "{}", err_msg);
        }
    }

    #[test]
    fn test_union_empty_inputs() {
        // Test that UnionExec::try_new fails with empty inputs
        let result = UnionExec::try_new(vec![]);
        assert!(
            result
                .unwrap_err()
                .to_string()
                .contains("UnionExec requires at least one input")
        );
    }

    #[test]
    fn test_union_schema_empty_inputs() {
        // Test that union_schema fails with empty inputs
        let result = union_schema(&[]);
        assert!(
            result
                .unwrap_err()
                .to_string()
                .contains("Cannot create union schema from empty inputs")
        );
    }

    #[test]
    fn test_union_single_input() -> Result<()> {
        // Test that UnionExec::try_new returns the single input directly
        let schema = create_test_schema()?;
        let memory_exec: Arc<dyn ExecutionPlan> =
            Arc::new(TestMemoryExec::try_new(&[], Arc::clone(&schema), None)?);
        let memory_exec_clone = Arc::clone(&memory_exec);
        let result = UnionExec::try_new(vec![memory_exec])?;

        // Check that the result is the same as the input (no UnionExec wrapper)
        assert_eq!(result.schema(), schema);
        // Verify it's the same execution plan
        assert!(Arc::ptr_eq(&result, &memory_exec_clone));

        Ok(())
    }

    #[test]
    fn test_union_schema_multiple_inputs() -> Result<()> {
        // Test that existing functionality with multiple inputs still works
        let schema = create_test_schema()?;
        let memory_exec1 =
            Arc::new(TestMemoryExec::try_new(&[], Arc::clone(&schema), None)?);
        let memory_exec2 =
            Arc::new(TestMemoryExec::try_new(&[], Arc::clone(&schema), None)?);

        let union_plan = UnionExec::try_new(vec![memory_exec1, memory_exec2])?;

        // Downcast to verify it's a UnionExec
        let union = union_plan
            .as_any()
            .downcast_ref::<UnionExec>()
            .expect("Expected UnionExec");

        // Check that schema is correct
        assert_eq!(union.schema(), schema);
        // Check that we have 2 inputs
        assert_eq!(union.inputs().len(), 2);

        Ok(())
    }

    #[test]
    fn test_union_schema_mismatch() {
        // Test that UnionExec properly rejects inputs with different field counts
        let schema = create_test_schema().unwrap();
        let schema2 = create_test_schema2().unwrap();
        let memory_exec1 =
            Arc::new(TestMemoryExec::try_new(&[], Arc::clone(&schema), None).unwrap());
        let memory_exec2 =
            Arc::new(TestMemoryExec::try_new(&[], Arc::clone(&schema2), None).unwrap());

        let result = UnionExec::try_new(vec![memory_exec1, memory_exec2]);
        assert!(result.is_err());
        assert!(
            result.unwrap_err().to_string().contains(
                "UnionExec/InterleaveExec requires all inputs to have the same number of fields"
            )
        );
    }
}