-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Push down InList or hash table references from HashJoinExec depending on the size of the build side #18393
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Push down InList or hash table references from HashJoinExec depending on the size of the build side #18393
Changes from all commits
83f7dd3
26a87d7
968b5b6
b417a41
b8d0a16
7dc44e4
dea8261
9892741
f8bc4ed
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,8 +26,9 @@ use crate::filter_pushdown::{ | |
| ChildPushdownResult, FilterDescription, FilterPushdownPhase, | ||
| FilterPushdownPropagation, | ||
| }; | ||
| use crate::joins::hash_join::inlist_builder::build_struct_inlist_values; | ||
| use crate::joins::hash_join::shared_bounds::{ | ||
| ColumnBounds, PartitionBounds, SharedBuildAccumulator, | ||
| ColumnBounds, PartitionBounds, PushdownStrategy, SharedBuildAccumulator, | ||
| }; | ||
| use crate::joins::hash_join::stream::{ | ||
| BuildSide, BuildSideInitialState, HashJoinStream, HashJoinStreamState, | ||
|
|
@@ -86,7 +87,7 @@ use futures::TryStreamExt; | |
| use parking_lot::Mutex; | ||
|
|
||
| /// Hard-coded seed to ensure hash values from the hash join differ from `RepartitionExec`, avoiding collisions. | ||
| const HASH_JOIN_SEED: RandomState = | ||
| pub(crate) const HASH_JOIN_SEED: RandomState = | ||
| RandomState::with_seeds('J' as u64, 'O' as u64, 'I' as u64, 'N' as u64); | ||
|
|
||
| /// HashTable and input data for the left (build side) of a join | ||
|
|
@@ -112,6 +113,9 @@ pub(super) struct JoinLeftData { | |
| /// If the partition is empty (no rows) this will be None. | ||
| /// If the partition has some rows this will be Some with the bounds for each join key column. | ||
| pub(super) bounds: Option<PartitionBounds>, | ||
| /// Membership testing strategy for filter pushdown | ||
| /// Contains either InList values for small build sides or hash table reference for large build sides | ||
| pub(super) membership: PushdownStrategy, | ||
| } | ||
|
|
||
| impl JoinLeftData { | ||
|
|
@@ -135,6 +139,11 @@ impl JoinLeftData { | |
| &self.visited_indices_bitmap | ||
| } | ||
|
|
||
| /// returns a reference to the InList values for filter pushdown | ||
| pub(super) fn membership(&self) -> &PushdownStrategy { | ||
| &self.membership | ||
| } | ||
|
|
||
| /// Decrements the counter of running threads, and returns `true` | ||
| /// if caller is the last running thread | ||
| pub(super) fn report_probe_completed(&self) -> bool { | ||
|
|
@@ -929,6 +938,16 @@ impl ExecutionPlan for HashJoinExec { | |
| need_produce_result_in_final(self.join_type), | ||
| self.right().output_partitioning().partition_count(), | ||
| enable_dynamic_filter_pushdown, | ||
| context | ||
| .session_config() | ||
| .options() | ||
| .optimizer | ||
| .hash_join_inlist_pushdown_max_size, | ||
| context | ||
| .session_config() | ||
| .options() | ||
| .optimizer | ||
| .hash_join_inlist_pushdown_max_distinct_values, | ||
| )) | ||
| })?, | ||
| PartitionMode::Partitioned => { | ||
|
|
@@ -947,6 +966,16 @@ impl ExecutionPlan for HashJoinExec { | |
| need_produce_result_in_final(self.join_type), | ||
| 1, | ||
| enable_dynamic_filter_pushdown, | ||
| context | ||
| .session_config() | ||
| .options() | ||
| .optimizer | ||
| .hash_join_inlist_pushdown_max_size, | ||
| context | ||
| .session_config() | ||
| .options() | ||
| .optimizer | ||
| .hash_join_inlist_pushdown_max_distinct_values, | ||
| )) | ||
| } | ||
| PartitionMode::Auto => { | ||
|
|
@@ -1346,6 +1375,8 @@ async fn collect_left_input( | |
| with_visited_indices_bitmap: bool, | ||
| probe_threads_count: usize, | ||
| should_compute_dynamic_filters: bool, | ||
| max_inlist_size: usize, | ||
| max_inlist_distinct_values: usize, | ||
| ) -> Result<JoinLeftData> { | ||
| let schema = left_stream.schema(); | ||
|
|
||
|
|
@@ -1469,6 +1500,29 @@ async fn collect_left_input( | |
| // Convert Box to Arc for sharing with SharedBuildAccumulator | ||
| let hash_map: Arc<dyn JoinHashMapType> = hashmap.into(); | ||
|
|
||
| let membership = if num_rows == 0 { | ||
| PushdownStrategy::Empty | ||
| } else { | ||
| // If the build side is small enough we can use IN list pushdown. | ||
| // If it's too big we fall back to pushing down a reference to the hash table. | ||
| // See `PushdownStrategy` for more details. | ||
| let estimated_size = left_values | ||
| .iter() | ||
| .map(|arr| arr.get_array_memory_size()) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we have a query like
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Makes sense. But maybe it's still okay since we would end up duplicating the values in the InListExpr? Either way like you say I think it's not a big deal, it's just a ballpark estimate...
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, since |
||
| .sum::<usize>(); | ||
| if left_values.is_empty() | ||
| || left_values[0].is_empty() | ||
| || estimated_size > max_inlist_size | ||
| || hash_map.len() > max_inlist_distinct_values | ||
| { | ||
| PushdownStrategy::HashTable(Arc::clone(&hash_map)) | ||
| } else if let Some(in_list_values) = build_struct_inlist_values(&left_values)? { | ||
| PushdownStrategy::InList(in_list_values) | ||
| } else { | ||
| PushdownStrategy::HashTable(Arc::clone(&hash_map)) | ||
| } | ||
| }; | ||
|
|
||
| let data = JoinLeftData { | ||
| hash_map, | ||
| batch, | ||
|
|
@@ -1477,6 +1531,7 @@ async fn collect_left_input( | |
| probe_threads_counter: AtomicUsize::new(probe_threads_count), | ||
| _reservation: reservation, | ||
| bounds, | ||
| membership, | ||
| }; | ||
|
|
||
| Ok(data) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,133 @@ | ||
| // 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. | ||
|
|
||
| //! Utilities for building InList expressions from hash join build side data | ||
|
|
||
| use std::sync::Arc; | ||
|
|
||
| use arrow::array::{ArrayRef, StructArray}; | ||
| use arrow::datatypes::{Field, FieldRef, Fields}; | ||
| use arrow::downcast_dictionary_array; | ||
| use arrow_schema::DataType; | ||
| use datafusion_common::Result; | ||
|
|
||
| pub(super) fn build_struct_fields(data_types: &[DataType]) -> Result<Fields> { | ||
| data_types | ||
| .iter() | ||
| .enumerate() | ||
| .map(|(i, dt)| Ok(Field::new(format!("c{i}"), dt.clone(), true))) | ||
| .collect() | ||
| } | ||
|
|
||
| /// Flattens dictionary-encoded arrays to their underlying value arrays. | ||
| /// Non-dictionary arrays are returned as-is. | ||
| fn flatten_dictionary_array(array: &ArrayRef) -> ArrayRef { | ||
| downcast_dictionary_array! { | ||
| array => { | ||
| // Recursively flatten in case of nested dictionaries | ||
| flatten_dictionary_array(array.values()) | ||
| } | ||
| _ => Arc::clone(array) | ||
| } | ||
| } | ||
|
|
||
| /// Builds InList values from join key column arrays. | ||
| /// | ||
| /// If `join_key_arrays` is: | ||
| /// 1. A single array, let's say Int32, this will produce a flat | ||
| /// InList expression where the lookup is expected to be scalar Int32 values, | ||
| /// that is: this will produce `IN LIST (1, 2, 3)` expected to be used as `2 IN LIST (1, 2, 3)`. | ||
| /// 2. An Int32 array and a Utf8 array, this will produce a Struct InList expression | ||
| /// where the lookup is expected to be Struct values with two fields (Int32, Utf8), | ||
| /// that is: this will produce `IN LIST ((1, "a"), (2, "b"))` expected to be used as `(2, "b") IN LIST ((1, "a"), (2, "b"))`. | ||
| /// The field names of the struct are auto-generated as "c0", "c1", ... and should match the struct expression used in the join keys. | ||
| /// | ||
| /// Note that this function does not deduplicate values - deduplication will happen later | ||
| /// when building an InList expression from this array via `InListExpr::try_new_from_array`. | ||
| /// | ||
| /// Returns `None` if the estimated size exceeds `max_size_bytes` or if the number of rows | ||
| /// exceeds `max_distinct_values`. | ||
| pub(super) fn build_struct_inlist_values( | ||
| join_key_arrays: &[ArrayRef], | ||
| ) -> Result<Option<ArrayRef>> { | ||
| // Flatten any dictionary-encoded arrays | ||
| let flattened_arrays: Vec<ArrayRef> = join_key_arrays | ||
| .iter() | ||
| .map(flatten_dictionary_array) | ||
| .collect(); | ||
|
|
||
| // Build the source array/struct | ||
| let source_array: ArrayRef = if flattened_arrays.len() == 1 { | ||
| // Single column: use directly | ||
| Arc::clone(&flattened_arrays[0]) | ||
| } else { | ||
| // Multi-column: build StructArray once from all columns | ||
| let fields = build_struct_fields( | ||
| &flattened_arrays | ||
| .iter() | ||
| .map(|arr| arr.data_type().clone()) | ||
| .collect::<Vec<_>>(), | ||
| )?; | ||
|
|
||
| // Build field references with proper Arc wrapping | ||
| let arrays_with_fields: Vec<(FieldRef, ArrayRef)> = fields | ||
| .iter() | ||
| .cloned() | ||
| .zip(flattened_arrays.iter().cloned()) | ||
| .collect(); | ||
|
|
||
| Arc::new(StructArray::from(arrays_with_fields)) | ||
| }; | ||
|
|
||
| Ok(Some(source_array)) | ||
| } | ||
|
|
||
| #[cfg(test)] | ||
| mod tests { | ||
| use super::*; | ||
| use arrow::array::{Int32Array, StringArray}; | ||
| use arrow_schema::DataType; | ||
| use std::sync::Arc; | ||
|
|
||
| #[test] | ||
| fn test_build_single_column_inlist_array() { | ||
| let array = Arc::new(Int32Array::from(vec![1, 2, 3, 2, 1])) as ArrayRef; | ||
| let result = build_struct_inlist_values(std::slice::from_ref(&array)) | ||
| .unwrap() | ||
| .unwrap(); | ||
|
|
||
| assert!(array.eq(&result)); | ||
| } | ||
|
|
||
| #[test] | ||
| fn test_build_multi_column_inlist() { | ||
| let array1 = Arc::new(Int32Array::from(vec![1, 2, 3, 2, 1])) as ArrayRef; | ||
| let array2 = | ||
| Arc::new(StringArray::from(vec!["a", "b", "c", "b", "a"])) as ArrayRef; | ||
|
|
||
| let result = build_struct_inlist_values(&[array1, array2]) | ||
| .unwrap() | ||
| .unwrap(); | ||
|
|
||
| assert_eq!( | ||
| *result.data_type(), | ||
| DataType::Struct( | ||
| build_struct_fields(&[DataType::Int32, DataType::Utf8]).unwrap() | ||
| ) | ||
| ); | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe a future improvement could be to also expose an option to limit the number of distinct values that can be inside an IN LIST. For instance, we could end up with a very large list like
x IN (1, 2, 3, ..., 1000000)that fits in 128KB but is still inefficient because we'd be duplicating values and performance might decrease.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I got this idea from trino:
https://trino.io/docs/current/admin/dynamic-filtering.html#dynamic-filter-collection-thresholds
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you elaborate? In my mind a large InList is not any more or less efficient than pushing down the hash table itself, but if it's big it looses access to the bloom filter pushdown optimization so it's probably not faster than the hash table itself. That said there are still reasons to push it down instead, namely that custom execution nodes that downcast match on a PhysicalExpr can recognize it.
So the idea with the 128kB is to balance how much CPU we burn upfront building the filter. But I agree it could be in terms of rows as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added a config
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So if the
IN LISTis very large, it loses the pruning advantage on the probe side - like you say, bloom filters becoming ineffective with that many values, so once we lose that, we might as well use the hash table instead and avoid having to copy the data from theIN LISTNot sure if I'm missing anything here, anyways this is just a thought and it would be better to check this with the benchmarks.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah that's the idea. As the build side gets larger:
So at some point it makes sense to cut the losses and go through each row with the hash table.
The
InListExprapproach is going to shine when there is a point lookup type query (i.e. one row from the build side) that can hit a bloom filter on the probe side.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Naive question: is the size (in bits) of the bloom filter tunable at the moment?
In principle you could use NDV to tune its size at build time, and lift the limitation on the number of elements (within a reasonable limit, of course).
I think the PR is already very useful as is, this could be tackled in a follow-up PR if the point raised makes sense (possibly another addition to the list of places where NDV can help, as tracked in #18628)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
🤔 As a parquet write propery, I don't think it can be tunned, you can control some write properties like NDV or FPP which in theory help control the size of the bloom filter as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah I think there may be a bit of confusion:
col IN (1, 2, 3)which inside of pruning logic gets converted tocol = 1 or col = 2 or col = 3as long as there are <20 elements in the list, and thencol = 1gets evaluated against any bloom filters oncol, thencol = 2, etc. for each row group.