-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
filter.rs
1248 lines (1149 loc) · 46.2 KB
/
filter.rs
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
// 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::{ready, Context, Poll};
use super::{
ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties,
RecordBatchStream, SendableRecordBatchStream, Statistics,
};
use crate::common::can_project;
use crate::{
metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet},
DisplayFormatType, ExecutionPlan,
};
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::stats::Precision;
use datafusion_common::{
internal_err, plan_err, project_schema, DataFusionError, Result,
};
use datafusion_execution::TaskContext;
use datafusion_expr::Operator;
use datafusion_physical_expr::equivalence::ProjectionMapping;
use datafusion_physical_expr::expressions::BinaryExpr;
use datafusion_physical_expr::intervals::utils::check_support;
use datafusion_physical_expr::utils::collect_columns;
use datafusion_physical_expr::{
analyze, split_conjunction, AnalysisContext, ConstExpr, ExprBoundaries, PhysicalExpr,
};
use crate::execution_plan::CardinalityEffect;
use futures::stream::{Stream, StreamExt};
use log::trace;
/// 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: PlanProperties,
/// The projection indices of the columns in the output schema of join
projection: Option<Vec<usize>>,
}
impl FilterExec {
/// Create a FilterExec on an input
pub fn try_new(
predicate: Arc<dyn PhysicalExpr>,
input: Arc<dyn ExecutionPlan>,
) -> Result<Self> {
match predicate.data_type(input.schema().as_ref())? {
DataType::Boolean => {
let default_selectivity = 20;
let cache = Self::compute_properties(
&input,
&predicate,
default_selectivity,
None,
)?;
Ok(Self {
predicate,
input: Arc::clone(&input),
metrics: ExecutionPlanMetricsSet::new(),
default_selectivity,
cache,
projection: None,
})
}
other => {
plan_err!("Filter predicate must return BOOLEAN values, got {other:?}")
}
}
}
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.
pub fn with_projection(&self, projection: Option<Vec<usize>>) -> Result<Self> {
// Check if the projection is valid
can_project(&self.schema(), projection.as_ref())?;
let projection = match projection {
Some(projection) => match &self.projection {
Some(p) => Some(projection.iter().map(|i| p[*i]).collect()),
None => Some(projection),
},
None => None,
};
let cache = Self::compute_properties(
&self.input,
&self.predicate,
self.default_selectivity,
projection.as_ref(),
)?;
Ok(Self {
predicate: Arc::clone(&self.predicate),
input: Arc::clone(&self.input),
metrics: self.metrics.clone(),
default_selectivity: self.default_selectivity,
cache,
projection,
})
}
/// 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<&Vec<usize>> {
self.projection.as_ref()
}
/// Calculates `Statistics` for `FilterExec`, by applying selectivity (either default, or estimated) to input statistics.
fn statistics_helper(
input: &Arc<dyn ExecutionPlan>,
predicate: &Arc<dyn PhysicalExpr>,
default_selectivity: u8,
) -> Result<Statistics> {
let input_stats = input.statistics()?;
let schema = input.schema();
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(
&input.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(
&input_stats.column_statistics,
analysis_ctx.boundaries,
);
Ok(Statistics {
num_rows,
total_byte_size,
column_statistics,
})
}
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>() {
if binary.op() == &Operator::Eq {
// Filter evaluates to single value for all partitions
if input_eqs.is_expr_constant(binary.left()) {
res_constants.push(
ConstExpr::from(binary.right()).with_across_partitions(true),
)
} else if input_eqs.is_expr_constant(binary.right()) {
res_constants.push(
ConstExpr::from(binary.left()).with_across_partitions(true),
)
}
}
}
}
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<&Vec<usize>>,
) -> Result<PlanProperties> {
// Combine the equal predicates with the input equivalence properties
// to construct the equivalence properties:
let stats = Self::statistics_helper(input, predicate, default_selectivity)?;
let mut eq_properties = input.equivalence_properties().clone();
let (equal_pairs, _) = collect_columns_from_predicate(predicate);
for (lhs, rhs) in equal_pairs {
eq_properties.add_equal_conditions(lhs, 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 expr = Arc::new(column) as _;
ConstExpr::new(expr).with_across_partitions(true)
});
// This is for statistics
eq_properties = eq_properties.with_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 =
eq_properties.with_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.execution_mode(),
))
}
}
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()
};
write!(f, "FilterExec: {}{}", self.predicate, display_projections)
}
}
}
}
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) -> &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>> {
FilterExec::try_new(Arc::clone(&self.predicate), children.swap_remove(0))
.and_then(|e| {
let selectivity = e.default_selectivity();
e.with_default_selectivity(selectivity)
})
.and_then(|e| e.with_projection(self.projection().cloned()))
.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 baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
Ok(Box::pin(FilterExecStream {
schema: self.schema(),
predicate: Arc::clone(&self.predicate),
input: self.input.execute(partition, context)?,
baseline_metrics,
projection: self.projection.clone(),
}))
}
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 statistics(&self) -> Result<Statistics> {
let stats = Self::statistics_helper(
&self.input,
self.predicate(),
self.default_selectivity,
)?;
Ok(stats.project(self.projection.as_ref()))
}
fn cardinality_effect(&self) -> CardinalityEffect {
CardinalityEffect::LowerEqual
}
}
/// 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(
input_column_stats: &[ColumnStatistics],
analysis_boundaries: Vec<ExprBoundaries>,
) -> Vec<ColumnStatistics> {
analysis_boundaries
.into_iter()
.enumerate()
.map(
|(
idx,
ExprBoundaries {
interval,
distinct_count,
..
},
)| {
let (lower, upper) = interval.into_bounds();
let (min_value, max_value) = if lower.eq(&upper) {
(Precision::Exact(lower), Precision::Exact(upper))
} else {
(Precision::Inexact(lower), Precision::Inexact(upper))
};
ColumnStatistics {
null_count: input_column_stats[idx].null_count.to_inexact(),
max_value,
min_value,
distinct_count: distinct_count.to_inexact(),
}
},
)
.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
baseline_metrics: BaselineMetrics,
/// The projection indices of the columns in the input schema
projection: Option<Vec<usize>>,
}
pub fn batch_filter(
batch: &RecordBatch,
predicate: &Arc<dyn PhysicalExpr>,
) -> Result<RecordBatch> {
filter_and_project(batch, predicate, None, &batch.schema())
}
fn filter_and_project(
batch: &RecordBatch,
predicate: &Arc<dyn PhysicalExpr>,
projection: Option<&Vec<usize>>,
output_schema: &SchemaRef,
) -> 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_columns = projection
.iter()
.map(|i| Arc::clone(batch.column(*i)))
.collect();
let projected_batch = RecordBatch::try_new(
Arc::clone(output_schema),
projected_columns,
)?;
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 poll;
loop {
match ready!(self.input.poll_next_unpin(cx)) {
Some(Ok(batch)) => {
let timer = self.baseline_metrics.elapsed_compute().timer();
let filtered_batch = filter_and_project(
&batch,
&self.predicate,
self.projection.as_ref(),
&self.schema,
)?;
timer.done();
// Skip entirely filtered batches
if filtered_batch.num_rows() == 0 {
continue;
}
poll = Poll::Ready(Some(Ok(filtered_batch)));
break;
}
value => {
poll = Poll::Ready(value);
break;
}
}
}
self.baseline_metrics.record_poll(poll)
}
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
fn collect_columns_from_predicate(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>() {
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};
use arrow_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(&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.statistics()?;
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.statistics()?;
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.statistics()?;
// 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.statistics()?;
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.statistics()?;
// 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.statistics()?.column_statistics;
let filter: Arc<dyn ExecutionPlan> =
Arc::new(FilterExec::try_new(predicate, input)?);
let statistics = filter.statistics()?;
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()
},
],