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
// 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 parking_lot::RwLock;
use std::{any::Any, fmt::Display, hash::Hash, sync::Arc};
use tokio::sync::watch;
use crate::PhysicalExpr;
use arrow::datatypes::{DataType, Schema};
use datafusion_common::{
Result,
tree_node::{Transformed, TransformedResult, TreeNode},
};
use datafusion_expr::ColumnarValue;
use datafusion_physical_expr_common::physical_expr::DynHash;
/// State of a dynamic filter, tracking both updates and completion.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
enum FilterState {
/// Filter is in progress and may receive more updates.
InProgress { generation: u64 },
/// Filter is complete and will not receive further updates.
Complete { generation: u64 },
}
impl FilterState {
fn generation(&self) -> u64 {
match self {
FilterState::InProgress { generation }
| FilterState::Complete { generation } => *generation,
}
}
}
/// A dynamic [`PhysicalExpr`] that can be updated by anyone with a reference to it.
///
/// Any `ExecutionPlan` that uses this expression and holds a reference to it internally should probably also
/// implement `ExecutionPlan::reset_state` to remain compatible with recursive queries and other situations where
/// the same `ExecutionPlan` is reused with different data.
///
/// For more background, please also see the [Dynamic Filters: Passing Information Between Operators During Execution for 25x Faster Queries blog]
///
/// [Dynamic Filters: Passing Information Between Operators During Execution for 25x Faster Queries blog]: https://datafusion.apache.org/blog/2025/09/10/dynamic-filters
#[derive(Debug)]
pub struct DynamicFilterPhysicalExpr {
/// The original children of this PhysicalExpr, if any.
/// This is necessary because the dynamic filter may be initialized with a placeholder (e.g. `lit(true)`)
/// and later remapped to the actual expressions that are being filtered.
/// But we need to know the children (e.g. columns referenced in the expression) ahead of time to evaluate the expression correctly.
children: Vec<Arc<dyn PhysicalExpr>>,
/// If any of the children were remapped / modified (e.g. to adjust for projections) we need to keep track of the new children
/// so that when we update `current()` in subsequent iterations we can re-apply the replacements.
remapped_children: Option<Vec<Arc<dyn PhysicalExpr>>>,
/// The source of dynamic filters.
inner: Arc<RwLock<Inner>>,
/// Broadcasts filter state (updates and completion) to all waiters.
state_watch: watch::Sender<FilterState>,
/// For testing purposes track the data type and nullability to make sure they don't change.
/// If they do, there's a bug in the implementation.
/// But this can have overhead in production, so it's only included in our tests.
data_type: Arc<RwLock<Option<DataType>>>,
nullable: Arc<RwLock<Option<bool>>>,
}
#[derive(Debug)]
struct Inner {
/// A counter that gets incremented every time the expression is updated so that we can track changes cheaply.
/// This is used for [`PhysicalExpr::snapshot_generation`] to have a cheap check for changes.
generation: u64,
expr: Arc<dyn PhysicalExpr>,
/// Flag for quick synchronous check if filter is complete.
/// This is redundant with the watch channel state, but allows us to return immediately
/// from `wait_complete()` without subscribing if already complete.
is_complete: bool,
}
impl Inner {
fn new(expr: Arc<dyn PhysicalExpr>) -> Self {
Self {
// Start with generation 1 which gives us a different result for [`PhysicalExpr::generation`] than the default 0.
// This is not currently used anywhere but it seems useful to have this simple distinction.
generation: 1,
expr,
is_complete: false,
}
}
/// Clone the inner expression.
fn expr(&self) -> &Arc<dyn PhysicalExpr> {
&self.expr
}
}
impl Hash for DynamicFilterPhysicalExpr {
fn hash<H: std::hash::Hasher>(&self, state: &mut H) {
// Use pointer identity of the inner Arc for stable hashing.
// This is stable across update() calls and consistent with Eq.
// See issue #19641 for details on why content-based hashing violates
// the Hash/Eq contract when the underlying expression can change.
Arc::as_ptr(&self.inner).hash(state);
self.children.dyn_hash(state);
self.remapped_children.dyn_hash(state);
}
}
impl PartialEq for DynamicFilterPhysicalExpr {
fn eq(&self, other: &Self) -> bool {
// Two dynamic filters are equal if they share the same inner source
// AND have the same children configuration.
// This is consistent with Hash using Arc::as_ptr.
// See issue #19641 for details on the Hash/Eq contract violation fix.
Arc::ptr_eq(&self.inner, &other.inner)
&& self.children == other.children
&& self.remapped_children == other.remapped_children
}
}
impl Eq for DynamicFilterPhysicalExpr {}
impl Display for DynamicFilterPhysicalExpr {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
self.render(f, |expr, f| write!(f, "{expr}"))
}
}
impl DynamicFilterPhysicalExpr {
/// Create a new [`DynamicFilterPhysicalExpr`]
/// from an initial expression and a list of children.
/// The list of children is provided separately because
/// the initial expression may not have the same children.
/// For example, if the initial expression is just `true`
/// it will not reference any columns, but we may know that
/// we are going to replace this expression with a real one
/// that does reference certain columns.
/// In this case you **must** pass in the columns that will be
/// used in the final expression as children to this function
/// since DataFusion is generally not compatible with dynamic
/// *children* in expressions.
///
/// To determine the children you can:
///
/// - Use [`collect_columns`] to collect the columns from the expression.
/// - Use existing information, such as the sort columns in a `SortExec`.
///
/// Generally the important bit is that the *leaf children that reference columns
/// do not change* since those will be used to determine what columns need to read or projected
/// when evaluating the expression.
///
/// Any `ExecutionPlan` that uses this expression and holds a reference to it internally should probably also
/// implement `ExecutionPlan::reset_state` to remain compatible with recursive queries and other situations where
/// the same `ExecutionPlan` is reused with different data.
///
/// [`collect_columns`]: crate::utils::collect_columns
pub fn new(
children: Vec<Arc<dyn PhysicalExpr>>,
inner: Arc<dyn PhysicalExpr>,
) -> Self {
let (state_watch, _) = watch::channel(FilterState::InProgress { generation: 1 });
Self {
children,
remapped_children: None, // Initially no remapped children
inner: Arc::new(RwLock::new(Inner::new(inner))),
state_watch,
data_type: Arc::new(RwLock::new(None)),
nullable: Arc::new(RwLock::new(None)),
}
}
fn remap_children(
children: &[Arc<dyn PhysicalExpr>],
remapped_children: Option<&Vec<Arc<dyn PhysicalExpr>>>,
expr: Arc<dyn PhysicalExpr>,
) -> Result<Arc<dyn PhysicalExpr>> {
if let Some(remapped_children) = remapped_children {
// Remap the children to the new children
// of the expression.
expr.transform_up(|child| {
// Check if this is any of our original children
if let Some(pos) =
children.iter().position(|c| c.as_ref() == child.as_ref())
{
// If so, remap it to the current children
// of the expression.
let new_child = Arc::clone(&remapped_children[pos]);
Ok(Transformed::yes(new_child))
} else {
// Otherwise, just return the expression
Ok(Transformed::no(child))
}
})
.data()
} else {
// If we don't have any remapped children, just return the expression
Ok(Arc::clone(&expr))
}
}
/// Get the current generation of the expression.
fn current_generation(&self) -> u64 {
self.inner.read().generation
}
/// Get the current expression.
/// This will return the current expression with any children
/// remapped to match calls to [`PhysicalExpr::with_new_children`].
pub fn current(&self) -> Result<Arc<dyn PhysicalExpr>> {
let expr = Arc::clone(self.inner.read().expr());
Self::remap_children(&self.children, self.remapped_children.as_ref(), expr)
}
/// Update the current expression and notify all waiters.
/// Any children of this expression must be a subset of the original children
/// passed to the constructor.
/// This should be called e.g.:
/// - When we've computed the probe side's hash table in a HashJoinExec
/// - After every batch is processed if we update the TopK heap in a SortExec using a TopK approach.
pub fn update(&self, new_expr: Arc<dyn PhysicalExpr>) -> Result<()> {
// Remap the children of the new expression to match the original children
// We still do this again in `current()` but doing it preventively here
// reduces the work needed in some cases if `current()` is called multiple times
// and the same externally facing `PhysicalExpr` is used for both `with_new_children` and `update()`.`
let new_expr = Self::remap_children(
&self.children,
self.remapped_children.as_ref(),
new_expr,
)?;
// Load the current inner, increment generation, and store the new one
let mut current = self.inner.write();
let new_generation = current.generation + 1;
*current = Inner {
generation: new_generation,
expr: new_expr,
is_complete: current.is_complete,
};
drop(current); // Release the lock before broadcasting
// Broadcast the new state to all waiters
let _ = self.state_watch.send(FilterState::InProgress {
generation: new_generation,
});
Ok(())
}
/// Mark this dynamic filter as complete and broadcast to all waiters.
///
/// This signals that all expected updates have been received.
/// Waiters using [`Self::wait_complete`] will be notified.
pub fn mark_complete(&self) {
let mut current = self.inner.write();
let current_generation = current.generation;
current.is_complete = true;
drop(current);
// Broadcast completion to all waiters
let _ = self.state_watch.send(FilterState::Complete {
generation: current_generation,
});
}
/// Wait asynchronously for any update to this filter.
///
/// This method will return when [`Self::update`] is called and the generation increases.
/// It does not guarantee that the filter is complete.
///
/// Producers (e.g.) HashJoinExec may never update the expression or mark it as completed if there are no consumers.
/// If you call this method on a dynamic filter created by such a producer and there are no consumers registered this method would wait indefinitely.
/// This should not happen under normal operation and would indicate a programming error either in your producer or in DataFusion if the producer is a built in node.
pub async fn wait_update(&self) {
let mut rx = self.state_watch.subscribe();
// Get the current generation
let current_gen = rx.borrow_and_update().generation();
// Wait until generation increases
let _ = rx.wait_for(|state| state.generation() > current_gen).await;
}
/// Wait asynchronously until this dynamic filter is marked as complete.
///
/// This method returns immediately if the filter is already complete.
/// Otherwise, it waits until [`Self::mark_complete`] is called.
///
/// Unlike [`Self::wait_update`], this method guarantees that when it returns,
/// the filter is fully complete with no more updates expected.
///
/// Producers (e.g.) HashJoinExec may never update the expression or mark it as completed if there are no consumers.
/// If you call this method on a dynamic filter created by such a producer and there are no consumers registered this method would wait indefinitely.
/// This should not happen under normal operation and would indicate a programming error either in your producer or in DataFusion if the producer is a built in node.
pub async fn wait_complete(&self) {
if self.inner.read().is_complete {
return;
}
let mut rx = self.state_watch.subscribe();
let _ = rx
.wait_for(|state| matches!(state, FilterState::Complete { .. }))
.await;
}
/// Check if this dynamic filter is being actively used by any consumers.
///
/// Returns `true` if there are references beyond the producer (e.g., the HashJoinExec
/// that created the filter). This is useful to avoid computing expensive filter
/// expressions when no consumer will actually use them.
///
/// # Implementation Details
///
/// We check both Arc counts to handle two cases:
/// - Transformed filters (via `with_new_children`) share the inner Arc (inner count > 1)
/// - Direct clones (via `Arc::clone`) increment the outer count (outer count > 1)
pub fn is_used(self: &Arc<Self>) -> bool {
// Strong count > 1 means at least one consumer is holding a reference beyond the producer.
Arc::strong_count(self) > 1 || Arc::strong_count(&self.inner) > 1
}
fn render(
&self,
f: &mut std::fmt::Formatter<'_>,
render_expr: impl FnOnce(
Arc<dyn PhysicalExpr>,
&mut std::fmt::Formatter<'_>,
) -> std::fmt::Result,
) -> std::fmt::Result {
let inner = self.current().map_err(|_| std::fmt::Error)?;
let current_generation = self.current_generation();
write!(f, "DynamicFilter [ ")?;
if current_generation == 1 {
write!(f, "empty")?;
} else {
render_expr(inner, f)?;
}
write!(f, " ]")
}
}
impl PhysicalExpr for DynamicFilterPhysicalExpr {
fn as_any(&self) -> &dyn Any {
self
}
fn children(&self) -> Vec<&Arc<dyn PhysicalExpr>> {
self.remapped_children
.as_ref()
.unwrap_or(&self.children)
.iter()
.collect()
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn PhysicalExpr>>,
) -> Result<Arc<dyn PhysicalExpr>> {
Ok(Arc::new(Self {
children: self.children.clone(),
remapped_children: Some(children),
inner: Arc::clone(&self.inner),
state_watch: self.state_watch.clone(),
data_type: Arc::clone(&self.data_type),
nullable: Arc::clone(&self.nullable),
}))
}
fn data_type(&self, input_schema: &Schema) -> Result<DataType> {
let res = self.current()?.data_type(input_schema)?;
#[cfg(test)]
{
use datafusion_common::internal_err;
// Check if the data type has changed.
let mut data_type_lock = self.data_type.write();
if let Some(existing) = &*data_type_lock {
if existing != &res {
// If the data type has changed, we have a bug.
return internal_err!(
"DynamicFilterPhysicalExpr data type has changed unexpectedly. \
Expected: {existing:?}, Actual: {res:?}"
);
}
} else {
*data_type_lock = Some(res.clone());
}
}
Ok(res)
}
fn nullable(&self, input_schema: &Schema) -> Result<bool> {
let res = self.current()?.nullable(input_schema)?;
#[cfg(test)]
{
use datafusion_common::internal_err;
// Check if the nullability has changed.
let mut nullable_lock = self.nullable.write();
if let Some(existing) = *nullable_lock {
if existing != res {
// If the nullability has changed, we have a bug.
return internal_err!(
"DynamicFilterPhysicalExpr nullability has changed unexpectedly. \
Expected: {existing}, Actual: {res}"
);
}
} else {
*nullable_lock = Some(res);
}
}
Ok(res)
}
fn evaluate(
&self,
batch: &arrow::record_batch::RecordBatch,
) -> Result<ColumnarValue> {
let current = self.current()?;
#[cfg(test)]
{
// Ensure that we are not evaluating after the expression has changed.
let schema = batch.schema();
self.nullable(&schema)?;
self.data_type(&schema)?;
};
current.evaluate(batch)
}
fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
self.render(f, |expr, f| expr.fmt_sql(f))
}
fn snapshot(&self) -> Result<Option<Arc<dyn PhysicalExpr>>> {
// Return the current expression as a snapshot.
Ok(Some(self.current()?))
}
fn snapshot_generation(&self) -> u64 {
// Return the current generation of the expression.
self.inner.read().generation
}
}
#[cfg(test)]
mod test {
use crate::{
expressions::{BinaryExpr, col, lit},
utils::reassign_expr_columns,
};
use arrow::{
array::RecordBatch,
datatypes::{DataType, Field, Schema},
};
use datafusion_common::ScalarValue;
use super::*;
#[test]
fn test_remap_children() {
let table_schema = Arc::new(Schema::new(vec![
Field::new("a", DataType::Int32, false),
Field::new("b", DataType::Int32, false),
]));
let expr = Arc::new(BinaryExpr::new(
col("a", &table_schema).unwrap(),
datafusion_expr::Operator::Eq,
lit(42) as Arc<dyn PhysicalExpr>,
));
let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new(
vec![col("a", &table_schema).unwrap()],
expr as Arc<dyn PhysicalExpr>,
));
// Simulate two `ParquetSource` files with different filter schemas
// Both of these should hit the same inner `PhysicalExpr` even after `update()` is called
// and be able to remap children independently.
let filter_schema_1 = Arc::new(Schema::new(vec![
Field::new("a", DataType::Int32, false),
Field::new("b", DataType::Int32, false),
]));
let filter_schema_2 = Arc::new(Schema::new(vec![
Field::new("b", DataType::Int32, false),
Field::new("a", DataType::Int32, false),
]));
// Each ParquetExec calls `with_new_children` on the DynamicFilterPhysicalExpr
// and remaps the children to the file schema.
let dynamic_filter_1 = reassign_expr_columns(
Arc::clone(&dynamic_filter) as Arc<dyn PhysicalExpr>,
&filter_schema_1,
)
.unwrap();
let snap = dynamic_filter_1.snapshot().unwrap().unwrap();
insta::assert_snapshot!(format!("{snap:?}"), @r#"BinaryExpr { left: Column { name: "a", index: 0 }, op: Eq, right: Literal { value: Int32(42), field: Field { name: "lit", data_type: Int32 } }, fail_on_overflow: false }"#);
let dynamic_filter_2 = reassign_expr_columns(
Arc::clone(&dynamic_filter) as Arc<dyn PhysicalExpr>,
&filter_schema_2,
)
.unwrap();
let snap = dynamic_filter_2.snapshot().unwrap().unwrap();
insta::assert_snapshot!(format!("{snap:?}"), @r#"BinaryExpr { left: Column { name: "a", index: 1 }, op: Eq, right: Literal { value: Int32(42), field: Field { name: "lit", data_type: Int32 } }, fail_on_overflow: false }"#);
// Both filters allow evaluating the same expression
let batch_1 = RecordBatch::try_new(
Arc::clone(&filter_schema_1),
vec![
// a
ScalarValue::Int32(Some(42)).to_array_of_size(1).unwrap(),
// b
ScalarValue::Int32(Some(43)).to_array_of_size(1).unwrap(),
],
)
.unwrap();
let batch_2 = RecordBatch::try_new(
Arc::clone(&filter_schema_2),
vec![
// b
ScalarValue::Int32(Some(43)).to_array_of_size(1).unwrap(),
// a
ScalarValue::Int32(Some(42)).to_array_of_size(1).unwrap(),
],
)
.unwrap();
// Evaluate the expression on both batches
let result_1 = dynamic_filter_1.evaluate(&batch_1).unwrap();
let result_2 = dynamic_filter_2.evaluate(&batch_2).unwrap();
// Check that the results are the same
let ColumnarValue::Array(arr_1) = result_1 else {
panic!("Expected ColumnarValue::Array");
};
let ColumnarValue::Array(arr_2) = result_2 else {
panic!("Expected ColumnarValue::Array");
};
assert!(arr_1.eq(&arr_2));
let expected = ScalarValue::Boolean(Some(true))
.to_array_of_size(1)
.unwrap();
assert!(arr_1.eq(&expected));
// Now lets update the expression
// Note that we update the *original* expression and that should be reflected in both the derived expressions
let new_expr = Arc::new(BinaryExpr::new(
col("a", &table_schema).unwrap(),
datafusion_expr::Operator::Gt,
lit(43) as Arc<dyn PhysicalExpr>,
));
dynamic_filter
.update(Arc::clone(&new_expr) as Arc<dyn PhysicalExpr>)
.expect("Failed to update expression");
// Now we should be able to evaluate the new expression on both batches
let result_1 = dynamic_filter_1.evaluate(&batch_1).unwrap();
let result_2 = dynamic_filter_2.evaluate(&batch_2).unwrap();
// Check that the results are the same
let ColumnarValue::Array(arr_1) = result_1 else {
panic!("Expected ColumnarValue::Array");
};
let ColumnarValue::Array(arr_2) = result_2 else {
panic!("Expected ColumnarValue::Array");
};
assert!(arr_1.eq(&arr_2));
let expected = ScalarValue::Boolean(Some(false))
.to_array_of_size(1)
.unwrap();
assert!(arr_1.eq(&expected));
}
#[test]
fn test_snapshot() {
let expr = lit(42) as Arc<dyn PhysicalExpr>;
let dynamic_filter = DynamicFilterPhysicalExpr::new(vec![], Arc::clone(&expr));
// Take a snapshot of the current expression
let snapshot = dynamic_filter.snapshot().unwrap();
assert_eq!(snapshot, Some(expr));
// Update the current expression
let new_expr = lit(100) as Arc<dyn PhysicalExpr>;
dynamic_filter.update(Arc::clone(&new_expr)).unwrap();
// Take another snapshot
let snapshot = dynamic_filter.snapshot().unwrap();
assert_eq!(snapshot, Some(new_expr));
}
#[test]
fn test_dynamic_filter_physical_expr_misbehaves_data_type_nullable() {
let dynamic_filter =
DynamicFilterPhysicalExpr::new(vec![], lit(42) as Arc<dyn PhysicalExpr>);
// First call to data_type and nullable should set the initial values.
let initial_data_type = dynamic_filter.data_type(&Schema::empty()).unwrap();
let initial_nullable = dynamic_filter.nullable(&Schema::empty()).unwrap();
// Call again and expect no change.
let second_data_type = dynamic_filter.data_type(&Schema::empty()).unwrap();
let second_nullable = dynamic_filter.nullable(&Schema::empty()).unwrap();
assert_eq!(
initial_data_type, second_data_type,
"Data type should not change on second call."
);
assert_eq!(
initial_nullable, second_nullable,
"Nullability should not change on second call."
);
// Now change the current expression to something else.
dynamic_filter
.update(lit(ScalarValue::Utf8(None)) as Arc<dyn PhysicalExpr>)
.expect("Failed to update expression");
// Check that we error if we call data_type, nullable or evaluate after changing the expression.
assert!(
dynamic_filter.data_type(&Schema::empty()).is_err(),
"Expected err when data_type is called after changing the expression."
);
assert!(
dynamic_filter.nullable(&Schema::empty()).is_err(),
"Expected err when nullable is called after changing the expression."
);
let batch = RecordBatch::new_empty(Arc::new(Schema::empty()));
assert!(
dynamic_filter.evaluate(&batch).is_err(),
"Expected err when evaluate is called after changing the expression."
);
}
#[tokio::test]
async fn test_wait_complete_already_complete() {
let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new(
vec![],
lit(42) as Arc<dyn PhysicalExpr>,
));
// Mark as complete immediately
dynamic_filter.mark_complete();
// wait_complete should return immediately
dynamic_filter.wait_complete().await;
}
#[test]
fn test_with_new_children_independence() {
// Create a schema with columns a, b, c, d
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),
]));
// Create expression col(a) + col(b)
let col_a = col("a", &schema).unwrap();
let col_b = col("b", &schema).unwrap();
let col_c = col("c", &schema).unwrap();
let col_d = col("d", &schema).unwrap();
let expr = Arc::new(BinaryExpr::new(
Arc::clone(&col_a),
datafusion_expr::Operator::Plus,
Arc::clone(&col_b),
));
// Create DynamicFilterPhysicalExpr with children [col_a, col_b]
let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new(
vec![Arc::clone(&col_a), Arc::clone(&col_b)],
expr as Arc<dyn PhysicalExpr>,
));
// Clone the Arc (two references to the same DynamicFilterPhysicalExpr)
let clone_1 = Arc::clone(&dynamic_filter);
let clone_2 = Arc::clone(&dynamic_filter);
// Call with_new_children with different children on each clone
// clone_1: replace [a, b] with [b, c] -> expression becomes b + c
let remapped_1 = clone_1
.with_new_children(vec![Arc::clone(&col_b), Arc::clone(&col_c)])
.unwrap();
// clone_2: replace [a, b] with [b, d] -> expression becomes b + d
let remapped_2 = clone_2
.with_new_children(vec![Arc::clone(&col_b), Arc::clone(&col_d)])
.unwrap();
// Create a RecordBatch with columns a=1,2,3 b=10,20,30 c=100,200,300 d=1000,2000,3000
let batch = RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(arrow::array::Int32Array::from(vec![1, 2, 3])), // a
Arc::new(arrow::array::Int32Array::from(vec![10, 20, 30])), // b
Arc::new(arrow::array::Int32Array::from(vec![100, 200, 300])), // c
Arc::new(arrow::array::Int32Array::from(vec![1000, 2000, 3000])), // d
],
)
.unwrap();
// Evaluate both remapped expressions
let result_1 = remapped_1.evaluate(&batch).unwrap();
let result_2 = remapped_2.evaluate(&batch).unwrap();
// Extract arrays from results
let ColumnarValue::Array(arr_1) = result_1 else {
panic!("Expected ColumnarValue::Array for result_1");
};
let ColumnarValue::Array(arr_2) = result_2 else {
panic!("Expected ColumnarValue::Array for result_2");
};
// Verify result_1 = b + c = [110, 220, 330]
let expected_1: Arc<dyn arrow::array::Array> =
Arc::new(arrow::array::Int32Array::from(vec![110, 220, 330]));
assert!(
arr_1.eq(&expected_1),
"Expected b + c = [110, 220, 330], got {arr_1:?}",
);
// Verify result_2 = b + d = [1010, 2020, 3030]
let expected_2: Arc<dyn arrow::array::Array> =
Arc::new(arrow::array::Int32Array::from(vec![1010, 2020, 3030]));
assert!(
arr_2.eq(&expected_2),
"Expected b + d = [1010, 2020, 3030], got {arr_2:?}",
);
}
#[test]
fn test_is_used() {
let filter = Arc::new(DynamicFilterPhysicalExpr::new(
vec![],
lit(true) as Arc<dyn PhysicalExpr>,
));
// Initially, only one reference to the inner Arc exists
assert!(
!filter.is_used(),
"Filter should not be used with only one inner reference"
);
// Simulate a consumer created via transformation (what happens during filter pushdown).
// When filters are pushed down and transformed via reassign_expr_columns/transform_down,
// with_new_children() is called which creates a new outer Arc but clones the inner Arc.
let consumer1_expr = Arc::clone(&filter).with_new_children(vec![]).unwrap();
let _consumer1 = consumer1_expr
.as_any()
.downcast_ref::<DynamicFilterPhysicalExpr>()
.expect("Should be DynamicFilterPhysicalExpr");
// Now the inner Arc is shared (inner_count = 2)
assert!(
filter.is_used(),
"Filter should be used when inner Arc is shared with transformed consumer"
);
// Create another transformed consumer
let consumer2_expr = Arc::clone(&filter).with_new_children(vec![]).unwrap();
let _consumer2 = consumer2_expr
.as_any()
.downcast_ref::<DynamicFilterPhysicalExpr>()
.expect("Should be DynamicFilterPhysicalExpr");
assert!(
filter.is_used(),
"Filter should still be used with multiple consumers"
);
}
/// Test that verifies the Hash/Eq contract is now satisfied (issue #19641 fix).
///
/// After the fix, Hash uses Arc::as_ptr(&self.inner) which is stable across
/// update() calls, fixing the HashMap key instability issue.
#[test]
fn test_hash_stable_after_update() {
use std::collections::hash_map::DefaultHasher;
use std::hash::{Hash, Hasher};
// Create filter with initial value
let filter =
DynamicFilterPhysicalExpr::new(vec![], lit(true) as Arc<dyn PhysicalExpr>);
// Compute hash BEFORE update
let mut hasher_before = DefaultHasher::new();
filter.hash(&mut hasher_before);
let hash_before = hasher_before.finish();
// Update changes the underlying expression
filter
.update(lit(false) as Arc<dyn PhysicalExpr>)
.expect("Update should succeed");
// Compute hash AFTER update
let mut hasher_after = DefaultHasher::new();
filter.hash(&mut hasher_after);
let hash_after = hasher_after.finish();
// FIXED: Hash should now be STABLE after update() because we use
// Arc::as_ptr for identity-based hashing instead of expression content.
assert_eq!(
hash_before, hash_after,
"Hash should be stable after update() - fix for issue #19641"
);
// Self-equality should still hold
assert!(filter.eq(&filter), "Self-equality should hold");
}
/// Test that verifies separate DynamicFilterPhysicalExpr instances
/// with the same expression are NOT equal (identity-based comparison).
#[test]
fn test_identity_based_equality() {
// Create two separate filters with identical initial expressions
let filter1 =
DynamicFilterPhysicalExpr::new(vec![], lit(true) as Arc<dyn PhysicalExpr>);
let filter2 =
DynamicFilterPhysicalExpr::new(vec![], lit(true) as Arc<dyn PhysicalExpr>);
// Different instances should NOT be equal even with same expression
// because they have independent inner Arcs (different update lifecycles)
assert!(
!filter1.eq(&filter2),
"Different instances should not be equal (identity-based)"
);
// Self-equality should hold
assert!(filter1.eq(&filter1), "Self-equality should hold");
}
/// Test that hash is stable for the same filter instance.
/// After the fix, hash uses Arc::as_ptr which is pointer-based.
#[test]
fn test_hash_stable_for_same_instance() {
use std::collections::hash_map::DefaultHasher;
use std::hash::{Hash, Hasher};
let filter =
DynamicFilterPhysicalExpr::new(vec![], lit(true) as Arc<dyn PhysicalExpr>);
// Compute hash twice for the same instance
let hash1 = {
let mut h = DefaultHasher::new();
filter.hash(&mut h);
h.finish()
};
let hash2 = {
let mut h = DefaultHasher::new();
filter.hash(&mut h);
h.finish()
};
assert_eq!(hash1, hash2, "Same instance should have stable hash");
// Update the expression
filter
.update(lit(false) as Arc<dyn PhysicalExpr>)
.expect("Update should succeed");
// Hash should STILL be the same (identity-based)
let hash3 = {
let mut h = DefaultHasher::new();
filter.hash(&mut h);
h.finish()
};
assert_eq!(
hash1, hash3,
"Hash should be stable after update (identity-based)"
);
}
}