alamb commented on code in PR #15770:
URL: https://github.com/apache/datafusion/pull/15770#discussion_r2145790872


##########
datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs:
##########
@@ -0,0 +1,387 @@
+// 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::collections::HashMap;
+use std::sync::{Arc, LazyLock};
+
+use arrow::array::{Int32Array, StringArray, StringDictionaryBuilder};
+use arrow::datatypes::Int32Type;
+use arrow::record_batch::RecordBatch;
+use arrow::util::pretty::pretty_format_batches;
+use arrow_schema::{DataType, Field, Schema};
+use datafusion::datasource::listing::{ListingOptions, ListingTable, 
ListingTableConfig};
+use datafusion::prelude::{SessionConfig, SessionContext};
+use datafusion_datasource::ListingTableUrl;
+use datafusion_datasource_parquet::ParquetFormat;
+use datafusion_execution::object_store::ObjectStoreUrl;
+use itertools::Itertools;
+use object_store::memory::InMemory;
+use object_store::path::Path;
+use object_store::{ObjectStore, PutPayload};
+use parquet::arrow::ArrowWriter;
+use rand::rngs::StdRng;
+use rand::{Rng, SeedableRng};
+use tokio::sync::Mutex;
+use tokio::task::JoinSet;
+
+#[derive(Clone)]
+struct TestDataSet {
+    store: Arc<dyn ObjectStore>,
+    schema: Arc<Schema>,
+}
+
+/// List of in memory parquet files with UTF8 data
+// Use a mutex rather than LazyLock to allow for async initialization
+static TESTFILES: LazyLock<Mutex<Vec<TestDataSet>>> =
+    LazyLock::new(|| Mutex::new(vec![]));
+
+async fn test_files() -> Vec<TestDataSet> {
+    let files_mutex = &TESTFILES;
+    let mut files = files_mutex.lock().await;
+    if !files.is_empty() {
+        return (*files).clone();
+    }
+
+    let mut rng = StdRng::seed_from_u64(0);
+
+    for nulls_in_ids in [false, true] {
+        for nulls_in_names in [false, true] {
+            for nulls_in_departments in [false, true] {
+                let store = Arc::new(InMemory::new());
+
+                let schema = Arc::new(Schema::new(vec![
+                    Field::new("id", DataType::Int32, nulls_in_ids),
+                    Field::new("name", DataType::Utf8, nulls_in_names),
+                    Field::new(
+                        "department",
+                        DataType::Dictionary(
+                            Box::new(DataType::Int32),
+                            Box::new(DataType::Utf8),
+                        ),
+                        nulls_in_departments,
+                    ),
+                ]));
+
+                let name_choices = if nulls_in_names {
+                    [Some("Alice"), Some("Bob"), None, Some("David"), None]
+                } else {
+                    [
+                        Some("Alice"),
+                        Some("Bob"),
+                        Some("Charlie"),
+                        Some("David"),
+                        Some("Eve"),
+                    ]
+                };
+
+                let department_choices = if nulls_in_departments {
+                    [
+                        Some("Theater"),
+                        Some("Engineering"),
+                        None,
+                        Some("Arts"),
+                        None,
+                    ]
+                } else {
+                    [
+                        Some("Theater"),
+                        Some("Engineering"),
+                        Some("Healthcare"),
+                        Some("Arts"),
+                        Some("Music"),
+                    ]
+                };
+
+                // Generate 5 files, some with overlapping or repeated ids 
some without
+                for i in 0..5 {
+                    let num_batches = rng.random_range(1..3);
+                    let mut batches = Vec::with_capacity(num_batches);
+                    for _ in 0..num_batches {
+                        let num_rows = 25;
+                        let ids = 
Int32Array::from_iter((0..num_rows).map(|file| {
+                            if nulls_in_ids {
+                                if rng.random_bool(1.0 / 10.0) {
+                                    None
+                                } else {
+                                    Some(rng.random_range(file..file + 5))
+                                }
+                            } else {
+                                Some(rng.random_range(file..file + 5))
+                            }
+                        }));
+                        let names = 
StringArray::from_iter((0..num_rows).map(|_| {
+                            // randomly select a name
+                            let idx = rng.random_range(0..name_choices.len());
+                            name_choices[idx].map(|s| s.to_string())
+                        }));
+                        let mut departments = 
StringDictionaryBuilder::<Int32Type>::new();
+                        for _ in 0..num_rows {
+                            // randomly select a department
+                            let idx = 
rng.random_range(0..department_choices.len());
+                            
departments.append_option(department_choices[idx].as_ref());
+                        }
+                        let batch = RecordBatch::try_new(
+                            schema.clone(),
+                            vec![
+                                Arc::new(ids),
+                                Arc::new(names),
+                                Arc::new(departments.finish()),
+                            ],
+                        )
+                        .unwrap();
+                        batches.push(batch);
+                    }
+                    let mut buf = vec![];
+                    {
+                        let mut writer =
+                            ArrowWriter::try_new(&mut buf, schema.clone(), 
None).unwrap();
+                        for batch in batches {
+                            writer.write(&batch).unwrap();
+                            writer.flush().unwrap();
+                        }
+                        writer.flush().unwrap();
+                        writer.finish().unwrap();
+                    }
+                    let payload = PutPayload::from(buf);
+                    let path = Path::from(format!("file_{i}.parquet"));
+                    store.put(&path, payload).await.unwrap();
+                }
+                files.push(TestDataSet { store, schema });
+            }
+        }
+    }
+    (*files).clone()
+}
+
+struct RunResult {
+    results: Vec<RecordBatch>,
+    explain_plan: String,
+}
+
+async fn run_query_with_config(
+    query: &str,
+    config: SessionConfig,
+    dataset: TestDataSet,
+) -> RunResult {
+    let store = dataset.store;
+    let schema = dataset.schema;
+    let ctx = SessionContext::new_with_config(config);
+    let url = ObjectStoreUrl::parse("memory://").unwrap();
+    ctx.register_object_store(url.as_ref(), store.clone());
+
+    let format = Arc::new(
+        ParquetFormat::default()
+            .with_options(ctx.state().table_options().parquet.clone()),
+    );
+    let options = ListingOptions::new(format);
+    let table_path = ListingTableUrl::parse("memory:///").unwrap();
+    let config = ListingTableConfig::new(table_path)
+        .with_listing_options(options)
+        .with_schema(schema);
+    let table = Arc::new(ListingTable::try_new(config).unwrap());
+
+    ctx.register_table("test_table", table).unwrap();
+
+    let results = ctx.sql(query).await.unwrap().collect().await.unwrap();
+    let explain_batches = ctx
+        .sql(&format!("EXPLAIN ANALYZE {query}"))
+        .await
+        .unwrap()
+        .collect()
+        .await
+        .unwrap();
+    let explain_plan = 
pretty_format_batches(&explain_batches).unwrap().to_string();
+    RunResult {
+        results,
+        explain_plan,
+    }
+}
+
+#[derive(Debug)]
+struct RunQueryResult {
+    query: String,
+    result: Vec<RecordBatch>,
+    expected: Vec<RecordBatch>,
+}
+
+impl RunQueryResult {
+    fn expected_formated(&self) -> String {
+        format!("{}", pretty_format_batches(&self.expected).unwrap())
+    }
+
+    fn result_formated(&self) -> String {
+        format!("{}", pretty_format_batches(&self.result).unwrap())
+    }
+
+    fn is_ok(&self) -> bool {
+        self.expected_formated() == self.result_formated()
+    }
+}
+
+/// Iterate over each line in the plan and check that one of them has 
`DataSourceExec` and `DynamicFilterPhysicalExpr` in the same line.
+fn has_dynamic_filter_expr_pushdown(plan: &str) -> bool {
+    for line in plan.lines() {
+        if line.contains("DataSourceExec") && 
line.contains("DynamicFilterPhysicalExpr") {
+            return true;
+        }
+    }
+    false
+}
+
+async fn run_query(
+    query: String,
+    cfg: SessionConfig,
+    dataset: TestDataSet,
+) -> RunQueryResult {
+    let cfg_with_dynamic_filters = cfg
+        .clone()
+        .set_bool("datafusion.optimizer.enable_dynamic_filter_pushdown", true);
+    let cfg_without_dynamic_filters = cfg
+        .clone()
+        .set_bool("datafusion.optimizer.enable_dynamic_filter_pushdown", 
false);
+
+    let expected_result =
+        run_query_with_config(&query, cfg_without_dynamic_filters, 
dataset.clone()).await;
+    let result =
+        run_query_with_config(&query, cfg_with_dynamic_filters, 
dataset.clone()).await;
+    // Check that dynamic filters were actually pushed down
+    if !has_dynamic_filter_expr_pushdown(&result.explain_plan) {

Review Comment:
   nice



##########
datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs:
##########
@@ -346,6 +359,137 @@ fn test_node_handles_child_pushdown_result() {
     );
 }
 
+#[tokio::test]
+async fn test_topk_dynamic_filter_pushdown() {
+    // This test is a bit of a hack, but it shows that we can push down 
dynamic filters
+    // into the DataSourceExec. The test is not perfect because we don't have 
a real
+    // implementation of the dynamic filter yet, so we just use a static 
filter.

Review Comment:
   this comment seems maybe out of date as we now have a real implementation of 
the filter



##########
datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs:
##########
@@ -70,6 +71,8 @@ pub fn assign_initial_requirements(sort_push_down: &mut 
SortPushDown) {
             // If the parent has a fetch value, assign it to the children
             // Or use the fetch value of the child.
             fetch: child.plan.fetch(),
+            // If the parent has a filter, assign it to the children

Review Comment:
   I am surprised to see that filter pushdown is happening in "EnforceSorting" 
-- I would have expected it to happen in the `FilterPushdown` physical 
optimizer rule (which was added I thought for precisely this usecase)



##########
datafusion/physical-plan/src/topk/mod.rs:
##########
@@ -232,6 +248,105 @@ impl TopK {
         // which means the top K won't change and the computation can be 
finished early.
         self.attempt_early_completion(&batch)?;
 
+        if updated {
+            // update the filter representation of our TopK heap
+            self.update_filter()?;
+        }
+
+        Ok(())
+    }
+
+    /// Update the filter representation of our TopK heap

Review Comment:
   FWIW I recommend adding an example to this doc string showing what the 
filter expression would be for a multi-column sort order



##########
datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs:
##########
@@ -346,6 +359,137 @@ fn test_node_handles_child_pushdown_result() {
     );
 }
 
+#[tokio::test]
+async fn test_topk_dynamic_filter_pushdown() {
+    // This test is a bit of a hack, but it shows that we can push down 
dynamic filters
+    // into the DataSourceExec. The test is not perfect because we don't have 
a real
+    // implementation of the dynamic filter yet, so we just use a static 
filter.
+    let batches = vec![
+        record_batch!(
+            ("a", Utf8, ["aa", "ab"]),
+            ("b", Utf8, ["bd", "bc"]),
+            ("c", Float64, [1.0, 2.0])
+        )
+        .unwrap(),
+        record_batch!(
+            ("a", Utf8, ["ac", "ad"]),
+            ("b", Utf8, ["bb", "ba"]),
+            ("c", Float64, [2.0, 1.0])
+        )
+        .unwrap(),
+    ];
+    let scan = TestScanBuilder::new(schema())
+        .with_support(true)
+        .with_batches(batches)
+        .build();
+    let plan = Arc::new(
+        SortExec::new(
+            LexOrdering::new(vec![PhysicalSortExpr::new(
+                col("b", &schema()).unwrap(),
+                SortOptions::new(true, false), // descending, nulls_first
+            )])
+            .unwrap(),
+            Arc::clone(&scan),
+        )
+        .with_fetch(Some(1)),
+    ) as Arc<dyn ExecutionPlan>;
+
+    // expect the predicate to be pushed down into the DataSource
+    insta::assert_snapshot!(
+        OptimizationTest::new(Arc::clone(&plan), FilterPushdown{}, true),
+        @r"
+    OptimizationTest:
+      input:
+        - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], 
preserve_partitioning=[false]
+        -   DataSourceExec: file_groups={1 group: [[test.paqruet]]}, 
projection=[a, b, c], file_type=test, pushdown_supported=true

Review Comment:
   I don't know if we care but parquet is misspelled here: `test.paqruet`



##########
datafusion/physical-plan/src/topk/mod.rs:
##########
@@ -570,6 +680,47 @@ impl TopKHeap {
             + self.store.size()
             + self.owned_bytes
     }
+
+    fn get_threshold_values(
+        &self,
+        sort_exprs: &[PhysicalSortExpr],
+    ) -> Result<Option<Vec<ScalarValue>>> {
+        // If the heap doesn't have k elements yet, we can't create thresholds
+        let max_row = match self.max() {
+            Some(row) => row,
+            None => return Ok(None),
+        };
+
+        // Get the batch that contains the max row
+        let batch_entry = match self.store.get(max_row.batch_id) {
+            Some(entry) => entry,
+            None => return internal_err!("Invalid batch ID in TopKRow"),
+        };
+
+        // Extract threshold values for each sort expression
+        let mut scalar_values = Vec::with_capacity(sort_exprs.len());
+        for sort_expr in sort_exprs {

Review Comment:
   100ms is a LONG time -- I wouldn't expect this to ever take that long. I am 
not sure exactly what we would parallelize 🤔 



##########
datafusion/physical-plan/src/sorts/sort.rs:
##########
@@ -843,6 +846,8 @@ pub struct SortExec {
     common_sort_prefix: Vec<PhysicalSortExpr>,
     /// Cache holding plan properties like equivalences, output partitioning 
etc.
     cache: PlanProperties,
+    /// Filter matching the state of the sort for dynamic filter pushdown
+    filter: Option<Arc<DynamicFilterPhysicalExpr>>,

Review Comment:
   it would also be nice to add a note about when this is field is set or not



##########
datafusion/physical-plan/src/topk/mod.rs:
##########
@@ -232,6 +248,105 @@ impl TopK {
         // which means the top K won't change and the computation can be 
finished early.
         self.attempt_early_completion(&batch)?;
 
+        if updated {
+            // update the filter representation of our TopK heap
+            self.update_filter()?;
+        }
+
+        Ok(())
+    }
+
+    /// Update the filter representation of our TopK heap
+    fn update_filter(&mut self) -> Result<()> {
+        let Some(filter) = &self.filter else {
+            return Ok(());
+        };
+        if let Some(thresholds) = self.heap.get_threshold_values(&self.expr)? {

Review Comment:
   (minor) You could probably remove another level of indent here with the same 
pattern.
   
   ```suggestion
           let Some(thresholds) = self.heap.get_threshold_values(&self.expr)? 
else {
             return Ok(());
            }
   ```



##########
datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt:
##########
@@ -246,38 +246,3 @@ physical_plan
 02)--FilterExec: val@0 != part@1
 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
 04)------DataSourceExec: file_groups={1 group: 
[[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]},
 projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != 
c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d 
!= val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != 
val_max@1), required_guarantees=[val not in (c, d)]
-
-# The order of filters should not matter

Review Comment:
   why is this removed?



##########
datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs:
##########
@@ -114,6 +118,18 @@ fn pushdown_sorts_helper(
             sort_push_down.data.fetch = fetch;
             sort_push_down.data.ordering_requirement =
                 Some(OrderingRequirements::from(sort_ordering));
+            let filter = plan
+                .as_any()
+                .downcast_ref::<SortExec>()
+                .and_then(|s| s.filter().clone());

Review Comment:
   I think all we need to do is add a method to the `ExecutionPlan` like `fn 
dynamic_filters` and it will be general and well documented



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org
For additional commands, e-mail: github-h...@datafusion.apache.org

Reply via email to