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
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
// 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;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll, ready};

use datafusion_physical_expr::projection::{ProjectionRef, combine_projections};
use itertools::Itertools;

use super::{
    ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties,
    RecordBatchStream, SendableRecordBatchStream, Statistics,
};
use crate::check_if_same_properties;
use crate::coalesce::{LimitedBatchCoalescer, PushBatchStatus};
use crate::common::can_project;
use crate::execution_plan::CardinalityEffect;
use crate::filter_pushdown::{
    ChildFilterDescription, ChildPushdownResult, FilterDescription, FilterPushdownPhase,
    FilterPushdownPropagation, PushedDown,
};
use crate::metrics::{MetricBuilder, MetricType};
use crate::projection::{
    EmbeddedProjection, ProjectionExec, ProjectionExpr, make_with_child,
    try_embed_projection, update_expr,
};
use crate::{
    DisplayFormatType, ExecutionPlan,
    metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, RatioMetrics},
};

use arrow::compute::filter_record_batch;
use arrow::datatypes::{DataType, SchemaRef};
use arrow::record_batch::RecordBatch;
use datafusion_common::cast::as_boolean_array;
use datafusion_common::config::ConfigOptions;
use datafusion_common::stats::Precision;
use datafusion_common::{
    DataFusionError, Result, ScalarValue, internal_err, plan_err, project_schema,
};
use datafusion_execution::TaskContext;
use datafusion_expr::Operator;
use datafusion_physical_expr::equivalence::ProjectionMapping;
use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal, lit};
use datafusion_physical_expr::intervals::utils::check_support;
use datafusion_physical_expr::utils::{collect_columns, reassign_expr_columns};
use datafusion_physical_expr::{
    AcrossPartitions, AnalysisContext, ConstExpr, EquivalenceProperties, ExprBoundaries,
    PhysicalExpr, analyze, conjunction, split_conjunction,
};

use datafusion_physical_expr_common::physical_expr::fmt_sql;
use futures::stream::{Stream, StreamExt};
use log::trace;

const FILTER_EXEC_DEFAULT_SELECTIVITY: u8 = 20;
const FILTER_EXEC_DEFAULT_BATCH_SIZE: usize = 8192;

/// FilterExec evaluates a boolean predicate against all input batches to determine which rows to
/// include in its output batches.
#[derive(Debug, Clone)]
pub struct FilterExec {
    /// The expression to filter on. This expression must evaluate to a boolean value.
    predicate: Arc<dyn PhysicalExpr>,
    /// The input plan
    input: Arc<dyn ExecutionPlan>,
    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
    /// Selectivity for statistics. 0 = no rows, 100 = all rows
    default_selectivity: u8,
    /// Properties equivalence properties, partitioning, etc.
    cache: Arc<PlanProperties>,
    /// The projection indices of the columns in the output schema of join
    projection: Option<ProjectionRef>,
    /// Target batch size for output batches
    batch_size: usize,
    /// Number of rows to fetch
    fetch: Option<usize>,
}

/// Builder for [`FilterExec`] to set optional parameters
pub struct FilterExecBuilder {
    predicate: Arc<dyn PhysicalExpr>,
    input: Arc<dyn ExecutionPlan>,
    projection: Option<ProjectionRef>,
    default_selectivity: u8,
    batch_size: usize,
    fetch: Option<usize>,
}

impl FilterExecBuilder {
    /// Create a new builder with required parameters (predicate and input)
    pub fn new(predicate: Arc<dyn PhysicalExpr>, input: Arc<dyn ExecutionPlan>) -> Self {
        Self {
            predicate,
            input,
            projection: None,
            default_selectivity: FILTER_EXEC_DEFAULT_SELECTIVITY,
            batch_size: FILTER_EXEC_DEFAULT_BATCH_SIZE,
            fetch: None,
        }
    }

    /// Set the input execution plan
    pub fn with_input(mut self, input: Arc<dyn ExecutionPlan>) -> Self {
        self.input = input;
        self
    }

    /// Set the predicate expression
    pub fn with_predicate(mut self, predicate: Arc<dyn PhysicalExpr>) -> Self {
        self.predicate = predicate;
        self
    }

    /// Set the projection, composing with any existing projection.
    ///
    /// If a projection is already set, the new projection indices are mapped
    /// through the existing projection. For example, if the current projection
    /// is `[0, 2, 3]` and `apply_projection(Some(vec![0, 2]))` is called, the
    /// resulting projection will be `[0, 3]` (indices 0 and 2 of `[0, 2, 3]`).
    ///
    /// If no projection is currently set, the new projection is used directly.
    /// If `None` is passed, the projection is cleared.
    pub fn apply_projection(self, projection: Option<Vec<usize>>) -> Result<Self> {
        let projection = projection.map(Into::into);
        self.apply_projection_by_ref(projection.as_ref())
    }

    /// The same as [`Self::apply_projection`] but takes projection shared reference.
    pub fn apply_projection_by_ref(
        mut self,
        projection: Option<&ProjectionRef>,
    ) -> Result<Self> {
        // Check if the projection is valid against current output schema
        can_project(&self.input.schema(), projection.map(AsRef::as_ref))?;
        self.projection = combine_projections(projection, self.projection.as_ref())?;
        Ok(self)
    }

    /// Set the default selectivity
    pub fn with_default_selectivity(mut self, default_selectivity: u8) -> Self {
        self.default_selectivity = default_selectivity;
        self
    }

    /// Set the batch size
    pub fn with_batch_size(mut self, batch_size: usize) -> Self {
        self.batch_size = batch_size;
        self
    }

    /// Set the fetch limit
    pub fn with_fetch(mut self, fetch: Option<usize>) -> Self {
        self.fetch = fetch;
        self
    }

    /// Build the FilterExec, computing properties once with all configured parameters
    pub fn build(self) -> Result<FilterExec> {
        // Validate predicate type
        match self.predicate.data_type(self.input.schema().as_ref())? {
            DataType::Boolean => {}
            other => {
                return plan_err!(
                    "Filter predicate must return BOOLEAN values, got {other:?}"
                );
            }
        }

        // Validate selectivity
        if self.default_selectivity > 100 {
            return plan_err!(
                "Default filter selectivity value needs to be less than or equal to 100"
            );
        }

        // Validate projection if provided
        can_project(&self.input.schema(), self.projection.as_deref())?;

        // Compute properties once with all parameters
        let cache = FilterExec::compute_properties(
            &self.input,
            &self.predicate,
            self.default_selectivity,
            self.projection.as_deref(),
        )?;

        Ok(FilterExec {
            predicate: self.predicate,
            input: self.input,
            metrics: ExecutionPlanMetricsSet::new(),
            default_selectivity: self.default_selectivity,
            cache: Arc::new(cache),
            projection: self.projection,
            batch_size: self.batch_size,
            fetch: self.fetch,
        })
    }
}

impl From<&FilterExec> for FilterExecBuilder {
    fn from(exec: &FilterExec) -> Self {
        Self {
            predicate: Arc::clone(&exec.predicate),
            input: Arc::clone(&exec.input),
            projection: exec.projection.clone(),
            default_selectivity: exec.default_selectivity,
            batch_size: exec.batch_size,
            fetch: exec.fetch,
            // We could cache / copy over PlanProperties
            // here but that would require invalidating them in FilterExecBuilder::apply_projection, etc.
            // and currently every call to this method ends up invalidating them anyway.
            // If useful this can be added in the future as a non-breaking change.
        }
    }
}

impl FilterExec {
    /// Create a FilterExec on an input using the builder pattern
    pub fn try_new(
        predicate: Arc<dyn PhysicalExpr>,
        input: Arc<dyn ExecutionPlan>,
    ) -> Result<Self> {
        FilterExecBuilder::new(predicate, input).build()
    }

    /// Get a batch size
    pub fn batch_size(&self) -> usize {
        self.batch_size
    }

    /// Set the default selectivity
    pub fn with_default_selectivity(
        mut self,
        default_selectivity: u8,
    ) -> Result<Self, DataFusionError> {
        if default_selectivity > 100 {
            return plan_err!(
                "Default filter selectivity value needs to be less than or equal to 100"
            );
        }
        self.default_selectivity = default_selectivity;
        Ok(self)
    }

    /// Return new instance of [FilterExec] with the given projection.
    ///
    /// # Deprecated
    /// Use [`FilterExecBuilder::apply_projection`] instead
    #[deprecated(
        since = "52.0.0",
        note = "Use FilterExecBuilder::apply_projection instead"
    )]
    pub fn with_projection(&self, projection: Option<Vec<usize>>) -> Result<Self> {
        let builder = FilterExecBuilder::from(self);
        builder.apply_projection(projection)?.build()
    }

    /// Set the batch size
    pub fn with_batch_size(&self, batch_size: usize) -> Result<Self> {
        Ok(Self {
            predicate: Arc::clone(&self.predicate),
            input: Arc::clone(&self.input),
            metrics: self.metrics.clone(),
            default_selectivity: self.default_selectivity,
            cache: Arc::clone(&self.cache),
            projection: self.projection.clone(),
            batch_size,
            fetch: self.fetch,
        })
    }

    /// The expression to filter on. This expression must evaluate to a boolean value.
    pub fn predicate(&self) -> &Arc<dyn PhysicalExpr> {
        &self.predicate
    }

    /// The input plan
    pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
        &self.input
    }

    /// The default selectivity
    pub fn default_selectivity(&self) -> u8 {
        self.default_selectivity
    }

    /// Projection
    pub fn projection(&self) -> &Option<ProjectionRef> {
        &self.projection
    }

    /// Calculates `Statistics` for `FilterExec`, by applying selectivity (either default, or estimated) to input statistics.
    fn statistics_helper(
        schema: &SchemaRef,
        input_stats: Statistics,
        predicate: &Arc<dyn PhysicalExpr>,
        default_selectivity: u8,
    ) -> Result<Statistics> {
        if !check_support(predicate, schema) {
            let selectivity = default_selectivity as f64 / 100.0;
            let mut stats = input_stats.to_inexact();
            stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity);
            stats.total_byte_size = stats
                .total_byte_size
                .with_estimated_selectivity(selectivity);
            return Ok(stats);
        }

        let num_rows = input_stats.num_rows;
        let total_byte_size = input_stats.total_byte_size;
        let input_analysis_ctx =
            AnalysisContext::try_from_statistics(schema, &input_stats.column_statistics)?;

        let analysis_ctx = analyze(predicate, input_analysis_ctx, schema)?;

        // Estimate (inexact) selectivity of predicate
        let selectivity = analysis_ctx.selectivity.unwrap_or(1.0);
        let num_rows = num_rows.with_estimated_selectivity(selectivity);
        let total_byte_size = total_byte_size.with_estimated_selectivity(selectivity);

        let column_statistics = collect_new_statistics(
            schema,
            &input_stats.column_statistics,
            analysis_ctx.boundaries,
        );
        Ok(Statistics {
            num_rows,
            total_byte_size,
            column_statistics,
        })
    }

    /// Returns the `AcrossPartitions` value for `expr` if it is constant:
    /// either already known constant in `input_eqs`, or a `Literal`
    /// (which is inherently constant across all partitions).
    fn expr_constant_or_literal(
        expr: &Arc<dyn PhysicalExpr>,
        input_eqs: &EquivalenceProperties,
    ) -> Option<AcrossPartitions> {
        input_eqs.is_expr_constant(expr).or_else(|| {
            expr.as_any()
                .downcast_ref::<Literal>()
                .map(|l| AcrossPartitions::Uniform(Some(l.value().clone())))
        })
    }

    fn extend_constants(
        input: &Arc<dyn ExecutionPlan>,
        predicate: &Arc<dyn PhysicalExpr>,
    ) -> Vec<ConstExpr> {
        let mut res_constants = Vec::new();
        let input_eqs = input.equivalence_properties();

        let conjunctions = split_conjunction(predicate);
        for conjunction in conjunctions {
            if let Some(binary) = conjunction.as_any().downcast_ref::<BinaryExpr>()
                && binary.op() == &Operator::Eq
            {
                // Check if either side is constant — either already known
                // constant from the input equivalence properties, or a literal
                // value (which is inherently constant across all partitions).
                let left_const = Self::expr_constant_or_literal(binary.left(), input_eqs);
                let right_const =
                    Self::expr_constant_or_literal(binary.right(), input_eqs);

                if let Some(left_across) = left_const {
                    // LEFT is constant, so RIGHT must also be constant.
                    // Use RIGHT's known across value if available, otherwise
                    // propagate LEFT's (e.g. Uniform from a literal).
                    let across = right_const.unwrap_or(left_across);
                    res_constants
                        .push(ConstExpr::new(Arc::clone(binary.right()), across));
                } else if let Some(right_across) = right_const {
                    // RIGHT is constant, so LEFT must also be constant.
                    res_constants
                        .push(ConstExpr::new(Arc::clone(binary.left()), right_across));
                }
            }
        }
        res_constants
    }
    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
    fn compute_properties(
        input: &Arc<dyn ExecutionPlan>,
        predicate: &Arc<dyn PhysicalExpr>,
        default_selectivity: u8,
        projection: Option<&[usize]>,
    ) -> Result<PlanProperties> {
        // Combine the equal predicates with the input equivalence properties
        // to construct the equivalence properties:
        let schema = input.schema();
        let stats = Self::statistics_helper(
            &schema,
            input.partition_statistics(None)?,
            predicate,
            default_selectivity,
        )?;
        let mut eq_properties = input.equivalence_properties().clone();
        let (equal_pairs, _) = collect_columns_from_predicate_inner(predicate);
        for (lhs, rhs) in equal_pairs {
            eq_properties.add_equal_conditions(Arc::clone(lhs), Arc::clone(rhs))?
        }
        // Add the columns that have only one viable value (singleton) after
        // filtering to constants.
        let constants = collect_columns(predicate)
            .into_iter()
            .filter(|column| stats.column_statistics[column.index()].is_singleton())
            .map(|column| {
                let value = stats.column_statistics[column.index()]
                    .min_value
                    .get_value();
                let expr = Arc::new(column) as _;
                ConstExpr::new(expr, AcrossPartitions::Uniform(value.cloned()))
            });
        // This is for statistics
        eq_properties.add_constants(constants)?;
        // This is for logical constant (for example: a = '1', then a could be marked as a constant)
        // to do: how to deal with multiple situation to represent = (for example c1 between 0 and 0)
        eq_properties.add_constants(Self::extend_constants(input, predicate))?;

        let mut output_partitioning = input.output_partitioning().clone();
        // If contains projection, update the PlanProperties.
        if let Some(projection) = projection {
            let schema = eq_properties.schema();
            let projection_mapping = ProjectionMapping::from_indices(projection, schema)?;
            let out_schema = project_schema(schema, Some(&projection))?;
            output_partitioning =
                output_partitioning.project(&projection_mapping, &eq_properties);
            eq_properties = eq_properties.project(&projection_mapping, out_schema);
        }

        Ok(PlanProperties::new(
            eq_properties,
            output_partitioning,
            input.pipeline_behavior(),
            input.boundedness(),
        ))
    }

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

impl DisplayAs for FilterExec {
    fn fmt_as(
        &self,
        t: DisplayFormatType,
        f: &mut std::fmt::Formatter,
    ) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                let display_projections = if let Some(projection) =
                    self.projection.as_ref()
                {
                    format!(
                        ", projection=[{}]",
                        projection
                            .iter()
                            .map(|index| format!(
                                "{}@{}",
                                self.input.schema().fields().get(*index).unwrap().name(),
                                index
                            ))
                            .collect::<Vec<_>>()
                            .join(", ")
                    )
                } else {
                    "".to_string()
                };
                let fetch = self
                    .fetch
                    .map_or_else(|| "".to_string(), |f| format!(", fetch={f}"));
                write!(
                    f,
                    "FilterExec: {}{}{}",
                    self.predicate, display_projections, fetch
                )
            }
            DisplayFormatType::TreeRender => {
                write!(f, "predicate={}", fmt_sql(self.predicate.as_ref()))
            }
        }
    }
}

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

    /// 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>> {
        vec![&self.input]
    }

    fn maintains_input_order(&self) -> Vec<bool> {
        // Tell optimizer this operator doesn't reorder its input
        vec![true]
    }

    fn with_new_children(
        self: Arc<Self>,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        check_if_same_properties!(self, children);
        let new_input = children.swap_remove(0);
        FilterExecBuilder::from(&*self)
            .with_input(new_input)
            .build()
            .map(|e| Arc::new(e) as _)
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        trace!(
            "Start FilterExec::execute for partition {} of context session_id {} and task_id {:?}",
            partition,
            context.session_id(),
            context.task_id()
        );
        let metrics = FilterExecMetrics::new(&self.metrics, partition);
        Ok(Box::pin(FilterExecStream {
            schema: self.schema(),
            predicate: Arc::clone(&self.predicate),
            input: self.input.execute(partition, context)?,
            metrics,
            projection: self.projection.clone(),
            batch_coalescer: LimitedBatchCoalescer::new(
                self.schema(),
                self.batch_size,
                self.fetch,
            ),
        }))
    }

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

    /// The output statistics of a filtering operation can be estimated if the
    /// predicate's selectivity value can be determined for the incoming data.
    fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
        let input_stats = self.input.partition_statistics(partition)?;
        let stats = Self::statistics_helper(
            &self.input.schema(),
            input_stats,
            self.predicate(),
            self.default_selectivity,
        )?;
        Ok(stats.project(self.projection.as_ref()))
    }

    fn cardinality_effect(&self) -> CardinalityEffect {
        CardinalityEffect::LowerEqual
    }

    /// Tries to swap `projection` with its input (`filter`). If possible, performs
    /// the swap and returns [`FilterExec`] as the top plan. Otherwise, returns `None`.
    fn try_swapping_with_projection(
        &self,
        projection: &ProjectionExec,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        // If the projection does not narrow the schema, we should not try to push it down:
        if projection.expr().len() < projection.input().schema().fields().len() {
            // Each column in the predicate expression must exist after the projection.
            if let Some(new_predicate) =
                update_expr(self.predicate(), projection.expr(), false)?
            {
                return FilterExecBuilder::from(self)
                    .with_input(make_with_child(projection, self.input())?)
                    .with_predicate(new_predicate)
                    // The original FilterExec projection referenced columns from its old
                    // input. After the swap the new input is the ProjectionExec which
                    // already handles column selection, so clear the projection here.
                    .apply_projection(None)?
                    .build()
                    .map(|e| Some(Arc::new(e) as _));
            }
        }
        try_embed_projection(projection, self)
    }

    fn gather_filters_for_pushdown(
        &self,
        phase: FilterPushdownPhase,
        parent_filters: Vec<Arc<dyn PhysicalExpr>>,
        _config: &ConfigOptions,
    ) -> Result<FilterDescription> {
        if phase != FilterPushdownPhase::Pre {
            let child =
                ChildFilterDescription::from_child(&parent_filters, self.input())?;
            return Ok(FilterDescription::new().with_child(child));
        }

        let child = ChildFilterDescription::from_child(&parent_filters, self.input())?
            .with_self_filters(
                split_conjunction(&self.predicate)
                    .into_iter()
                    .cloned()
                    .collect(),
            );

        Ok(FilterDescription::new().with_child(child))
    }

    fn handle_child_pushdown_result(
        &self,
        phase: FilterPushdownPhase,
        child_pushdown_result: ChildPushdownResult,
        _config: &ConfigOptions,
    ) -> Result<FilterPushdownPropagation<Arc<dyn ExecutionPlan>>> {
        if phase != FilterPushdownPhase::Pre {
            return Ok(FilterPushdownPropagation::if_all(child_pushdown_result));
        }
        // We absorb any parent filters that were not handled by our children
        let mut unsupported_parent_filters: Vec<Arc<dyn PhysicalExpr>> =
            child_pushdown_result
                .parent_filters
                .iter()
                .filter_map(|f| {
                    matches!(f.all(), PushedDown::No).then_some(Arc::clone(&f.filter))
                })
                .collect();

        // If this FilterExec has a projection, the unsupported parent filters
        // are in the output schema (after projection) coordinates. We need to
        // remap them to the input schema coordinates before combining with self filters.
        if self.projection.is_some() {
            let input_schema = self.input().schema();
            unsupported_parent_filters = unsupported_parent_filters
                .into_iter()
                .map(|expr| reassign_expr_columns(expr, &input_schema))
                .collect::<Result<Vec<_>>>()?;
        }

        let unsupported_self_filters = child_pushdown_result
            .self_filters
            .first()
            .expect("we have exactly one child")
            .iter()
            .filter_map(|f| match f.discriminant {
                PushedDown::Yes => None,
                PushedDown::No => Some(&f.predicate),
            })
            .cloned();

        let unhandled_filters = unsupported_parent_filters
            .into_iter()
            .chain(unsupported_self_filters)
            .collect_vec();

        // If we have unhandled filters, we need to create a new FilterExec
        let filter_input = Arc::clone(self.input());
        let new_predicate = conjunction(unhandled_filters);
        let updated_node = if new_predicate.eq(&lit(true)) {
            // FilterExec is no longer needed, but we may need to leave a projection in place
            match self.projection().as_ref() {
                Some(projection_indices) => {
                    let filter_child_schema = filter_input.schema();
                    let proj_exprs = projection_indices
                        .iter()
                        .map(|p| {
                            let field = filter_child_schema.field(*p).clone();
                            ProjectionExpr {
                                expr: Arc::new(Column::new(field.name(), *p))
                                    as Arc<dyn PhysicalExpr>,
                                alias: field.name().to_string(),
                            }
                        })
                        .collect::<Vec<_>>();
                    Some(Arc::new(ProjectionExec::try_new(proj_exprs, filter_input)?)
                        as Arc<dyn ExecutionPlan>)
                }
                None => {
                    // No projection needed, just return the input
                    Some(filter_input)
                }
            }
        } else if new_predicate.eq(&self.predicate) {
            // The new predicate is the same as our current predicate
            None
        } else {
            // Create a new FilterExec with the new predicate, preserving the projection
            let new = FilterExec {
                predicate: Arc::clone(&new_predicate),
                input: Arc::clone(&filter_input),
                metrics: self.metrics.clone(),
                default_selectivity: self.default_selectivity,
                cache: Arc::new(Self::compute_properties(
                    &filter_input,
                    &new_predicate,
                    self.default_selectivity,
                    self.projection.as_deref(),
                )?),
                projection: self.projection.clone(),
                batch_size: self.batch_size,
                fetch: self.fetch,
            };
            Some(Arc::new(new) as _)
        };

        Ok(FilterPushdownPropagation {
            filters: vec![PushedDown::Yes; child_pushdown_result.parent_filters.len()],
            updated_node,
        })
    }

    fn fetch(&self) -> Option<usize> {
        self.fetch
    }

    fn with_fetch(&self, fetch: Option<usize>) -> Option<Arc<dyn ExecutionPlan>> {
        Some(Arc::new(Self {
            predicate: Arc::clone(&self.predicate),
            input: Arc::clone(&self.input),
            metrics: self.metrics.clone(),
            default_selectivity: self.default_selectivity,
            cache: Arc::clone(&self.cache),
            projection: self.projection.clone(),
            batch_size: self.batch_size,
            fetch,
        }))
    }

    fn with_preserve_order(
        &self,
        preserve_order: bool,
    ) -> Option<Arc<dyn ExecutionPlan>> {
        self.input
            .with_preserve_order(preserve_order)
            .and_then(|new_input| {
                Arc::new(self.clone())
                    .with_new_children(vec![new_input])
                    .ok()
            })
    }
}

impl EmbeddedProjection for FilterExec {
    fn with_projection(&self, projection: Option<Vec<usize>>) -> Result<Self> {
        FilterExecBuilder::from(self)
            .apply_projection(projection)?
            .build()
    }
}

/// Converts an interval bound to a [`Precision`] value. NULL bounds (which
/// represent "unbounded" in the interval type) map to [`Precision::Absent`].
fn interval_bound_to_precision(
    bound: ScalarValue,
    is_exact: bool,
) -> Precision<ScalarValue> {
    if bound.is_null() {
        Precision::Absent
    } else if is_exact {
        Precision::Exact(bound)
    } else {
        Precision::Inexact(bound)
    }
}

/// This function ensures that all bounds in the `ExprBoundaries` vector are
/// converted to closed bounds. If a lower/upper bound is initially open, it
/// is adjusted by using the next/previous value for its data type to convert
/// it into a closed bound.
fn collect_new_statistics(
    schema: &SchemaRef,
    input_column_stats: &[ColumnStatistics],
    analysis_boundaries: Vec<ExprBoundaries>,
) -> Vec<ColumnStatistics> {
    analysis_boundaries
        .into_iter()
        .enumerate()
        .map(
            |(
                idx,
                ExprBoundaries {
                    interval,
                    distinct_count,
                    ..
                },
            )| {
                let Some(interval) = interval else {
                    // If the interval is `None`, we can say that there are no rows.
                    // Use a typed null to preserve the column's data type, so that
                    // downstream interval analysis can still intersect intervals
                    // of the same type.
                    let typed_null = ScalarValue::try_from(schema.field(idx).data_type())
                        .unwrap_or(ScalarValue::Null);
                    return ColumnStatistics {
                        null_count: Precision::Exact(0),
                        max_value: Precision::Exact(typed_null.clone()),
                        min_value: Precision::Exact(typed_null.clone()),
                        sum_value: Precision::Exact(typed_null),
                        distinct_count: Precision::Exact(0),
                        byte_size: input_column_stats[idx].byte_size,
                    };
                };
                let (lower, upper) = interval.into_bounds();
                let is_exact = !lower.is_null() && !upper.is_null() && lower == upper;
                let min_value = interval_bound_to_precision(lower, is_exact);
                let max_value = interval_bound_to_precision(upper, is_exact);
                ColumnStatistics {
                    null_count: input_column_stats[idx].null_count.to_inexact(),
                    max_value,
                    min_value,
                    sum_value: Precision::Absent,
                    distinct_count: distinct_count.to_inexact(),
                    byte_size: input_column_stats[idx].byte_size,
                }
            },
        )
        .collect()
}

/// The FilterExec streams wraps the input iterator and applies the predicate expression to
/// determine which rows to include in its output batches
struct FilterExecStream {
    /// Output schema after the projection
    schema: SchemaRef,
    /// The expression to filter on. This expression must evaluate to a boolean value.
    predicate: Arc<dyn PhysicalExpr>,
    /// The input partition to filter.
    input: SendableRecordBatchStream,
    /// Runtime metrics recording
    metrics: FilterExecMetrics,
    /// The projection indices of the columns in the input schema
    projection: Option<ProjectionRef>,
    /// Batch coalescer to combine small batches
    batch_coalescer: LimitedBatchCoalescer,
}

/// The metrics for `FilterExec`
struct FilterExecMetrics {
    /// Common metrics for most operators
    baseline_metrics: BaselineMetrics,
    /// Selectivity of the filter, calculated as output_rows / input_rows
    selectivity: RatioMetrics,
    // Remember to update `docs/source/user-guide/metrics.md` when adding new metrics,
    // or modifying metrics comments
}

impl FilterExecMetrics {
    pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self {
        Self {
            baseline_metrics: BaselineMetrics::new(metrics, partition),
            selectivity: MetricBuilder::new(metrics)
                .with_type(MetricType::SUMMARY)
                .ratio_metrics("selectivity", partition),
        }
    }
}

pub fn batch_filter(
    batch: &RecordBatch,
    predicate: &Arc<dyn PhysicalExpr>,
) -> Result<RecordBatch> {
    filter_and_project(batch, predicate, None)
}

fn filter_and_project(
    batch: &RecordBatch,
    predicate: &Arc<dyn PhysicalExpr>,
    projection: Option<&Vec<usize>>,
) -> Result<RecordBatch> {
    predicate
        .evaluate(batch)
        .and_then(|v| v.into_array(batch.num_rows()))
        .and_then(|array| {
            Ok(match (as_boolean_array(&array), projection) {
                // Apply filter array to record batch
                (Ok(filter_array), None) => filter_record_batch(batch, filter_array)?,
                (Ok(filter_array), Some(projection)) => {
                    let projected_batch = batch.project(projection)?;
                    filter_record_batch(&projected_batch, filter_array)?
                }
                (Err(_), _) => {
                    return internal_err!(
                        "Cannot create filter_array from non-boolean predicates"
                    );
                }
            })
        })
}

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

    fn poll_next(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<Option<Self::Item>> {
        let elapsed_compute = self.metrics.baseline_metrics.elapsed_compute().clone();
        loop {
            // If there is a completed batch ready, return it
            if let Some(batch) = self.batch_coalescer.next_completed_batch() {
                self.metrics.selectivity.add_part(batch.num_rows());
                let poll = Poll::Ready(Some(Ok(batch)));
                return self.metrics.baseline_metrics.record_poll(poll);
            }

            if self.batch_coalescer.is_finished() {
                // If input is done and no batches are ready, return None to signal end of stream.
                return Poll::Ready(None);
            }

            // Attempt to pull the next batch from the input stream.
            match ready!(self.input.poll_next_unpin(cx)) {
                None => {
                    self.batch_coalescer.finish()?;
                    // continue draining the coalescer
                }
                Some(Ok(batch)) => {
                    let timer = elapsed_compute.timer();
                    let status = self.predicate.as_ref()
                        .evaluate(&batch)
                        .and_then(|v| v.into_array(batch.num_rows()))
                        .and_then(|array| {
                            Ok(match self.projection.as_ref()  {
                                Some(projection) => {
                                    let projected_batch = batch.project(projection)?;
                                    (array, projected_batch)
                                },
                                None => (array, batch)
                            })
                        }).and_then(|(array, batch)| {
                            match as_boolean_array(&array) {
                                Ok(filter_array) => {
                                    self.metrics.selectivity.add_total(batch.num_rows());
                                    // TODO: support push_batch_with_filter in LimitedBatchCoalescer
                                    let batch = filter_record_batch(&batch, filter_array)?;
                                    let state = self.batch_coalescer.push_batch(batch)?;
                                    Ok(state)
                                }
                                Err(_) => {
                                    internal_err!(
                                        "Cannot create filter_array from non-boolean predicates"
                                    )
                                }
                            }
                        })?;
                    timer.done();

                    match status {
                        PushBatchStatus::Continue => {
                            // Keep pushing more batches
                        }
                        PushBatchStatus::LimitReached => {
                            // limit was reached, so stop early
                            self.batch_coalescer.finish()?;
                            // continue draining the coalescer
                        }
                    }
                }

                // Error case
                other => return Poll::Ready(other),
            }
        }
    }

    fn size_hint(&self) -> (usize, Option<usize>) {
        // Same number of record batches
        self.input.size_hint()
    }
}
impl RecordBatchStream for FilterExecStream {
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }
}

/// Return the equals Column-Pairs and Non-equals Column-Pairs
#[deprecated(
    since = "51.0.0",
    note = "This function will be internal in the future"
)]
pub fn collect_columns_from_predicate(
    predicate: &'_ Arc<dyn PhysicalExpr>,
) -> EqualAndNonEqual<'_> {
    collect_columns_from_predicate_inner(predicate)
}

fn collect_columns_from_predicate_inner(
    predicate: &'_ Arc<dyn PhysicalExpr>,
) -> EqualAndNonEqual<'_> {
    let mut eq_predicate_columns = Vec::<PhysicalExprPairRef>::new();
    let mut ne_predicate_columns = Vec::<PhysicalExprPairRef>::new();

    let predicates = split_conjunction(predicate);
    predicates.into_iter().for_each(|p| {
        if let Some(binary) = p.as_any().downcast_ref::<BinaryExpr>() {
            // Only extract pairs where at least one side is a Column reference.
            // Pairs like `complex_expr = literal` should not create equivalence
            // classes — the literal could appear in many unrelated expressions
            // (e.g. sort keys), and normalize_expr's deep traversal would
            // replace those occurrences with the complex expression, corrupting
            // sort orderings. Constant propagation for such pairs is handled
            // separately by `extend_constants`.
            let has_direct_column_operand =
                binary.left().as_any().downcast_ref::<Column>().is_some()
                    || binary.right().as_any().downcast_ref::<Column>().is_some();
            if !has_direct_column_operand {
                return;
            }
            match binary.op() {
                Operator::Eq => {
                    eq_predicate_columns.push((binary.left(), binary.right()))
                }
                Operator::NotEq => {
                    ne_predicate_columns.push((binary.left(), binary.right()))
                }
                _ => {}
            }
        }
    });

    (eq_predicate_columns, ne_predicate_columns)
}

/// Pair of `Arc<dyn PhysicalExpr>`s
pub type PhysicalExprPairRef<'a> = (&'a Arc<dyn PhysicalExpr>, &'a Arc<dyn PhysicalExpr>);

/// The equals Column-Pairs and Non-equals Column-Pairs in the Predicates
pub type EqualAndNonEqual<'a> =
    (Vec<PhysicalExprPairRef<'a>>, Vec<PhysicalExprPairRef<'a>>);

#[cfg(test)]
mod tests {
    use super::*;
    use crate::empty::EmptyExec;
    use crate::expressions::*;
    use crate::test;
    use crate::test::exec::StatisticsExec;
    use arrow::datatypes::{Field, Schema, UnionFields, UnionMode};
    use datafusion_common::ScalarValue;

    #[tokio::test]
    async fn collect_columns_predicates() -> Result<()> {
        let schema = test::aggr_test_schema();
        let predicate: Arc<dyn PhysicalExpr> = binary(
            binary(
                binary(col("c2", &schema)?, Operator::GtEq, lit(1u32), &schema)?,
                Operator::And,
                binary(col("c2", &schema)?, Operator::Eq, lit(4u32), &schema)?,
                &schema,
            )?,
            Operator::And,
            binary(
                binary(
                    col("c2", &schema)?,
                    Operator::Eq,
                    col("c9", &schema)?,
                    &schema,
                )?,
                Operator::And,
                binary(
                    col("c1", &schema)?,
                    Operator::NotEq,
                    col("c13", &schema)?,
                    &schema,
                )?,
                &schema,
            )?,
            &schema,
        )?;

        let (equal_pairs, ne_pairs) = collect_columns_from_predicate_inner(&predicate);
        assert_eq!(2, equal_pairs.len());
        assert!(equal_pairs[0].0.eq(&col("c2", &schema)?));
        assert!(equal_pairs[0].1.eq(&lit(4u32)));

        assert!(equal_pairs[1].0.eq(&col("c2", &schema)?));
        assert!(equal_pairs[1].1.eq(&col("c9", &schema)?));

        assert_eq!(1, ne_pairs.len());
        assert!(ne_pairs[0].0.eq(&col("c1", &schema)?));
        assert!(ne_pairs[0].1.eq(&col("c13", &schema)?));

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_basic_expr() -> Result<()> {
        // Table:
        //      a: min=1, max=100
        let bytes_per_row = 4;
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(100),
                total_byte_size: Precision::Inexact(100 * bytes_per_row),
                column_statistics: vec![ColumnStatistics {
                    min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                    max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                    ..Default::default()
                }],
            },
            schema.clone(),
        ));

        // a <= 25
        let predicate: Arc<dyn PhysicalExpr> =
            binary(col("a", &schema)?, Operator::LtEq, lit(25i32), &schema)?;

        // WHERE a <= 25
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);

        let statistics = filter.partition_statistics(None)?;
        assert_eq!(statistics.num_rows, Precision::Inexact(25));
        assert_eq!(
            statistics.total_byte_size,
            Precision::Inexact(25 * bytes_per_row)
        );
        assert_eq!(
            statistics.column_statistics,
            vec![ColumnStatistics {
                min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                max_value: Precision::Inexact(ScalarValue::Int32(Some(25))),
                ..Default::default()
            }]
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_column_level_nested() -> Result<()> {
        // Table:
        //      a: min=1, max=100
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(100),
                column_statistics: vec![ColumnStatistics {
                    min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                    max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                    ..Default::default()
                }],
                total_byte_size: Precision::Absent,
            },
            schema.clone(),
        ));

        // WHERE a <= 25
        let sub_filter: Arc<dyn ExecutionPlan> = Arc::new(FilterExec::try_new(
            binary(col("a", &schema)?, Operator::LtEq, lit(25i32), &schema)?,
            input,
        )?);

        // Nested filters (two separate physical plans, instead of AND chain in the expr)
        // WHERE a >= 10
        // WHERE a <= 25
        let filter: Arc<dyn ExecutionPlan> = Arc::new(FilterExec::try_new(
            binary(col("a", &schema)?, Operator::GtEq, lit(10i32), &schema)?,
            sub_filter,
        )?);

        let statistics = filter.partition_statistics(None)?;
        assert_eq!(statistics.num_rows, Precision::Inexact(16));
        assert_eq!(
            statistics.column_statistics,
            vec![ColumnStatistics {
                min_value: Precision::Inexact(ScalarValue::Int32(Some(10))),
                max_value: Precision::Inexact(ScalarValue::Int32(Some(25))),
                ..Default::default()
            }]
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_column_level_nested_multiple() -> Result<()> {
        // Table:
        //      a: min=1, max=100
        //      b: min=1, max=50
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(100),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(50))),
                        ..Default::default()
                    },
                ],
                total_byte_size: Precision::Absent,
            },
            schema.clone(),
        ));

        // WHERE a <= 25
        let a_lte_25: Arc<dyn ExecutionPlan> = Arc::new(FilterExec::try_new(
            binary(col("a", &schema)?, Operator::LtEq, lit(25i32), &schema)?,
            input,
        )?);

        // WHERE b > 45
        let b_gt_5: Arc<dyn ExecutionPlan> = Arc::new(FilterExec::try_new(
            binary(col("b", &schema)?, Operator::Gt, lit(45i32), &schema)?,
            a_lte_25,
        )?);

        // WHERE a >= 10
        let filter: Arc<dyn ExecutionPlan> = Arc::new(FilterExec::try_new(
            binary(col("a", &schema)?, Operator::GtEq, lit(10i32), &schema)?,
            b_gt_5,
        )?);
        let statistics = filter.partition_statistics(None)?;
        // On a uniform distribution, only fifteen rows will satisfy the
        // filter that 'a' proposed (a >= 10 AND a <= 25) (15/100) and only
        // 5 rows will satisfy the filter that 'b' proposed (b > 45) (5/50).
        //
        // Which would result with a selectivity of  '15/100 * 5/50' or 0.015
        // and that means about %1.5 of the all rows (rounded up to 2 rows).
        assert_eq!(statistics.num_rows, Precision::Inexact(2));
        assert_eq!(
            statistics.column_statistics,
            vec![
                ColumnStatistics {
                    min_value: Precision::Inexact(ScalarValue::Int32(Some(10))),
                    max_value: Precision::Inexact(ScalarValue::Int32(Some(25))),
                    ..Default::default()
                },
                ColumnStatistics {
                    min_value: Precision::Inexact(ScalarValue::Int32(Some(46))),
                    max_value: Precision::Inexact(ScalarValue::Int32(Some(50))),
                    ..Default::default()
                }
            ]
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_when_input_stats_missing() -> Result<()> {
        // Table:
        //      a: min=???, max=??? (missing)
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
        let input = Arc::new(StatisticsExec::new(
            Statistics::new_unknown(&schema),
            schema.clone(),
        ));

        // a <= 25
        let predicate: Arc<dyn PhysicalExpr> =
            binary(col("a", &schema)?, Operator::LtEq, lit(25i32), &schema)?;

        // WHERE a <= 25
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);

        let statistics = filter.partition_statistics(None)?;
        assert_eq!(statistics.num_rows, Precision::Absent);

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_multiple_columns() -> Result<()> {
        // Table:
        //      a: min=1, max=100
        //      b: min=1, max=3
        //      c: min=1000.0  max=1100.0
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
            Field::new("c", DataType::Float32, false),
        ]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(4000),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(3))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Float32(Some(1000.0))),
                        max_value: Precision::Inexact(ScalarValue::Float32(Some(1100.0))),
                        ..Default::default()
                    },
                ],
            },
            schema,
        ));
        // WHERE a<=53 AND (b=3 AND (c<=1075.0 AND a>b))
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(BinaryExpr::new(
                Arc::new(Column::new("a", 0)),
                Operator::LtEq,
                Arc::new(Literal::new(ScalarValue::Int32(Some(53)))),
            )),
            Operator::And,
            Arc::new(BinaryExpr::new(
                Arc::new(BinaryExpr::new(
                    Arc::new(Column::new("b", 1)),
                    Operator::Eq,
                    Arc::new(Literal::new(ScalarValue::Int32(Some(3)))),
                )),
                Operator::And,
                Arc::new(BinaryExpr::new(
                    Arc::new(BinaryExpr::new(
                        Arc::new(Column::new("c", 2)),
                        Operator::LtEq,
                        Arc::new(Literal::new(ScalarValue::Float32(Some(1075.0)))),
                    )),
                    Operator::And,
                    Arc::new(BinaryExpr::new(
                        Arc::new(Column::new("a", 0)),
                        Operator::Gt,
                        Arc::new(Column::new("b", 1)),
                    )),
                )),
            )),
        ));
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);
        let statistics = filter.partition_statistics(None)?;
        // 0.5 (from a) * 0.333333... (from b) * 0.798387... (from c) ≈ 0.1330...
        // num_rows after ceil => 133.0... => 134
        // total_byte_size after ceil => 532.0... => 533
        assert_eq!(statistics.num_rows, Precision::Inexact(134));
        assert_eq!(statistics.total_byte_size, Precision::Inexact(533));
        let exp_col_stats = vec![
            ColumnStatistics {
                min_value: Precision::Inexact(ScalarValue::Int32(Some(4))),
                max_value: Precision::Inexact(ScalarValue::Int32(Some(53))),
                ..Default::default()
            },
            ColumnStatistics {
                min_value: Precision::Inexact(ScalarValue::Int32(Some(3))),
                max_value: Precision::Inexact(ScalarValue::Int32(Some(3))),
                ..Default::default()
            },
            ColumnStatistics {
                min_value: Precision::Inexact(ScalarValue::Float32(Some(1000.0))),
                max_value: Precision::Inexact(ScalarValue::Float32(Some(1075.0))),
                ..Default::default()
            },
        ];
        let _ = exp_col_stats
            .into_iter()
            .zip(statistics.column_statistics)
            .map(|(expected, actual)| {
                if let Some(val) = actual.min_value.get_value() {
                    if val.data_type().is_floating() {
                        // Windows rounds arithmetic operation results differently for floating point numbers.
                        // Therefore, we check if the actual values are in an epsilon range.
                        let actual_min = actual.min_value.get_value().unwrap();
                        let actual_max = actual.max_value.get_value().unwrap();
                        let expected_min = expected.min_value.get_value().unwrap();
                        let expected_max = expected.max_value.get_value().unwrap();
                        let eps = ScalarValue::Float32(Some(1e-6));

                        assert!(actual_min.sub(expected_min).unwrap() < eps);
                        assert!(actual_min.sub(expected_min).unwrap() < eps);

                        assert!(actual_max.sub(expected_max).unwrap() < eps);
                        assert!(actual_max.sub(expected_max).unwrap() < eps);
                    } else {
                        assert_eq!(actual, expected);
                    }
                } else {
                    assert_eq!(actual, expected);
                }
            });

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_full_selective() -> Result<()> {
        // Table:
        //      a: min=1, max=100
        //      b: min=1, max=3
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(4000),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(3))),
                        ..Default::default()
                    },
                ],
            },
            schema,
        ));
        // WHERE a<200 AND 1<=b
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(BinaryExpr::new(
                Arc::new(Column::new("a", 0)),
                Operator::Lt,
                Arc::new(Literal::new(ScalarValue::Int32(Some(200)))),
            )),
            Operator::And,
            Arc::new(BinaryExpr::new(
                Arc::new(Literal::new(ScalarValue::Int32(Some(1)))),
                Operator::LtEq,
                Arc::new(Column::new("b", 1)),
            )),
        ));
        // Since filter predicate passes all entries, statistics after filter shouldn't change.
        let expected = input.partition_statistics(None)?.column_statistics;
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);
        let statistics = filter.partition_statistics(None)?;

        assert_eq!(statistics.num_rows, Precision::Inexact(1000));
        assert_eq!(statistics.total_byte_size, Precision::Inexact(4000));
        assert_eq!(statistics.column_statistics, expected);

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_zero_selective() -> Result<()> {
        // Table:
        //      a: min=1, max=100
        //      b: min=1, max=3
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(4000),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(3))),
                        ..Default::default()
                    },
                ],
            },
            schema,
        ));
        // WHERE a>200 AND 1<=b
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(BinaryExpr::new(
                Arc::new(Column::new("a", 0)),
                Operator::Gt,
                Arc::new(Literal::new(ScalarValue::Int32(Some(200)))),
            )),
            Operator::And,
            Arc::new(BinaryExpr::new(
                Arc::new(Literal::new(ScalarValue::Int32(Some(1)))),
                Operator::LtEq,
                Arc::new(Column::new("b", 1)),
            )),
        ));
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);
        let statistics = filter.partition_statistics(None)?;

        assert_eq!(statistics.num_rows, Precision::Inexact(0));
        assert_eq!(statistics.total_byte_size, Precision::Inexact(0));
        assert_eq!(
            statistics.column_statistics,
            vec![
                ColumnStatistics {
                    min_value: Precision::Exact(ScalarValue::Int32(None)),
                    max_value: Precision::Exact(ScalarValue::Int32(None)),
                    sum_value: Precision::Exact(ScalarValue::Int32(None)),
                    distinct_count: Precision::Exact(0),
                    null_count: Precision::Exact(0),
                    byte_size: Precision::Absent,
                },
                ColumnStatistics {
                    min_value: Precision::Exact(ScalarValue::Int32(None)),
                    max_value: Precision::Exact(ScalarValue::Int32(None)),
                    sum_value: Precision::Exact(ScalarValue::Int32(None)),
                    distinct_count: Precision::Exact(0),
                    null_count: Precision::Exact(0),
                    byte_size: Precision::Absent,
                },
            ]
        );

        Ok(())
    }

    /// Regression test: stacking two FilterExecs where the inner filter
    /// proves zero selectivity should not panic with a type mismatch
    /// during interval intersection.
    ///
    /// Previously, when a filter proved no rows could match, the column
    /// statistics used untyped `ScalarValue::Null` (data type `Null`).
    /// If an outer FilterExec then tried to analyze its own predicate
    /// against those statistics, `Interval::intersect` would fail with:
    ///   "Only intervals with the same data type are intersectable, lhs:Null, rhs:Int32"
    #[tokio::test]
    async fn test_nested_filter_with_zero_selectivity_inner() -> Result<()> {
        // Inner table: a: [1, 100], b: [1, 3]
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(4000),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(3))),
                        ..Default::default()
                    },
                ],
            },
            schema,
        ));

        // Inner filter: a > 200 (impossible given a max=100 → zero selectivity)
        let inner_predicate: Arc<dyn PhysicalExpr> = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(200)))),
        ));
        let inner_filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(inner_predicate, input)?);

        // Outer filter: a = 50
        // Before the fix, this would panic because the inner filter's
        // zero-selectivity statistics produced Null-typed intervals for
        // column `a`, which couldn't intersect with the Int32 literal.
        let outer_predicate: Arc<dyn PhysicalExpr> = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Eq,
            Arc::new(Literal::new(ScalarValue::Int32(Some(50)))),
        ));
        let outer_filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(outer_predicate, inner_filter)?);

        // Should succeed without error
        let statistics = outer_filter.partition_statistics(None)?;
        assert_eq!(statistics.num_rows, Precision::Inexact(0));

        Ok(())
    }

    #[tokio::test]
    async fn test_filter_statistics_more_inputs() -> Result<()> {
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(4000),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                ],
            },
            schema,
        ));
        // WHERE a<50
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Lt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(50)))),
        ));
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);
        let statistics = filter.partition_statistics(None)?;

        assert_eq!(statistics.num_rows, Precision::Inexact(490));
        assert_eq!(statistics.total_byte_size, Precision::Inexact(1960));
        assert_eq!(
            statistics.column_statistics,
            vec![
                ColumnStatistics {
                    min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                    max_value: Precision::Inexact(ScalarValue::Int32(Some(49))),
                    ..Default::default()
                },
                ColumnStatistics {
                    min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                    max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                    ..Default::default()
                },
            ]
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_empty_input_statistics() -> Result<()> {
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
        let input = Arc::new(StatisticsExec::new(
            Statistics::new_unknown(&schema),
            schema,
        ));
        // WHERE a <= 10 AND 0 <= a - 5
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(BinaryExpr::new(
                Arc::new(Column::new("a", 0)),
                Operator::LtEq,
                Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
            )),
            Operator::And,
            Arc::new(BinaryExpr::new(
                Arc::new(Literal::new(ScalarValue::Int32(Some(0)))),
                Operator::LtEq,
                Arc::new(BinaryExpr::new(
                    Arc::new(Column::new("a", 0)),
                    Operator::Minus,
                    Arc::new(Literal::new(ScalarValue::Int32(Some(5)))),
                )),
            )),
        ));
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);
        let filter_statistics = filter.partition_statistics(None)?;

        let expected_filter_statistics = Statistics {
            num_rows: Precision::Absent,
            total_byte_size: Precision::Absent,
            column_statistics: vec![ColumnStatistics {
                null_count: Precision::Absent,
                min_value: Precision::Inexact(ScalarValue::Int32(Some(5))),
                max_value: Precision::Inexact(ScalarValue::Int32(Some(10))),
                sum_value: Precision::Absent,
                distinct_count: Precision::Absent,
                byte_size: Precision::Absent,
            }],
        };

        assert_eq!(filter_statistics, expected_filter_statistics);

        Ok(())
    }

    #[tokio::test]
    async fn test_statistics_with_constant_column() -> Result<()> {
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
        let input = Arc::new(StatisticsExec::new(
            Statistics::new_unknown(&schema),
            schema,
        ));
        // WHERE a = 10
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Eq,
            Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
        ));
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);
        let filter_statistics = filter.partition_statistics(None)?;
        // First column is "a", and it is a column with only one value after the filter.
        assert!(filter_statistics.column_statistics[0].is_singleton());

        Ok(())
    }

    #[tokio::test]
    async fn test_validation_filter_selectivity() -> Result<()> {
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
        let input = Arc::new(StatisticsExec::new(
            Statistics::new_unknown(&schema),
            schema,
        ));
        // WHERE a = 10
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Eq,
            Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
        ));
        let filter = FilterExec::try_new(predicate, input)?;
        assert!(filter.with_default_selectivity(120).is_err());
        Ok(())
    }

    #[tokio::test]
    async fn test_custom_filter_selectivity() -> Result<()> {
        // Need a decimal to trigger inexact selectivity
        let schema =
            Schema::new(vec![Field::new("a", DataType::Decimal128(2, 3), false)]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(4000),
                column_statistics: vec![ColumnStatistics {
                    ..Default::default()
                }],
            },
            schema,
        ));
        // WHERE a = 10
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Eq,
            Arc::new(Literal::new(ScalarValue::Decimal128(Some(10), 10, 10))),
        ));
        let filter = FilterExec::try_new(predicate, input)?;
        let statistics = filter.partition_statistics(None)?;
        assert_eq!(statistics.num_rows, Precision::Inexact(200));
        assert_eq!(statistics.total_byte_size, Precision::Inexact(800));
        let filter = filter.with_default_selectivity(40)?;
        let statistics = filter.partition_statistics(None)?;
        assert_eq!(statistics.num_rows, Precision::Inexact(400));
        assert_eq!(statistics.total_byte_size, Precision::Inexact(1600));
        Ok(())
    }

    #[test]
    fn test_equivalence_properties_union_type() -> Result<()> {
        let union_type = DataType::Union(
            UnionFields::try_new(
                vec![0, 1],
                vec![
                    Field::new("f1", DataType::Int32, true),
                    Field::new("f2", DataType::Utf8, true),
                ],
            )
            .unwrap(),
            UnionMode::Sparse,
        );

        let schema = Arc::new(Schema::new(vec![
            Field::new("c1", DataType::Int32, true),
            Field::new("c2", union_type, true),
        ]));

        let exec = FilterExec::try_new(
            binary(
                binary(col("c1", &schema)?, Operator::GtEq, lit(1i32), &schema)?,
                Operator::And,
                binary(col("c1", &schema)?, Operator::LtEq, lit(4i32), &schema)?,
                &schema,
            )?,
            Arc::new(EmptyExec::new(Arc::clone(&schema))),
        )?;

        exec.partition_statistics(None).unwrap();

        Ok(())
    }

    #[tokio::test]
    async fn test_builder_with_projection() -> Result<()> {
        // Create a schema with multiple columns
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
            Field::new("c", DataType::Int32, false),
        ]));

        let input = Arc::new(EmptyExec::new(Arc::clone(&schema)));

        // Create a filter predicate: a > 10
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
        ));

        // Create filter with projection [0, 2] (columns a and c) using builder
        let projection = Some(vec![0, 2]);
        let filter = FilterExecBuilder::new(predicate, input)
            .apply_projection(projection.clone())
            .unwrap()
            .build()?;

        // Verify projection is set correctly
        assert_eq!(filter.projection(), &Some([0, 2].into()));

        // Verify schema contains only projected columns
        let output_schema = filter.schema();
        assert_eq!(output_schema.fields().len(), 2);
        assert_eq!(output_schema.field(0).name(), "a");
        assert_eq!(output_schema.field(1).name(), "c");

        Ok(())
    }

    #[tokio::test]
    async fn test_builder_without_projection() -> Result<()> {
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]));

        let input = Arc::new(EmptyExec::new(Arc::clone(&schema)));

        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(5)))),
        ));

        // Create filter without projection using builder
        let filter = FilterExecBuilder::new(predicate, input).build()?;

        // Verify no projection is set
        assert!(filter.projection().is_none());

        // Verify schema contains all columns
        let output_schema = filter.schema();
        assert_eq!(output_schema.fields().len(), 2);

        Ok(())
    }

    #[tokio::test]
    async fn test_builder_invalid_projection() -> Result<()> {
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]));

        let input = Arc::new(EmptyExec::new(Arc::clone(&schema)));

        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(5)))),
        ));

        // Try to create filter with invalid projection (index out of bounds) using builder
        let result =
            FilterExecBuilder::new(predicate, input).apply_projection(Some(vec![0, 5])); // 5 is out of bounds

        // Should return an error
        assert!(result.is_err());

        Ok(())
    }

    #[tokio::test]
    async fn test_builder_vs_with_projection() -> Result<()> {
        // This test verifies that the builder with projection produces the same result
        // as try_new().with_projection(), but more efficiently (one compute_properties call)
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
            Field::new("c", DataType::Int32, false),
            Field::new("d", DataType::Int32, false),
        ]);

        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(4000),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        ..Default::default()
                    },
                    ColumnStatistics {
                        ..Default::default()
                    },
                    ColumnStatistics {
                        ..Default::default()
                    },
                ],
            },
            schema,
        ));
        let input: Arc<dyn ExecutionPlan> = input;

        let predicate: Arc<dyn PhysicalExpr> = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Lt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(50)))),
        ));

        let projection = Some(vec![0, 2]);

        // Method 1: Builder with projection (one call to compute_properties)
        let filter1 = FilterExecBuilder::new(Arc::clone(&predicate), Arc::clone(&input))
            .apply_projection(projection.clone())
            .unwrap()
            .build()?;

        // Method 2: Also using builder for comparison (deprecated try_new().with_projection() removed)
        let filter2 = FilterExecBuilder::new(predicate, input)
            .apply_projection(projection)
            .unwrap()
            .build()?;

        // Both methods should produce equivalent results
        assert_eq!(filter1.schema(), filter2.schema());
        assert_eq!(filter1.projection(), filter2.projection());

        // Verify statistics are the same
        let stats1 = filter1.partition_statistics(None)?;
        let stats2 = filter2.partition_statistics(None)?;
        assert_eq!(stats1.num_rows, stats2.num_rows);
        assert_eq!(stats1.total_byte_size, stats2.total_byte_size);

        Ok(())
    }

    #[tokio::test]
    async fn test_builder_statistics_with_projection() -> Result<()> {
        // Test that statistics are correctly computed when using builder with projection
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
            Field::new("c", DataType::Int32, false),
        ]);

        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Inexact(12000),
                column_statistics: vec![
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(1))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(100))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(10))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(200))),
                        ..Default::default()
                    },
                    ColumnStatistics {
                        min_value: Precision::Inexact(ScalarValue::Int32(Some(5))),
                        max_value: Precision::Inexact(ScalarValue::Int32(Some(50))),
                        ..Default::default()
                    },
                ],
            },
            schema,
        ));

        // Filter: a < 50, Project: [0, 2]
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Lt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(50)))),
        ));

        let filter = FilterExecBuilder::new(predicate, input)
            .apply_projection(Some(vec![0, 2]))
            .unwrap()
            .build()?;

        let statistics = filter.partition_statistics(None)?;

        // Verify statistics reflect both filtering and projection
        assert!(matches!(statistics.num_rows, Precision::Inexact(_)));

        // Schema should only have 2 columns after projection
        assert_eq!(filter.schema().fields().len(), 2);

        Ok(())
    }

    #[test]
    fn test_builder_predicate_validation() -> Result<()> {
        // Test that builder validates predicate type correctly
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]));

        let input = Arc::new(EmptyExec::new(Arc::clone(&schema)));

        // Create a predicate that doesn't return boolean (returns Int32)
        let invalid_predicate = Arc::new(Column::new("a", 0));

        // Should fail because predicate doesn't return boolean
        let result = FilterExecBuilder::new(invalid_predicate, input)
            .apply_projection(Some(vec![0]))
            .unwrap()
            .build();

        assert!(result.is_err());

        Ok(())
    }

    #[tokio::test]
    async fn test_builder_projection_composition() -> Result<()> {
        // Test that calling apply_projection multiple times composes projections
        // If initial projection is [0, 2, 3] and we call apply_projection([0, 2]),
        // the result should be [0, 3] (indices 0 and 2 of [0, 2, 3])
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
            Field::new("c", DataType::Int32, false),
            Field::new("d", DataType::Int32, false),
        ]));

        let input = Arc::new(EmptyExec::new(Arc::clone(&schema)));

        // Create a filter predicate: a > 10
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
        ));

        // First projection: [0, 2, 3] -> select columns a, c, d
        // Second projection: [0, 2] -> select indices 0 and 2 of [0, 2, 3] -> [0, 3]
        // Final result: columns a and d
        let filter = FilterExecBuilder::new(predicate, input)
            .apply_projection(Some(vec![0, 2, 3]))?
            .apply_projection(Some(vec![0, 2]))?
            .build()?;

        // Verify composed projection is [0, 3]
        assert_eq!(filter.projection(), &Some([0, 3].into()));

        // Verify schema contains only columns a and d
        let output_schema = filter.schema();
        assert_eq!(output_schema.fields().len(), 2);
        assert_eq!(output_schema.field(0).name(), "a");
        assert_eq!(output_schema.field(1).name(), "d");

        Ok(())
    }

    #[tokio::test]
    async fn test_builder_projection_composition_none_clears() -> Result<()> {
        // Test that passing None clears the projection
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]));

        let input = Arc::new(EmptyExec::new(Arc::clone(&schema)));

        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
        ));

        // Set a projection then clear it with None
        let filter = FilterExecBuilder::new(predicate, input)
            .apply_projection(Some(vec![0]))?
            .apply_projection(None)?
            .build()?;

        // Projection should be cleared
        assert_eq!(filter.projection(), &None);

        // Schema should have all columns
        let output_schema = filter.schema();
        assert_eq!(output_schema.fields().len(), 2);

        Ok(())
    }

    #[test]
    fn test_filter_with_projection_remaps_post_phase_parent_filters() -> Result<()> {
        // Test that FilterExec with a projection must remap parent dynamic
        // filter column indices from its output schema to the input schema
        // before passing them to the child.
        let input_schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Utf8, false),
            Field::new("c", DataType::Float64, false),
        ]));
        let input = Arc::new(EmptyExec::new(Arc::clone(&input_schema)));

        // FilterExec: a > 0, projection=[c@2]
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int32(Some(0)))),
        ));
        let filter = FilterExecBuilder::new(predicate, input)
            .apply_projection(Some(vec![2]))?
            .build()?;

        // Output schema should be [c:Float64]
        let output_schema = filter.schema();
        assert_eq!(output_schema.fields().len(), 1);
        assert_eq!(output_schema.field(0).name(), "c");

        // Simulate a parent dynamic filter referencing output column c@0
        let parent_filter: Arc<dyn PhysicalExpr> = Arc::new(Column::new("c", 0));

        let config = ConfigOptions::new();
        let desc = filter.gather_filters_for_pushdown(
            FilterPushdownPhase::Post,
            vec![parent_filter],
            &config,
        )?;

        // The filter pushed to the child must reference c@2 (input schema),
        // not c@0 (output schema).
        let parent_filters = desc.parent_filters();
        assert_eq!(parent_filters.len(), 1); // one child
        assert_eq!(parent_filters[0].len(), 1); // one filter
        let remapped = &parent_filters[0][0].predicate;
        let display = format!("{remapped}");
        assert_eq!(
            display, "c@2",
            "Post-phase parent filter column index must be remapped \
             from output schema (c@0) to input schema (c@2)"
        );

        Ok(())
    }

    /// Regression test for https://github.com/apache/datafusion/issues/20194
    ///
    /// `collect_columns_from_predicate_inner` should only extract equality
    /// pairs where at least one side is a Column. Pairs like
    /// `complex_expr = literal` must not create equivalence classes because
    /// `normalize_expr`'s deep traversal would replace the literal inside
    /// unrelated expressions (e.g. sort keys) with the complex expression.
    #[test]
    fn test_collect_columns_skips_non_column_pairs() -> Result<()> {
        let schema = test::aggr_test_schema();

        // Simulate: nvl(c2, 0) = 0  →  (c2 IS DISTINCT FROM 0) = 0
        // Neither side is a Column, so this should NOT be extracted.
        let complex_expr: Arc<dyn PhysicalExpr> = binary(
            col("c2", &schema)?,
            Operator::IsDistinctFrom,
            lit(0u32),
            &schema,
        )?;
        let predicate: Arc<dyn PhysicalExpr> =
            binary(complex_expr, Operator::Eq, lit(0u32), &schema)?;

        let (equal_pairs, _) = collect_columns_from_predicate_inner(&predicate);
        assert_eq!(
            0,
            equal_pairs.len(),
            "Should not extract equality pairs where neither side is a Column"
        );

        // But col = literal should still be extracted
        let predicate: Arc<dyn PhysicalExpr> =
            binary(col("c2", &schema)?, Operator::Eq, lit(0u32), &schema)?;
        let (equal_pairs, _) = collect_columns_from_predicate_inner(&predicate);
        assert_eq!(
            1,
            equal_pairs.len(),
            "Should extract equality pairs where one side is a Column"
        );

        Ok(())
    }

    /// Columns with Absent min/max statistics should remain Absent after
    /// FilterExec.
    #[tokio::test]
    async fn test_filter_statistics_absent_columns_stay_absent() -> Result<()> {
        let schema = Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Int32, false),
        ]);
        let input = Arc::new(StatisticsExec::new(
            Statistics {
                num_rows: Precision::Inexact(1000),
                total_byte_size: Precision::Absent,
                column_statistics: vec![
                    ColumnStatistics::default(),
                    ColumnStatistics::default(),
                ],
            },
            schema.clone(),
        ));

        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("a", 0)),
            Operator::Eq,
            Arc::new(Literal::new(ScalarValue::Int32(Some(42)))),
        ));
        let filter: Arc<dyn ExecutionPlan> =
            Arc::new(FilterExec::try_new(predicate, input)?);

        let statistics = filter.partition_statistics(None)?;
        let col_b_stats = &statistics.column_statistics[1];
        assert_eq!(col_b_stats.min_value, Precision::Absent);
        assert_eq!(col_b_stats.max_value, Precision::Absent);

        Ok(())
    }

    /// Regression test: ProjectionExec on top of a FilterExec that already has
    /// an explicit projection must not panic when `try_swapping_with_projection`
    /// attempts to swap the two nodes.
    ///
    /// Before the fix, `FilterExecBuilder::from(self)` copied the old projection
    /// (e.g. `[0, 1, 2]`) from the FilterExec. After `.with_input` replaced the
    /// input with the narrower ProjectionExec (2 columns), `.build()` tried to
    /// validate the stale `[0, 1, 2]` projection against the 2-column schema and
    /// panicked with "project index 2 out of bounds, max field 2".
    #[test]
    fn test_filter_with_projection_swap_does_not_panic() -> Result<()> {
        use crate::projection::ProjectionExpr;
        use datafusion_physical_expr::expressions::col;

        // Schema: [ts: Int64, tokens: Int64, svc: Utf8]
        let schema = Arc::new(Schema::new(vec![
            Field::new("ts", DataType::Int64, false),
            Field::new("tokens", DataType::Int64, false),
            Field::new("svc", DataType::Utf8, false),
        ]));
        let input = Arc::new(EmptyExec::new(Arc::clone(&schema)));

        // FilterExec: ts > 0, projection=[ts@0, tokens@1, svc@2] (all 3 cols)
        let predicate = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("ts", 0)),
            Operator::Gt,
            Arc::new(Literal::new(ScalarValue::Int64(Some(0)))),
        ));
        let filter = Arc::new(
            FilterExecBuilder::new(predicate, input)
                .apply_projection(Some(vec![0, 1, 2]))?
                .build()?,
        );

        // ProjectionExec: narrows to [ts, tokens] (drops svc)
        let proj_exprs = vec![
            ProjectionExpr {
                expr: col("ts", &filter.schema())?,
                alias: "ts".to_string(),
            },
            ProjectionExpr {
                expr: col("tokens", &filter.schema())?,
                alias: "tokens".to_string(),
            },
        ];
        let projection = Arc::new(ProjectionExec::try_new(
            proj_exprs,
            Arc::clone(&filter) as _,
        )?);

        // This must not panic
        let result = filter.try_swapping_with_projection(&projection)?;
        assert!(result.is_some(), "swap should succeed");

        let new_plan = result.unwrap();
        // Output schema must still be [ts, tokens]
        let out_schema = new_plan.schema();
        assert_eq!(out_schema.fields().len(), 2);
        assert_eq!(out_schema.field(0).name(), "ts");
        assert_eq!(out_schema.field(1).name(), "tokens");

        Ok(())
    }
}